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());
}
}
}

View File

@@ -18,12 +18,6 @@
package org.apache.hudi;
import static org.apache.spark.sql.functions.callUDF;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.ReflectionUtils;
@@ -41,8 +35,17 @@ import org.apache.spark.sql.api.java.UDF1;
import org.apache.spark.sql.functions;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.StructType;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import scala.collection.JavaConverters;
import static org.apache.spark.sql.functions.callUDF;
/**
* Helper class to assist in preparing {@link Dataset<Row>}s for bulk insert with datasource implementation.
*/
@@ -112,4 +115,40 @@ public class HoodieDatasetBulkInsertHelper {
return bulkInsertPartitionerRows.repartitionRecords(colOrderedDataset, config.getBulkInsertShuffleParallelism());
}
/**
* Add empty meta fields and reorder such that meta fields are at the beginning.
*
* @param rows
* @return
*/
public static Dataset<Row> prepareHoodieDatasetForBulkInsertWithoutMetaFields(Dataset<Row> rows) {
// add empty meta cols.
Dataset<Row> rowsWithMetaCols = rows
.withColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD,
functions.lit("").cast(DataTypes.StringType))
.withColumn(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
functions.lit("").cast(DataTypes.StringType))
.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD,
functions.lit("").cast(DataTypes.StringType))
.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD,
functions.lit("").cast(DataTypes.StringType))
.withColumn(HoodieRecord.FILENAME_METADATA_FIELD,
functions.lit("").cast(DataTypes.StringType));
List<Column> originalFields =
Arrays.stream(rowsWithMetaCols.schema().fields()).filter(field -> !field.name().contains("_hoodie_")).map(f -> new Column(f.name())).collect(Collectors.toList());
List<Column> metaFields =
Arrays.stream(rowsWithMetaCols.schema().fields()).filter(field -> field.name().contains("_hoodie_")).map(f -> new Column(f.name())).collect(Collectors.toList());
// reorder such that all meta columns are at the beginning followed by original columns
List<Column> allCols = new ArrayList<>();
allCols.addAll(metaFields);
allCols.addAll(originalFields);
return rowsWithMetaCols.select(
JavaConverters.collectionAsScalaIterableConverter(allCols).asScala().toSeq());
}
}

View File

