1
0

[HUDI-2161] Adding support to disable meta columns with bulk insert operation (#3247)

This commit is contained in:
Sivabalan Narayanan
2021-07-19 20:43:48 -04:00
committed by GitHub
parent 2099bf41db
commit d5026e9a24
53 changed files with 1063 additions and 269 deletions

View File

@@ -18,15 +18,25 @@
package org.apache.hudi.internal;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieRowCreateHandle;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.io.storage.row.HoodieRowCreateHandleWithoutMetaFields;
import org.apache.hudi.io.storage.row.HoodieRowCreateHandle;
import org.apache.hudi.keygen.BuiltinKeyGenerator;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.StructType;
import java.io.IOException;
@@ -34,6 +44,7 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.UUID;
/**
@@ -52,15 +63,20 @@ public class BulkInsertDataInternalWriterHelper {
private final StructType structType;
private final Boolean arePartitionRecordsSorted;
private final List<HoodieInternalWriteStatus> writeStatusList = new ArrayList<>();
private HoodieRowCreateHandle handle;
private String lastKnownPartitionPath = null;
private String fileIdPrefix;
private int numFilesWritten = 0;
private Map<String, HoodieRowCreateHandle> handles = new HashMap<>();
private final boolean populateMetaFields;
private Option<BuiltinKeyGenerator> keyGeneratorOpt = null;
private boolean simpleKeyGen = false;
private int simplePartitionFieldIndex = -1;
private DataType simplePartitionFieldDataType;
public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType, boolean arePartitionRecordsSorted) {
String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType,
boolean populateMetaFields, boolean arePartitionRecordsSorted) {
this.hoodieTable = hoodieTable;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
@@ -68,14 +84,57 @@ public class BulkInsertDataInternalWriterHelper {
this.taskId = taskId;
this.taskEpochId = taskEpochId;
this.structType = structType;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.fileIdPrefix = UUID.randomUUID().toString();
if (!populateMetaFields) {
this.keyGeneratorOpt = getKeyGenerator(writeConfig.getProps());
if (keyGeneratorOpt.isPresent() && keyGeneratorOpt.get() instanceof SimpleKeyGenerator) {
simpleKeyGen = true;
simplePartitionFieldIndex = (Integer) structType.getFieldIndex((keyGeneratorOpt.get()).getPartitionPathFields().get(0)).get();
simplePartitionFieldDataType = structType.fields()[simplePartitionFieldIndex].dataType();
}
}
}
/**
* Instantiate {@link BuiltinKeyGenerator}.
*
* @param properties properties map.
* @return the key generator thus instantiated.
*/
private Option<BuiltinKeyGenerator> getKeyGenerator(Properties properties) {
TypedProperties typedProperties = new TypedProperties();
typedProperties.putAll(properties);
if (properties.get(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key()).equals(NonpartitionedKeyGenerator.class.getName())) {
return Option.empty(); // Do not instantiate NonPartitionKeyGen
} else {
try {
return Option.of((BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(typedProperties));
} catch (ClassCastException cce) {
throw new HoodieIOException("Only those key generators implementing BuiltInKeyGenerator interface is supported with virtual keys");
} catch (IOException e) {
throw new HoodieIOException("Key generator instantiation failed ", e);
}
}
}
public void write(InternalRow record) throws IOException {
try {
String partitionPath = record.getUTF8String(
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString();
String partitionPath = null;
if (populateMetaFields) { // usual path where meta fields are pre populated in prep step.
partitionPath = record.getUTF8String(
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString();
} else { // if meta columns are disabled.
if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen
partitionPath = "";
} else if (simpleKeyGen) { // SimpleKeyGen
partitionPath = (record.get(simplePartitionFieldIndex, simplePartitionFieldDataType)).toString();
} else {
// only BuiltIn key generators are supported if meta fields are disabled.
partitionPath = keyGeneratorOpt.get().getPartitionPath(record, structType);
}
}
if ((lastKnownPartitionPath == null) || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) {
LOG.info("Creating new file for partition path " + partitionPath);
@@ -103,20 +162,24 @@ public class BulkInsertDataInternalWriterHelper {
if (arePartitionRecordsSorted) {
close();
}
handles.put(partitionPath, new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
instantTime, taskPartitionId, taskId, taskEpochId, structType));
HoodieRowCreateHandle rowCreateHandle = populateMetaFields ? new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
instantTime, taskPartitionId, taskId, taskEpochId, structType) : new HoodieRowCreateHandleWithoutMetaFields(hoodieTable, writeConfig, partitionPath, getNextFileId(),
instantTime, taskPartitionId, taskId, taskEpochId, structType);
handles.put(partitionPath, rowCreateHandle);
} else if (!handles.get(partitionPath).canWrite()) {
// even if there is a handle to the partition path, it could have reached its max size threshold. So, we close the handle here and
// create a new one.
writeStatusList.add(handles.remove(partitionPath).close());
handles.put(partitionPath, new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
instantTime, taskPartitionId, taskId, taskEpochId, structType));
HoodieRowCreateHandle rowCreateHandle = populateMetaFields ? new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(),
instantTime, taskPartitionId, taskId, taskEpochId, structType) : new HoodieRowCreateHandleWithoutMetaFields(hoodieTable, writeConfig, partitionPath, getNextFileId(),
instantTime, taskPartitionId, taskId, taskEpochId, structType);
handles.put(partitionPath, rowCreateHandle);
}
return handles.get(partitionPath);
}
public void close() throws IOException {
for (HoodieRowCreateHandle rowCreateHandle: handles.values()) {
for (HoodieRowCreateHandle rowCreateHandle : handles.values()) {
writeStatusList.add(rowCreateHandle.close());
}
handles.clear();

View File

@@ -66,6 +66,7 @@ public class DataSourceInternalWriterHelper {
writeClient.startCommitWithTime(instantTime);
this.metaClient = HoodieTableMetaClient.builder().setConf(configuration).setBasePath(writeConfig.getBasePath()).build();
this.metaClient.validateTableProperties(writeConfig.getProps(), WriteOperationType.BULK_INSERT);
this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient);
}

View File

@@ -1,89 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hudi
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
import org.apache.spark.sql.execution.datasources.SparkParsePartitionUtil
import org.apache.spark.sql.internal.SQLConf
/**
* An interface to adapter the difference between spark2 and spark3
* in some spark related class.
*/
trait SparkAdapter extends Serializable {
/**
* Create the SparkRowSerDe.
*/
def createSparkRowSerDe(encoder: ExpressionEncoder[Row]): SparkRowSerDe
/**
* Convert a AliasIdentifier to TableIdentifier.
*/
def toTableIdentify(aliasId: AliasIdentifier): TableIdentifier
/**
* Convert a UnresolvedRelation to TableIdentifier.
*/
def toTableIdentify(relation: UnresolvedRelation): TableIdentifier
/**
* Create Join logical plan.
*/
def createJoin(left: LogicalPlan, right: LogicalPlan, joinType: JoinType): Join
/**
* Test if the logical plan is a Insert Into LogicalPlan.
*/
def isInsertInto(plan: LogicalPlan): Boolean
/**
* Get the member of the Insert Into LogicalPlan.
*/
def getInsertIntoChildren(plan: LogicalPlan):
Option[(LogicalPlan, Map[String, Option[String]], LogicalPlan, Boolean, Boolean)]
/**
* Create a Insert Into LogicalPlan.
*/
def createInsertInto(table: LogicalPlan, partition: Map[String, Option[String]],
query: LogicalPlan, overwrite: Boolean, ifPartitionNotExists: Boolean): LogicalPlan
/**
* Create the hoodie's extended spark sql parser.
*/
def createExtendedSparkParser: Option[(SparkSession, ParserInterface) => ParserInterface] = None
/**
* Create the SparkParsePartitionUtil.
*/
def createSparkParsePartitionUtil(conf: SQLConf): SparkParsePartitionUtil
/**
* Create Like expression.
*/
def createLike(left: Expression, right: Expression): Expression
}

View File

@@ -18,12 +18,17 @@
package org.apache.hudi.internal;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.SparkDatasetTestUtils;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
@@ -33,8 +38,10 @@ import org.junit.jupiter.api.BeforeEach;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Random;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -62,6 +69,17 @@ public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarn
cleanupResources();
}
protected HoodieWriteConfig getWriteConfig(boolean populateMetaFields) {
Properties properties = new Properties();
if (!populateMetaFields) {
properties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(), SimpleKeyGenerator.class.getName());
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), SparkDatasetTestUtils.RECORD_KEY_FIELD_NAME);
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), SparkDatasetTestUtils.PARTITION_PATH_FIELD_NAME);
properties.setProperty(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), "false");
}
return getConfigBuilder(basePath).withProperties(properties).build();
}
protected void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size,
Option<List<String>> fileAbsPaths, Option<List<String>> fileNames) {
assertWriteStatuses(writeStatuses, batches, size, false, fileAbsPaths, fileNames);
@@ -126,21 +144,27 @@ public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarn
}
}
protected void assertOutput(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, Option<List<String>> fileNames) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)));
if (fileNames.isPresent()) {
assertTrue(fileNames.get().contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS
.get(HoodieRecord.FILENAME_METADATA_FIELD))));
}
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
});
protected void assertOutput(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, Option<List<String>> fileNames,
boolean populateMetaColumns) {
if (populateMetaColumns) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)));
if (fileNames.isPresent()) {
assertTrue(fileNames.get().contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS
.get(HoodieRecord.FILENAME_METADATA_FIELD))));
}
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
});
// after trimming 2 of the meta fields, rest of the fields should match
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
assertEquals(0, trimmedActual.except(trimmedExpected).count());
// after trimming 2 of the meta fields, rest of the fields should match
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
assertEquals(0, trimmedActual.except(trimmedExpected).count());
} else { // operation = BULK_INSERT_APPEND_ONLY
// all meta columns are untouched
assertEquals(0, expectedRows.except(actualRows).count());
}
}
}