@@ -32,7 +32,7 @@ import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP}
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerWithRowsFactory
import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRowsFactory, NonSortPartitionerWithRows}
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
import org.apache.hudi.index.SparkHoodieIndex
import org.apache.hudi.internal.DataSourceInternalWriterHelper
@@ -128,6 +128,7 @@ object HoodieSparkSqlWriter {
.setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS_OPT_KEY))
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
.setPartitionColumns(partitionColumns)
.setPopulateMetaFields(parameters.getOrElse(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()).toBoolean)
.initTable(sparkContext.hadoopConfiguration, path.get)
tableConfig = tableMetaClient.getTableConfig
}
@@ -139,7 +140,8 @@ object HoodieSparkSqlWriter {
if (hoodieConfig.getBoolean(ENABLE_ROW_WRITER_OPT_KEY) &&
operation == WriteOperationType.BULK_INSERT) {
val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName,
basePath, path, instantTime)
basePath, path, instantTime, parameters.getOrElse(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(),
HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()).toBoolean)
return (success, commitTime, common.util.Option.empty(), common.util.Option.empty(), hoodieWriteClient.orNull, tableConfig)
}
// scalastyle:on
@@ -330,7 +332,8 @@ object HoodieSparkSqlWriter {
tblName: String,
basePath: Path,
path: Option[String],
instantTime: String): (Boolean, common.util.Option[String]) = {
instantTime: String,
populateMetaFields: Boolean): (Boolean, common.util.Option[String]) = {
val sparkContext = sqlContext.sparkContext
// register classes & schemas
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName)
@@ -345,22 +348,36 @@ object HoodieSparkSqlWriter {
}
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA.key, schema.toString)
val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
val bulkInsertPartitionerRows : BulkInsertPartitioner[Dataset[Row]] = if (userDefinedBulkInsertPartitionerOpt.isPresent) {
userDefinedBulkInsertPartitionerOpt.get
}
else {
BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig.getBulkInsertSortMode)
val bulkInsertPartitionerRows : BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) {
val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
if (userDefinedBulkInsertPartitionerOpt.isPresent) {
userDefinedBulkInsertPartitionerOpt.get
}
else {
BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig.getBulkInsertSortMode)
}
} else {
// Sort modes are not yet supported when meta fields are disabled
new NonSortPartitionerWithRows()
}
val arePartitionRecordsSorted = bulkInsertPartitionerRows.arePartitionRecordsSorted();
parameters.updated(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, arePartitionRecordsSorted.toString)
val isGlobalIndex = SparkHoodieIndex.isGlobalIndex(writeConfig)
val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace,
bulkInsertPartitionerRows, isGlobalIndex)
val isGlobalIndex = if (populateMetaFields) {
SparkHoodieIndex.isGlobalIndex(writeConfig)
} else {
false
}
val hoodieDF = if (populateMetaFields) {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace,
bulkInsertPartitionerRows, isGlobalIndex)
} else {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsertWithoutMetaFields(df)
}
if (SPARK_VERSION.startsWith("2.")) {
hoodieDF.write.format("org.apache.hudi.internal")
.option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
.options(params)
.mode(SaveMode.Append)
.save()
} else if (SPARK_VERSION.startsWith("3.")) {
hoodieDF.write.format("org.apache.hudi.spark3.internal")

View File

@@ -1,206 +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.hudi
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hudi.common.model.HoodieRecord
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{DataFrame, SparkSession}
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal}
import org.apache.spark.sql.sources.{And, EqualNullSafe, EqualTo, Filter, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Not, Or, StringContains, StringEndsWith, StringStartsWith}
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFileIndex}
import org.apache.spark.sql.types.{StringType, StructField, StructType}
import scala.collection.JavaConverters._
object HoodieSparkUtils extends SparkAdapterSupport {
def getMetaSchema: StructType = {
StructType(HoodieRecord.HOODIE_META_COLUMNS.asScala.map(col => {
StructField(col, StringType, nullable = true)
}))
}
/**
* This method copied from [[org.apache.spark.deploy.SparkHadoopUtil]].
* [[org.apache.spark.deploy.SparkHadoopUtil]] becomes private since Spark 3.0.0 and hence we had to copy it locally.
*/
def isGlobPath(pattern: Path): Boolean = {
pattern.toString.exists("{}[]*?\\".toSet.contains)
}
/**
* This method copied from [[org.apache.spark.deploy.SparkHadoopUtil]].
* [[org.apache.spark.deploy.SparkHadoopUtil]] becomes private since Spark 3.0.0 and hence we had to copy it locally.
*/
def globPath(fs: FileSystem, pattern: Path): Seq[Path] = {
Option(fs.globStatus(pattern)).map { statuses =>
statuses.map(_.getPath.makeQualified(fs.getUri, fs.getWorkingDirectory)).toSeq
}.getOrElse(Seq.empty[Path])
}
/**
* This method copied from [[org.apache.spark.deploy.SparkHadoopUtil]].
* [[org.apache.spark.deploy.SparkHadoopUtil]] becomes private since Spark 3.0.0 and hence we had to copy it locally.
*/
def globPathIfNecessary(fs: FileSystem, pattern: Path): Seq[Path] = {
if (isGlobPath(pattern)) globPath(fs, pattern) else Seq(pattern)
}
/**
* Checks to see whether input path contains a glob pattern and if yes, maps it to a list of absolute paths
* which match the glob pattern. Otherwise, returns original path
*
* @param paths List of absolute or globbed paths
* @param fs File system
* @return list of absolute file paths
*/
def checkAndGlobPathIfNecessary(paths: Seq[String], fs: FileSystem): Seq[Path] = {
paths.flatMap(path => {
val qualified = new Path(path).makeQualified(fs.getUri, fs.getWorkingDirectory)
val globPaths = globPathIfNecessary(fs, qualified)
globPaths
})
}
def createInMemoryFileIndex(sparkSession: SparkSession, globbedPaths: Seq[Path]): InMemoryFileIndex = {
val fileStatusCache = FileStatusCache.getOrCreate(sparkSession)
new InMemoryFileIndex(sparkSession, globbedPaths, Map(), Option.empty, fileStatusCache)
}
def createRdd(df: DataFrame, structName: String, recordNamespace: String): RDD[GenericRecord] = {
val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, recordNamespace)
createRdd(df, avroSchema, structName, recordNamespace)
}
def createRdd(df: DataFrame, avroSchema: Schema, structName: String, recordNamespace: String)
: RDD[GenericRecord] = {
// Use the Avro schema to derive the StructType which has the correct nullability information
val dataType = SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType]
val encoder = RowEncoder.apply(dataType).resolveAndBind()
val deserializer = sparkAdapter.createSparkRowSerDe(encoder)
df.queryExecution.toRdd.map(row => deserializer.deserializeRow(row))
.mapPartitions { records =>
if (records.isEmpty) Iterator.empty
else {
val convertor = AvroConversionHelper.createConverterToAvro(dataType, structName, recordNamespace)
records.map { x => convertor(x).asInstanceOf[GenericRecord] }
}
}
}
/**
* Convert Filters to Catalyst Expressions and joined by And. If convert success return an
* Non-Empty Option[Expression],or else return None.
*/
def convertToCatalystExpressions(filters: Array[Filter],
tableSchema: StructType): Option[Expression] = {
val expressions = filters.map(convertToCatalystExpression(_, tableSchema))
if (expressions.forall(p => p.isDefined)) {
if (expressions.isEmpty) {
None
} else if (expressions.length == 1) {
expressions(0)
} else {
Some(expressions.map(_.get).reduce(org.apache.spark.sql.catalyst.expressions.And))
}
} else {
None
}
}
/**
* Convert Filter to Catalyst Expression. If convert success return an Non-Empty
* Option[Expression],or else return None.
*/
def convertToCatalystExpression(filter: Filter, tableSchema: StructType): Option[Expression] = {
Option(
filter match {
case EqualTo(attribute, value) =>
org.apache.spark.sql.catalyst.expressions.EqualTo(toAttribute(attribute, tableSchema), Literal.create(value))
case EqualNullSafe(attribute, value) =>
org.apache.spark.sql.catalyst.expressions.EqualNullSafe(toAttribute(attribute, tableSchema), Literal.create(value))
case GreaterThan(attribute, value) =>
org.apache.spark.sql.catalyst.expressions.GreaterThan(toAttribute(attribute, tableSchema), Literal.create(value))
case GreaterThanOrEqual(attribute, value) =>
org.apache.spark.sql.catalyst.expressions.GreaterThanOrEqual(toAttribute(attribute, tableSchema), Literal.create(value))
case LessThan(attribute, value) =>
org.apache.spark.sql.catalyst.expressions.LessThan(toAttribute(attribute, tableSchema), Literal.create(value))
case LessThanOrEqual(attribute, value) =>
org.apache.spark.sql.catalyst.expressions.LessThanOrEqual(toAttribute(attribute, tableSchema), Literal.create(value))
case In(attribute, values) =>
val attrExp = toAttribute(attribute, tableSchema)
val valuesExp = values.map(v => Literal.create(v))
org.apache.spark.sql.catalyst.expressions.In(attrExp, valuesExp)
case IsNull(attribute) =>
org.apache.spark.sql.catalyst.expressions.IsNull(toAttribute(attribute, tableSchema))
case IsNotNull(attribute) =>
org.apache.spark.sql.catalyst.expressions.IsNotNull(toAttribute(attribute, tableSchema))
case And(left, right) =>
val leftExp = convertToCatalystExpression(left, tableSchema)
val rightExp = convertToCatalystExpression(right, tableSchema)
if (leftExp.isEmpty || rightExp.isEmpty) {
null
} else {
org.apache.spark.sql.catalyst.expressions.And(leftExp.get, rightExp.get)
}
case Or(left, right) =>
val leftExp = convertToCatalystExpression(left, tableSchema)
val rightExp = convertToCatalystExpression(right, tableSchema)
if (leftExp.isEmpty || rightExp.isEmpty) {
null
} else {
org.apache.spark.sql.catalyst.expressions.Or(leftExp.get, rightExp.get)
}
case Not(child) =>
val childExp = convertToCatalystExpression(child, tableSchema)
if (childExp.isEmpty) {
null
} else {
org.apache.spark.sql.catalyst.expressions.Not(childExp.get)
}
case StringStartsWith(attribute, value) =>
val leftExp = toAttribute(attribute, tableSchema)
val rightExp = Literal.create(s"$value%")
sparkAdapter.createLike(leftExp, rightExp)
case StringEndsWith(attribute, value) =>
val leftExp = toAttribute(attribute, tableSchema)
val rightExp = Literal.create(s"%$value")
sparkAdapter.createLike(leftExp, rightExp)
case StringContains(attribute, value) =>
val leftExp = toAttribute(attribute, tableSchema)
val rightExp = Literal.create(s"%$value%")
sparkAdapter.createLike(leftExp, rightExp)
case _=> null
}
)
}
private def toAttribute(columnName: String, tableSchema: StructType): AttributeReference = {
val field = tableSchema.find(p => p.name == columnName)
assert(field.isDefined, s"Cannot find column: $columnName, Table Columns are: " +
s"${tableSchema.fieldNames.mkString(",")}")
AttributeReference(columnName, field.get.dataType, field.get.nullable)()
}
}

View File

@@ -1,37 +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.hudi
import org.apache.spark.sql.hudi.{HoodieSqlUtils, SparkAdapter}
/**
* Use the SparkAdapterSupport trait to get the SparkAdapter when we
* need to adapt the difference between spark2 and spark3.
*/
trait SparkAdapterSupport {
lazy val sparkAdapter: SparkAdapter = {
val adapterClass = if (HoodieSqlUtils.isSpark3) {
"org.apache.spark.sql.adapter.Spark3Adapter"
} else {
"org.apache.spark.sql.adapter.Spark2Adapter"
}
getClass.getClassLoader.loadClass(adapterClass)
.newInstance().asInstanceOf[SparkAdapter]
}
}

View File

@@ -113,6 +113,33 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
assertTrue(dataset.except(trimmedOutput).count() == 0);
}
@Test
public void testBulkInsertHelperNoMetaFields() {
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsertWithoutMetaFields(dataset);
StructType resultSchema = result.schema();
assertEquals(result.count(), 10);
assertEquals(resultSchema.fieldNames().length, structType.fieldNames().length + HoodieRecord.HOODIE_META_COLUMNS.size());
for (Map.Entry<String, Integer> entry : HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.entrySet()) {
assertTrue(resultSchema.fieldIndex(entry.getKey()) == entry.getValue());
}
result.toJavaRDD().foreach(entry -> {
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD)).equals(""));
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).equals(""));
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)).equals(""));
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).equals(""));
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.FILENAME_METADATA_FIELD)).equals(""));
});
Dataset<Row> trimmedOutput = result.drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD)
.drop(HoodieRecord.FILENAME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
assertTrue(dataset.except(trimmedOutput).count() == 0);
}
@ParameterizedTest
@MethodSource("providePreCombineArgs")
public void testBulkInsertPreCombine(boolean enablePreCombine) {

View File

@@ -26,6 +26,7 @@ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.client.{SparkRDDWriteClient, TestBootstrap}
import org.apache.hudi.common.config.HoodieConfig
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord, HoodieRecordPayload}
import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
import org.apache.hudi.exception.HoodieException
@@ -119,61 +120,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
}
}
List(BulkInsertSortMode.GLOBAL_SORT.name(), BulkInsertSortMode.NONE.name(), BulkInsertSortMode.PARTITION_SORT.name())
List(BulkInsertSortMode.GLOBAL_SORT, BulkInsertSortMode.NONE, BulkInsertSortMode.PARTITION_SORT)
.foreach(sortMode => {
test("test_bulk_insert_for_" + sortMode) {
initSparkContext("test_bulk_insert_datasource")
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
try {
val hoodieFooTableName = "hoodie_foo_tbl"
//create a new table
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
val inserts = DataSourceTestUtils.generateRandomRows(1000)
// add some updates so that preCombine kicks in
val toUpdateDataset = sqlContext.createDataFrame(DataSourceTestUtils.getUniqueRows(inserts, 40), structType)
val updates = DataSourceTestUtils.updateRowsWithHigherTs(toUpdateDataset)
val records = inserts.union(updates)
val recordsSeq = convertRowListToSeq(records)
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
// collect all parition paths to issue read of parquet files
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
// Check the entire dataset has all records still
val fullPartitionPaths = new Array[String](3)
for (i <- 0 until fullPartitionPaths.length) {
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
}
// fetch all records from parquet files generated from write to hudi
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
val resultRows = actualDf.collectAsList()
// remove metadata columns so that expected and actual DFs can be compared as is
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
assert(df.except(trimmedDf).count() == 0)
testBulkInsertWithSortMode(sortMode, path)
} finally {
spark.stop()
FileUtils.deleteDirectory(path.toFile)
@@ -181,6 +134,118 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
}
})
List(true, false)
.foreach(populateMetaFields => {
test("test_bulk_insert_for_populate_meta_fields_" + populateMetaFields) {
initSparkContext("test_bulk_insert_datasource_populate_meta_fields")
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path_populate_meta_fields")
try {
testBulkInsertWithSortMode(BulkInsertSortMode.NONE, path, populateMetaFields)
} finally {
spark.stop()
FileUtils.deleteDirectory(path.toFile)
}
}
})
def testBulkInsertWithSortMode(sortMode: BulkInsertSortMode, path: java.nio.file.Path, populateMetaFields : Boolean = true) : Unit = {
val hoodieFooTableName = "hoodie_foo_tbl"
//create a new table
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() -> String.valueOf(populateMetaFields),
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
HoodieWriteConfig.BULKINSERT_SORT_MODE.key() -> sortMode.name(),
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
val inserts = DataSourceTestUtils.generateRandomRows(1000)
// add some updates so that preCombine kicks in
val toUpdateDataset = sqlContext.createDataFrame(DataSourceTestUtils.getUniqueRows(inserts, 40), structType)
val updates = DataSourceTestUtils.updateRowsWithHigherTs(toUpdateDataset)
val records = inserts.union(updates)
val recordsSeq = convertRowListToSeq(records)
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
// collect all parition paths to issue read of parquet files
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
// Check the entire dataset has all records still
val fullPartitionPaths = new Array[String](3)
for (i <- 0 until fullPartitionPaths.length) {
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
}
// fetch all records from parquet files generated from write to hudi
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
if (!populateMetaFields) {
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(0)).filter(entry => !(entry.mkString(",").equals(""))).count())
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(1)).filter(entry => !(entry.mkString(",").equals(""))).count())
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(2)).filter(entry => !(entry.mkString(",").equals(""))).count())
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(3)).filter(entry => !(entry.mkString(",").equals(""))).count())
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(4)).filter(entry => !(entry.mkString(",").equals(""))).count())
}
// remove metadata columns so that expected and actual DFs can be compared as is
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
assert(df.except(trimmedDf).count() == 0)
}
test("test disable and enable meta fields") {
initSparkContext("test_disable_enable_meta_fields")
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
try {
testBulkInsertWithSortMode(BulkInsertSortMode.NONE, path, false)
// enabling meta fields back should throw exception
val hoodieFooTableName = "hoodie_foo_tbl"
//create a new table
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
HoodieWriteConfig.BULKINSERT_SORT_MODE.key() -> BulkInsertSortMode.NONE.name(),
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
val inserts = DataSourceTestUtils.generateRandomRows(1000)
val df = spark.createDataFrame(sc.parallelize(inserts), structType)
try {
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
fail("Should have thrown exception")
} catch {
case e: HoodieException => assertTrue(e.getMessage.contains("hoodie.populate.meta.fields already disabled for the table. Can't be re-enabled back"))
}
} finally {
spark.stop()
FileUtils.deleteDirectory(path.toFile)
}
}
test("test drop duplicates row writing for bulk_insert") {
initSparkContext("test_append_mode")
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.internal;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.config.HoodieInternalConfig;
import org.apache.hudi.config.HoodieWriteConfig;
@@ -61,12 +62,14 @@ public class DefaultSource extends BaseDefaultSource implements DataSourceV2,
String instantTime = options.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY).get();
String path = options.get("path").get();
String tblName = options.get(HoodieWriteConfig.TABLE_NAME.key()).get();
boolean populateMetaFields = options.getBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(),
Boolean.parseBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()));
// 1st arg to createHooodieConfig is not really reuqired to be set. but passing it anyways.
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(options.get(HoodieWriteConfig.AVRO_SCHEMA.key()).get(), path, tblName, options.asMap());
boolean arePartitionRecordsSorted = HoodieInternalConfig.getBulkInsertIsPartitionRecordsSorted(
options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).isPresent()
? options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).get() : null);
return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(),
getConfiguration(), options, arePartitionRecordsSorted));
getConfiguration(), options, populateMetaFields, arePartitionRecordsSorted));
}
}

View File

@@ -36,10 +36,10 @@ public class HoodieBulkInsertDataInternalWriter implements DataWriter<InternalRo
private final BulkInsertDataInternalWriterHelper bulkInsertWriterHelper;
public HoodieBulkInsertDataInternalWriter(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.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable,
writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType, arePartitionRecordsSorted);
writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType, populateMetaFields, arePartitionRecordsSorted);
}
@Override

View File

@@ -36,19 +36,22 @@ public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFact
private final HoodieWriteConfig writeConfig;
private final StructType structType;
private final boolean arePartitionRecordsSorted;
private final boolean populateMetaFields;
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
String instantTime, StructType structType, boolean arePartitionRecordsSorted) {
String instantTime, StructType structType, boolean populateMetaFields,
boolean arePartitionRecordsSorted) {
this.hoodieTable = hoodieTable;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
this.structType = structType;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
}
@Override
public DataWriter<InternalRow> createDataWriter(int partitionId, long taskId, long epochId) {
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId, epochId,
structType, arePartitionRecordsSorted);
structType, populateMetaFields, arePartitionRecordsSorted);
}
}

View File

@@ -49,15 +49,17 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
private final HoodieWriteConfig writeConfig;
private final StructType structType;
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
private final boolean populateMetaFields;
private final Boolean arePartitionRecordsSorted;
private Map<String, String> extraMetadataMap = new HashMap<>();
public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession sparkSession, Configuration configuration, DataSourceOptions dataSourceOptions,
boolean arePartitionRecordsSorted) {
boolean populateMetaFields, boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.extraMetadataMap = DataSourceUtils.getExtraMetadata(dataSourceOptions.asMap());
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
@@ -69,7 +71,7 @@ public class HoodieDataSourceInternalWriter implements DataSourceWriter {
dataSourceInternalWriterHelper.createInflightCommit();
if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) {
return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(),
writeConfig, instantTime, structType, arePartitionRecordsSorted);
writeConfig, instantTime, structType, populateMetaFields, arePartitionRecordsSorted);
} else {
throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported ");
}

View File

@@ -38,7 +38,6 @@ import java.util.stream.Stream;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
@@ -51,6 +50,16 @@ public class TestHoodieBulkInsertDataInternalWriter extends
HoodieBulkInsertInternalWriterTestBase {
private static Stream<Arguments> configParams() {
Object[][] data = new Object[][] {
{true, true},
{true, false},
{false, true},
{false, false}
};
return Stream.of(data).map(Arguments::of);
}
private static Stream<Arguments> bulkInsertTypeParams() {
Object[][] data = new Object[][] {
{true},
{false}
@@ -60,16 +69,16 @@ public class TestHoodieBulkInsertDataInternalWriter extends
@ParameterizedTest
@MethodSource("configParams")
public void testDataInternalWriter(boolean sorted) throws Exception {
public void testDataInternalWriter(boolean sorted, boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
// execute N rounds
for (int i = 0; i < 3; i++) {
String instantTime = "00" + i;
// init writer
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE,
sorted);
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(),
STRUCT_TYPE, populateMetaFields, sorted);
int size = 10 + RANDOM.nextInt(1000);
// write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
@@ -96,11 +105,10 @@ public class TestHoodieBulkInsertDataInternalWriter extends
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
assertOutput(totalInputRows, result, instantTime, fileNames);
assertOutput(totalInputRows, result, instantTime, fileNames, populateMetaFields);
}
}
/**
* Issue some corrupted or wrong schematized InternalRow after few valid InternalRows so that global error is thrown. write batch 1 of valid records write batch2 of invalid records which is expected
* to throw Global Error. Verify global error is set appropriately and only first batch of records are written to disk.
@@ -108,13 +116,13 @@ public class TestHoodieBulkInsertDataInternalWriter extends
@Test
public void testGlobalFailure() throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(true);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
String instantTime = "001";
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE,
false);
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(),
STRUCT_TYPE, true, false);
int size = 10 + RANDOM.nextInt(100);
int totalFailures = 5;
@@ -150,7 +158,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
assertOutput(inputRows, result, instantTime, fileNames);
assertOutput(inputRows, result, instantTime, fileNames, true);
}
private void writeRows(Dataset<Row> inputRows, HoodieBulkInsertDataInternalWriter writer)

View File

@@ -31,6 +31,9 @@ import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.DataSourceOptions;
import org.apache.spark.sql.sources.v2.writer.DataWriter;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.util.ArrayList;
import java.util.Arrays;
@@ -38,10 +41,10 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Stream;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -53,18 +56,28 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieDataSourceInternalWriter extends
HoodieBulkInsertInternalWriterTestBase {
@Test
public void testDataSourceWriter() throws Exception {
testDataSourceWriterInternal(Collections.EMPTY_MAP, Collections.EMPTY_MAP);
private static Stream<Arguments> bulkInsertTypeParams() {
Object[][] data = new Object[][] {
{true},
{false}
};
return Stream.of(data).map(Arguments::of);
}
private void testDataSourceWriterInternal(Map<String, String> extraMetadata, Map<String, String> expectedExtraMetadata) throws Exception {
@ParameterizedTest
@MethodSource("bulkInsertTypeParams")
public void testDataSourceWriter(boolean populateMetaFields) throws Exception {
testDataSourceWriterInternal(Collections.EMPTY_MAP, Collections.EMPTY_MAP, populateMetaFields);
}
private void testDataSourceWriterInternal(Map<String, String> extraMetadata, Map<String, String> expectedExtraMetadata, boolean populateMetaFields)
throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
String instantTime = "001";
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(extraMetadata), false);
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(extraMetadata), populateMetaFields, false);
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
@@ -95,7 +108,7 @@ public class TestHoodieDataSourceInternalWriter extends
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify output
assertOutput(totalInputRows, result, instantTime, Option.empty());
assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
// verify extra metadata
@@ -122,7 +135,7 @@ public class TestHoodieDataSourceInternalWriter extends
expectedMetadata.remove(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key());
expectedMetadata.remove("commit_extra_c");
testDataSourceWriterInternal(extraMeta, expectedMetadata);
testDataSourceWriterInternal(extraMeta, expectedMetadata, true);
}
@Test
@@ -134,13 +147,14 @@ public class TestHoodieDataSourceInternalWriter extends
extraMeta.put("keyB", "valB");
extraMeta.put("commit_extra_c", "valC");
// none of the keys has commit metadata key prefix.
testDataSourceWriterInternal(extraMeta, Collections.EMPTY_MAP);
testDataSourceWriterInternal(extraMeta, Collections.EMPTY_MAP, true);
}
@Test
public void testMultipleDataSourceWrites() throws Exception {
@ParameterizedTest
@MethodSource("bulkInsertTypeParams")
public void testMultipleDataSourceWrites(boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
int partitionCounter = 0;
// execute N rounds
@@ -148,7 +162,7 @@ public class TestHoodieDataSourceInternalWriter extends
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), false);
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), populateMetaFields, false);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
@@ -172,18 +186,20 @@ public class TestHoodieDataSourceInternalWriter extends
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime,
populateMetaFields);
// verify output
assertOutput(totalInputRows, result, instantTime, Option.empty());
assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
}
}
@Test
public void testLargeWrites() throws Exception {
@ParameterizedTest
@MethodSource("bulkInsertTypeParams")
public void testLargeWrites(boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
int partitionCounter = 0;
// execute N rounds
@@ -191,7 +207,7 @@ public class TestHoodieDataSourceInternalWriter extends
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), false);
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), populateMetaFields, false);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
@@ -215,10 +231,11 @@ public class TestHoodieDataSourceInternalWriter extends
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime,
populateMetaFields);
// verify output
assertOutput(totalInputRows, result, instantTime, Option.empty());
assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
}
}
@@ -229,15 +246,16 @@ public class TestHoodieDataSourceInternalWriter extends
* abort batch2
* verify only records from batch1 is available to read
*/
@Test
public void testAbort() throws Exception {
@ParameterizedTest
@MethodSource("bulkInsertTypeParams")
public void testAbort(boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
String instantTime0 = "00" + 0;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), false);
new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), populateMetaFields, false);
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
@@ -269,13 +287,13 @@ public class TestHoodieDataSourceInternalWriter extends
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify rows
assertOutput(totalInputRows, result, instantTime0, Option.empty());
assertOutput(totalInputRows, result, instantTime0, Option.empty(), populateMetaFields);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
// 2nd batch. abort in the end
String instantTime1 = "00" + 1;
dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), false);
new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, new DataSourceOptions(Collections.EMPTY_MAP), populateMetaFields, false);
writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(1, RANDOM.nextLong(), RANDOM.nextLong());
for (int j = 0; j < batches; j++) {
@@ -293,7 +311,7 @@ public class TestHoodieDataSourceInternalWriter extends
result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify rows
// only rows from first batch should be present
assertOutput(totalInputRows, result, instantTime0, Option.empty());
assertOutput(totalInputRows, result, instantTime0, Option.empty(), populateMetaFields);
}
private void writeRows(Dataset<Row> inputRows, DataWriter<InternalRow> writer) throws Exception {

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.spark3.internal;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.config.HoodieInternalConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.internal.BaseDefaultSource;
@@ -48,11 +49,13 @@ public class DefaultSource extends BaseDefaultSource implements TableProvider {
String instantTime = properties.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY);
String path = properties.get("path");
String tblName = properties.get(HoodieWriteConfig.TABLE_NAME.key());
boolean populateMetaFields = Boolean.parseBoolean(properties.getOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(),
HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()));
boolean arePartitionRecordsSorted = Boolean.parseBoolean(properties.getOrDefault(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED,
Boolean.toString(HoodieInternalConfig.DEFAULT_BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED)));
// 1st arg to createHooodieConfig is not really reuqired to be set. but passing it anyways.
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(properties.get(HoodieWriteConfig.AVRO_SCHEMA.key()), path, tblName, properties);
return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(),
getConfiguration(), properties, arePartitionRecordsSorted);
getConfiguration(), properties, populateMetaFields, arePartitionRecordsSorted);
}
}

View File

@@ -37,9 +37,10 @@ public class HoodieBulkInsertDataInternalWriter implements DataWriter<InternalRo
private final BulkInsertDataInternalWriterHelper bulkInsertWriterHelper;
public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
String instantTime, int taskPartitionId, long taskId, StructType structType, boolean arePartitionRecordsSorted) {
String instantTime, int taskPartitionId, long taskId, StructType structType, boolean populateMetaFields,
boolean arePartitionRecordsSorted) {
this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable,
writeConfig, instantTime, taskPartitionId, taskId, 0, structType, arePartitionRecordsSorted);
writeConfig, instantTime, taskPartitionId, taskId, 0, structType, populateMetaFields, arePartitionRecordsSorted);
}
@Override
@@ -61,4 +62,4 @@ public class HoodieBulkInsertDataInternalWriter implements DataWriter<InternalRo
public void close() throws IOException {
bulkInsertWriterHelper.close();
}
}
}

View File

@@ -35,20 +35,23 @@ public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFact
private final HoodieTable hoodieTable;
private final HoodieWriteConfig writeConfig;
private final StructType structType;
private final boolean populateMetaFields;
private final boolean arePartitionRecordsSorted;
public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig,
String instantTime, StructType structType, boolean arePartitionRecordsSorted) {
String instantTime, StructType structType, boolean populateMetaFields,
boolean arePartitionRecordsSorted) {
this.hoodieTable = hoodieTable;
this.writeConfig = writeConfig;
this.instantTime = instantTime;
this.structType = structType;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
}
@Override
public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId,
structType, arePartitionRecordsSorted);
structType, populateMetaFields, arePartitionRecordsSorted);
}
}

View File

@@ -27,10 +27,10 @@ import org.apache.hudi.internal.DataSourceInternalWriterHelper;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.connector.write.DataWriterFactory;
import org.apache.spark.sql.connector.write.WriterCommitMessage;
import org.apache.spark.sql.connector.write.BatchWrite;
import org.apache.spark.sql.connector.write.DataWriterFactory;
import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
import org.apache.spark.sql.connector.write.WriterCommitMessage;
import org.apache.spark.sql.types.StructType;
import java.util.Arrays;
@@ -49,14 +49,16 @@ public class HoodieDataSourceInternalBatchWrite implements BatchWrite {
private final HoodieWriteConfig writeConfig;
private final StructType structType;
private final boolean arePartitionRecordsSorted;
private final boolean populateMetaFields;
private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper;
private Map<String, String> extraMetadata = new HashMap<>();
public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties, boolean arePartitionRecordsSorted) {
SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties, boolean populateMetaFields, boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.extraMetadata = DataSourceUtils.getExtraMetadata(properties);
this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType,
@@ -68,7 +70,7 @@ public class HoodieDataSourceInternalBatchWrite implements BatchWrite {
dataSourceInternalWriterHelper.createInflightCommit();
if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) {
return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(),
writeConfig, instantTime, structType, arePartitionRecordsSorted);
writeConfig, instantTime, structType, populateMetaFields, arePartitionRecordsSorted);
} else {
throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported ");
}

View File

@@ -39,23 +39,26 @@ public class HoodieDataSourceInternalBatchWriteBuilder implements WriteBuilder {
private final StructType structType;
private final SparkSession jss;
private final Configuration hadoopConfiguration;
private final boolean arePartitionRecordsSorted;
private final Map<String, String> properties;
private final boolean populateMetaFields;
private final boolean arePartitionRecordsSorted;
public HoodieDataSourceInternalBatchWriteBuilder(String instantTime, HoodieWriteConfig writeConfig, StructType structType,
SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties, boolean arePartitionRecordsSorted) {
SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties, boolean populateMetaFields,
boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = writeConfig;
this.structType = structType;
this.jss = jss;
this.hadoopConfiguration = hadoopConfiguration;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.properties = properties;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
}
@Override
public BatchWrite buildForBatch() {
return new HoodieDataSourceInternalBatchWrite(instantTime, writeConfig, structType, jss,
hadoopConfiguration, properties, arePartitionRecordsSorted);
hadoopConfiguration, properties, populateMetaFields, arePartitionRecordsSorted);
}
}

View File

@@ -18,9 +18,9 @@
package org.apache.hudi.spark3.internal;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.connector.catalog.SupportsWrite;
import org.apache.spark.sql.connector.catalog.TableCapability;
@@ -44,17 +44,19 @@ class HoodieDataSourceInternalTable implements SupportsWrite {
private final Configuration hadoopConfiguration;
private final boolean arePartitionRecordsSorted;
private final Map<String, String> properties;
private final boolean populateMetaFields;
public HoodieDataSourceInternalTable(String instantTime, HoodieWriteConfig config,
StructType schema, SparkSession jss, Configuration hadoopConfiguration, Map<String, String> properties,
boolean arePartitionRecordsSorted) {
boolean populateMetaFields, boolean arePartitionRecordsSorted) {
this.instantTime = instantTime;
this.writeConfig = config;
this.structType = schema;
this.jss = jss;
this.hadoopConfiguration = hadoopConfiguration;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
this.properties = properties;
this.populateMetaFields = populateMetaFields;
this.arePartitionRecordsSorted = arePartitionRecordsSorted;
}
@Override
@@ -69,7 +71,8 @@ class HoodieDataSourceInternalTable implements SupportsWrite {
@Override
public Set<TableCapability> capabilities() {
return new HashSet<TableCapability>() {{
return new HashSet<TableCapability>() {
{
add(TableCapability.BATCH_WRITE);
add(TableCapability.TRUNCATE);
}
@@ -79,6 +82,6 @@ class HoodieDataSourceInternalTable implements SupportsWrite {
@Override
public WriteBuilder newWriteBuilder(LogicalWriteInfo logicalWriteInfo) {
return new HoodieDataSourceInternalBatchWriteBuilder(instantTime, writeConfig, structType, jss,
hadoopConfiguration, properties, arePartitionRecordsSorted);
hadoopConfiguration, properties, populateMetaFields, arePartitionRecordsSorted);
}
}

View File

@@ -39,7 +39,6 @@ import java.util.stream.Stream;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
@@ -52,6 +51,16 @@ public class TestHoodieBulkInsertDataInternalWriter extends
HoodieBulkInsertInternalWriterTestBase {
private static Stream<Arguments> configParams() {
Object[][] data = new Object[][] {
{true, true},
{true, false},
{false, true},
{false, false}
};
return Stream.of(data).map(Arguments::of);
}
private static Stream<Arguments> bulkInsertTypeParams() {
Object[][] data = new Object[][] {
{true},
{false}
@@ -61,16 +70,16 @@ public class TestHoodieBulkInsertDataInternalWriter extends
@ParameterizedTest
@MethodSource("configParams")
public void testDataInternalWriter(boolean sorted) throws Exception {
public void testDataInternalWriter(boolean sorted, boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
// execute N rounds
for (int i = 0; i < 5; i++) {
String instantTime = "00" + i;
// init writer
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE,
sorted);
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000),
RANDOM.nextLong(), STRUCT_TYPE, populateMetaFields, sorted);
int size = 10 + RANDOM.nextInt(1000);
// write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
@@ -97,7 +106,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
assertOutput(totalInputRows, result, instantTime, fileNames);
assertOutput(totalInputRows, result, instantTime, fileNames, populateMetaFields);
}
}
@@ -109,12 +118,13 @@ public class TestHoodieBulkInsertDataInternalWriter extends
@Test
public void testGlobalFailure() throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(true);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
String instantTime = "001";
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE, false);
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000),
RANDOM.nextLong(), STRUCT_TYPE, true, false);
int size = 10 + RANDOM.nextInt(100);
int totalFailures = 5;
@@ -150,7 +160,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
assertOutput(inputRows, result, instantTime, fileNames);
assertOutput(inputRows, result, instantTime, fileNames, true);
}
private void writeRows(Dataset<Row> inputRows, HoodieBulkInsertDataInternalWriter writer)

View File

@@ -33,6 +33,9 @@ import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.connector.write.DataWriter;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.util.ArrayList;
import java.util.Arrays;
@@ -40,10 +43,10 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Stream;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -55,19 +58,28 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieDataSourceInternalBatchWrite extends
HoodieBulkInsertInternalWriterTestBase {
@Test
public void testDataSourceWriter() throws Exception {
testDataSourceWriterInternal(Collections.EMPTY_MAP, Collections.EMPTY_MAP);
private static Stream<Arguments> bulkInsertTypeParams() {
Object[][] data = new Object[][] {
{true},
{false}
};
return Stream.of(data).map(Arguments::of);
}
private void testDataSourceWriterInternal(Map<String, String> extraMetadata, Map<String, String> expectedExtraMetadata) throws Exception {
@ParameterizedTest
@MethodSource("bulkInsertTypeParams")
public void testDataSourceWriter(boolean populateMetaFields) throws Exception {
testDataSourceWriterInternal(Collections.EMPTY_MAP, Collections.EMPTY_MAP, populateMetaFields);
}
private void testDataSourceWriterInternal(Map<String, String> extraMetadata, Map<String, String> expectedExtraMetadata, boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
String instantTime = "001";
// init writer
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, extraMetadata, false);
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, extraMetadata, populateMetaFields, false);
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
@@ -99,7 +111,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify output
assertOutput(totalInputRows, result, instantTime, Option.empty());
assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
// verify extra metadata
@@ -125,7 +137,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
expectedMetadata.remove(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key());
expectedMetadata.remove("commit_extra_c");
testDataSourceWriterInternal(extraMeta, expectedMetadata);
testDataSourceWriterInternal(extraMeta, expectedMetadata, true);
}
@Test
@@ -137,13 +149,14 @@ public class TestHoodieDataSourceInternalBatchWrite extends
extraMeta.put("keyB", "valB");
extraMeta.put("commit_extra_c", "valC");
// none of the keys has commit metadata key prefix.
testDataSourceWriterInternal(extraMeta, Collections.EMPTY_MAP);
testDataSourceWriterInternal(extraMeta, Collections.EMPTY_MAP, true);
}
@Test
public void testMultipleDataSourceWrites() throws Exception {
@ParameterizedTest
@MethodSource("bulkInsertTypeParams")
public void testMultipleDataSourceWrites(boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
int partitionCounter = 0;
@@ -152,7 +165,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, populateMetaFields, false);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong());
@@ -176,18 +189,19 @@ public class TestHoodieDataSourceInternalBatchWrite extends
dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, populateMetaFields);
// verify output
assertOutput(totalInputRows, result, instantTime, Option.empty());
assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
}
}
@Test
public void testLargeWrites() throws Exception {
@ParameterizedTest
@MethodSource("bulkInsertTypeParams")
public void testLargeWrites(boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
int partitionCounter = 0;
@@ -196,7 +210,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, populateMetaFields, false);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong());
@@ -220,10 +234,11 @@ public class TestHoodieDataSourceInternalBatchWrite extends
dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime,
populateMetaFields);
// verify output
assertOutput(totalInputRows, result, instantTime, Option.empty());
assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
}
}
@@ -234,15 +249,16 @@ public class TestHoodieDataSourceInternalBatchWrite extends
* abort batch2
* verify only records from batch1 is available to read
*/
@Test
public void testAbort() throws Exception {
@ParameterizedTest
@MethodSource("bulkInsertTypeParams")
public void testAbort(boolean populateMetaFields) throws Exception {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
String instantTime0 = "00" + 0;
// init writer
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, populateMetaFields, false);
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
@@ -274,13 +290,13 @@ public class TestHoodieDataSourceInternalBatchWrite extends
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify rows
assertOutput(totalInputRows, result, instantTime0, Option.empty());
assertOutput(totalInputRows, result, instantTime0, Option.empty(), populateMetaFields);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
// 2nd batch. abort in the end
String instantTime1 = "00" + 1;
dataSourceInternalBatchWrite =
new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, populateMetaFields, false);
writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(1, RANDOM.nextLong());
for (int j = 0; j < batches; j++) {
@@ -298,7 +314,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify rows
// only rows from first batch should be present
assertOutput(totalInputRows, result, instantTime0, Option.empty());
assertOutput(totalInputRows, result, instantTime0, Option.empty(), populateMetaFields);
}
private void writeRows(Dataset<Row> inputRows, DataWriter<InternalRow> writer) throws Exception {