diff --git a/azure-pipelines.yml b/azure-pipelines.yml index cb9787bb3f930..48efde2d1fc8f 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -179,7 +179,7 @@ stages: displayName: Top 100 long-running testcases - job: UT_FT_4 displayName: UT FT other modules - timeoutInMinutes: '150' + timeoutInMinutes: '180' steps: - task: Maven@3.205.1 displayName: maven install diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 2ef02b1dae54c..06c56c73441c3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -226,7 +226,10 @@ private Option prepareRecord(HoodieRecord hoodieRecord) { // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema HoodieRecord rewrittenRecord = schemaOnReadEnabled ? finalRecord.get().rewriteRecordWithNewSchema(tableSchema, recordProperties, writeSchemaWithMetaFields) : finalRecord.get().rewriteRecord(tableSchema, recordProperties, writeSchemaWithMetaFields); - HoodieRecord populatedRecord = populateMetadataFields(rewrittenRecord, writeSchemaWithMetaFields, recordProperties); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into the recordList(List). + HoodieRecord populatedRecord = populateMetadataFields(rewrittenRecord.copy(), writeSchemaWithMetaFields, recordProperties); finalRecord = Option.of(populatedRecord); if (isUpdateRecord) { updatedRecordsWritten++; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index 4e5370f10898e..64ef804291c93 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -157,15 +157,6 @@ public static HoodieBaseFile getLatestBaseFile(HoodieTable hoodieTab return baseFileOp.get(); } - @Override - public Schema getWriterSchemaWithMetaFields() { - return writeSchemaWithMetaFields; - } - - public Schema getWriterSchema() { - return writeSchema; - } - /** * Extract old file path, initialize StorageWriter and WriteStatus. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 3e0691cdf044d..0e9004b0d518b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -177,6 +177,10 @@ public Schema getWriterSchemaWithMetaFields() { return writeSchemaWithMetaFields; } + public Schema getWriterSchema() { + return writeSchema; + } + /** * Determines whether we can accept the incoming records, into the current file. Depending on *

diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java index 73ad53e4a6c16..38801851600ca 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java @@ -36,7 +36,7 @@ public BootstrapRecordConsumer(HoodieBootstrapHandle bootstrapHandle) { @Override protected void consumeOneRecord(HoodieRecord record) { - bootstrapHandle.write(record, bootstrapHandle.getWriterSchemaWithMetaFields(), new TypedProperties()); + bootstrapHandle.write(record, bootstrapHandle.getWriterSchema(), new TypedProperties()); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java index e4fb3face4a59..e6a3bd813d8cb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java @@ -132,7 +132,7 @@ public void runMerge(HoodieTable>, HoodieData>, HoodieData(table.getConfig().getWriteBufferLimitBytes(), readerIterator, new UpdateHandler(mergeHandle), record -> { - HoodieRecord recordCopy = record.copy(); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into queue of BoundedInMemoryExecutor. if (!externalSchemaTransformation) { - return recordCopy; + return record.copy(); } try { - return recordCopy.rewriteRecord(writerSchema, new Properties(), readerSchema); + return record.rewriteRecord(writerSchema, new Properties(), readerSchema).copy(); } catch (IOException e) { throw new HoodieException(String.format("Failed to rewrite record. WriterSchema: %s; ReaderSchema: %s", writerSchema, readerSchema), e); } @@ -175,10 +177,10 @@ class RewriteIterator implements ClosableIterator { private final Properties prop; private final Map renameCols; - public RewriteIterator(ClosableIterator iter, Schema newSchema, Schema recordSchema, Properties prop, Map renameCols) { + public RewriteIterator(ClosableIterator iter, Schema recordSchema, Schema newSchema, Properties prop, Map renameCols) { this.iter = iter; - this.newSchema = newSchema; this.recordSchema = recordSchema; + this.newSchema = newSchema; this.prop = prop; this.renameCols = renameCols; } @@ -191,7 +193,8 @@ public boolean hasNext() { @Override public HoodieRecord next() { try { - return iter.next().rewriteRecordWithNewSchema(recordSchema, prop, newSchema, renameCols); + HoodieRecord record = iter.next(); + return record.rewriteRecordWithNewSchema(recordSchema, prop, newSchema, renameCols); } catch (IOException e) { LOG.error("Error rewrite record with new schema", e); throw new HoodieException(e); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java index 06886dbb07f78..12a2cefcbfe4f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java @@ -92,12 +92,14 @@ public void runMerge(HoodieTable>, List, List wrapper = new BoundedInMemoryExecutor<>(table.getConfig().getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(readerIterator), Option.of(new UpdateHandler(mergeHandle)), record -> { - HoodieRecord recordCopy = record.copy(); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into queue of BoundedInMemoryExecutor. if (!externalSchemaTransformation) { - return recordCopy; + return record.copy(); } try { - return recordCopy.rewriteRecord(writerSchema, new Properties(), readerSchema); + return record.rewriteRecord(writerSchema, new Properties(), readerSchema).copy(); } catch (IOException e) { throw new HoodieException(e); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java index e62643ad4ed87..9aefdc4f39982 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java @@ -216,6 +216,9 @@ private List> readRecordsForGroupBaseFiles(List recordIterator = baseFileReader.getRecordIterator(readerSchema); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into the records(List). recordIterator.forEachRemaining(record -> records.add(record.copy().wrapIntoHoodieRecordPayloadWithKeyGen(readerSchema, new Properties(), Option.empty()))); } catch (IOException e) { throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java index 54a044ae26742..c9e348f0b2981 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java @@ -90,12 +90,14 @@ public void runMerge(HoodieTable>, List, List wrapper = new BoundedInMemoryExecutor<>(table.getConfig().getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(readerIterator), Option.of(new UpdateHandler(mergeHandle)), record -> { - HoodieRecord recordCopy = record.copy(); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into queue of BoundedInMemoryExecutor. if (!externalSchemaTransformation) { - return recordCopy; + return record.copy(); } try { - return recordCopy.rewriteRecord(writerSchema, new Properties(), readerSchema); + return record.rewriteRecord(writerSchema, new Properties(), readerSchema).copy(); } catch (IOException e) { throw new HoodieException(e); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 64264a1a10097..569489a0c0b86 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -220,6 +220,9 @@ private CompletableFuture> runClusteringForGroupAsync(Ho JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(getEngineContext()); HoodieData> inputRecords = readRecordsForGroup(jsc, clusteringGroup, instantTime); Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be shuffled later. List inputFileIds = clusteringGroup.getSlices().stream() .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) .collect(Collectors.toList()); @@ -331,6 +334,9 @@ private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContex HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath())); Option keyGeneratorOp = writeConfig.populateMetaFields() ? Option.empty() : Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(writeConfig.getProps())); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be shuffled later. MappingIterator mappingIterator = new MappingIterator((ClosableIterator) baseFileReader.getRecordIterator(readerSchema), rec -> ((HoodieRecord) rec).copy().wrapIntoHoodieRecordPayloadWithKeyGen(readerSchema, writeConfig.getProps(), keyGeneratorOp)); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java index 765fa663f2bcf..5ddb306a8fa2e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java @@ -151,6 +151,9 @@ private Iterator> readRecordsForGroupBaseFiles(List keyGeneratorOp = writeConfig.populateMetaFields() ? Option.empty() : Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(writeConfig.getProps())); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be shuffled later. MappingIterator mappingIterator = new MappingIterator((ClosableIterator) baseFileReader.getRecordIterator(readerSchema), rec -> ((HoodieRecord) rec).copy().wrapIntoHoodieRecordPayloadWithKeyGen(readerSchema, getWriteConfig().getProps(), keyGeneratorOp)); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java index 2577f2601cb6a..7370ee0f088a2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java @@ -29,8 +29,6 @@ import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; -import java.util.Arrays; - /** * Hudi internal implementation of the {@link InternalRow} allowing to extend arbitrary * {@link InternalRow} overlaying Hudi-internal meta-fields on top of it. @@ -231,7 +229,11 @@ public MapData getMap(int ordinal) { @Override public InternalRow copy() { - return new HoodieInternalRow(Arrays.copyOf(metaFields, metaFields.length), sourceRow.copy(), sourceContainsMetaFields); + UTF8String[] copyMetaFields = new UTF8String[metaFields.length]; + for (int i = 0; i < metaFields.length; i++) { + copyMetaFields[i] = metaFields[i] != null ? metaFields[i].copy() : null; + } + return new HoodieInternalRow(copyMetaFields, sourceRow.copy(), sourceContainsMetaFields); } private int rebaseOrdinal(int ordinal) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java index 19b8cb5c658f4..ea251c106b6ef 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java @@ -23,7 +23,6 @@ import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; import org.apache.avro.Schema; -import org.apache.hudi.HoodieInternalRowUtils; import org.apache.hudi.SparkAdapterSupport$; import org.apache.hudi.client.model.HoodieInternalRow; import org.apache.hudi.common.model.HoodieAvroIndexedRecord; @@ -40,7 +39,7 @@ import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.SparkKeyGeneratorInterface; import org.apache.hudi.util.HoodieSparkRecordUtils; -import org.apache.spark.sql.HoodieCatalystExpressionUtils$; +import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.HoodieUnsafeRowUtils; import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath; import org.apache.spark.sql.catalyst.CatalystTypeConverters; @@ -116,7 +115,7 @@ public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, boo } private HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, boolean copy) { - super(key, data, operation); + super(key, data, operation, Option.empty()); validateRow(data, schema); this.copy = copy; @@ -197,13 +196,15 @@ public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema); - boolean containMetaFields = hasMetaFields(structType); - UTF8String[] metaFields = tryExtractMetaFields(data, structType); + // TODO HUDI-5281 Rewrite HoodieSparkRecord with UnsafeRowWriter + InternalRow rewriteRecord = HoodieInternalRowUtils.rewriteRecord(this.data, structType, targetStructType); + UnsafeRow unsafeRow = HoodieInternalRowUtils.getCachedUnsafeProjection(targetStructType, targetStructType).apply(rewriteRecord); - // TODO add actual rewriting - InternalRow finalRow = new HoodieInternalRow(metaFields, data, containMetaFields); + boolean containMetaFields = hasMetaFields(targetStructType); + UTF8String[] metaFields = tryExtractMetaFields(unsafeRow, targetStructType); + HoodieInternalRow internalRow = new HoodieInternalRow(metaFields, unsafeRow, containMetaFields); - return new HoodieSparkRecord(getKey(), finalRow, targetStructType, getOperation(), this.currentLocation, this.newLocation, copy); + return new HoodieSparkRecord(getKey(), internalRow, targetStructType, getOperation(), this.currentLocation, this.newLocation, false); } @Override @@ -211,14 +212,15 @@ public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties p StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema); - boolean containMetaFields = hasMetaFields(structType); - UTF8String[] metaFields = tryExtractMetaFields(data, structType); + // TODO HUDI-5281 Rewrite HoodieSparkRecord with UnsafeRowWriter + InternalRow rewriteRecord = HoodieInternalRowUtils.rewriteRecordWithNewSchema(this.data, structType, newStructType, renameCols); + UnsafeRow unsafeRow = HoodieInternalRowUtils.getCachedUnsafeProjection(newStructType, newStructType).apply(rewriteRecord); - InternalRow rewrittenRow = - HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols); - HoodieInternalRow finalRow = new HoodieInternalRow(metaFields, rewrittenRow, containMetaFields); + boolean containMetaFields = hasMetaFields(newStructType); + UTF8String[] metaFields = tryExtractMetaFields(unsafeRow, newStructType); + HoodieInternalRow internalRow = new HoodieInternalRow(metaFields, unsafeRow, containMetaFields); - return new HoodieSparkRecord(getKey(), finalRow, newStructType, getOperation(), this.currentLocation, this.newLocation, copy); + return new HoodieSparkRecord(getKey(), internalRow, newStructType, getOperation(), this.currentLocation, this.newLocation, false); } @Override @@ -299,6 +301,7 @@ public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Schema recordSchema, P @Override public Option> getMetadata() { + // TODO HUDI-5282 support metaData return Option.empty(); } @@ -320,7 +323,7 @@ public HoodieSparkRecord copy() { public Comparable getOrderingValue(Schema recordSchema, Properties props) { StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); String orderingField = ConfigUtils.getOrderingField(props); - if (!HoodieCatalystExpressionUtils$.MODULE$.existField(structType, orderingField)) { + if (!HoodieInternalRowUtils.existField(structType, orderingField)) { return 0; } else { NestedFieldPath nestedFieldPath = HoodieInternalRowUtils.getCachedPosList(structType, orderingField); @@ -377,7 +380,10 @@ private static HoodieInternalRow wrapIntoUpdatableOverlay(InternalRow data, Stru private static UTF8String[] tryExtractMetaFields(InternalRow row, StructType structType) { boolean containsMetaFields = hasMetaFields(structType); - if (containsMetaFields) { + if (containsMetaFields && structType.size() == 1) { + // Support bootstrap with RECORD_KEY_SCHEMA + return new UTF8String[] {row.getUTF8String(0)}; + } else if (containsMetaFields) { return HoodieRecord.HOODIE_META_COLUMNS.stream() .map(col -> row.getUTF8String(HOODIE_META_COLUMNS_NAME_TO_POS.get(col))) .toArray(UTF8String[]::new); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java index 92fa6530619f2..0e9d4eaf9811b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java @@ -21,7 +21,6 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hudi.AvroConversionUtils; -import org.apache.hudi.HoodieInternalRowUtils; import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.SparkConversionUtils; import org.apache.hudi.client.common.HoodieSparkEngineContext; @@ -37,6 +36,7 @@ import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.Row; import org.apache.spark.sql.types.StructType; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java index a4211656b05c6..d2ab83f1481e8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java @@ -18,7 +18,6 @@ package org.apache.hudi.io.storage; -import org.apache.hudi.HoodieInternalRowUtils; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieStorageConfig; @@ -34,6 +33,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.spark.sql.HoodieInternalRowUtils; import java.io.IOException; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index b162cbbadc757..eed52de3cb254 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -21,7 +21,7 @@ import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hudi.HoodieInternalRowUtils; +import org.apache.hudi.SparkAdapterSupport$; import org.apache.hudi.commmon.model.HoodieSparkRecord; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieFileFormat; @@ -30,12 +30,18 @@ import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.ParquetReaderIterator; +import org.apache.hudi.common.util.ParquetUtils; +import org.apache.hudi.common.util.StringUtils; + import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.api.ReadSupport; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport; +import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; @@ -45,6 +51,7 @@ import java.util.Set; import static org.apache.hudi.common.util.TypeUtils.unsafeCast; +import static org.apache.parquet.avro.AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS; public class HoodieSparkParquetReader implements HoodieSparkFileReader { @@ -55,7 +62,9 @@ public class HoodieSparkParquetReader implements HoodieSparkFileReader { public HoodieSparkParquetReader(Configuration conf, Path path) { this.path = path; - this.conf = conf; + this.conf = new Configuration(conf); + // Avoid adding record in list element when convert parquet schema to avro schema + conf.set(ADD_LIST_ELEMENT_RECORDS, "false"); this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); } @@ -108,7 +117,13 @@ private ClosableIterator getInternalRowIterator(Schema readerSchema @Override public Schema getSchema() { - return parquetUtils.readAvroSchema(conf, path); + // Some types in avro are not compatible with parquet. + // Avro only supports representing Decimals as fixed byte array + // and therefore if we convert to Avro directly we'll lose logical type-info. + MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(conf, path); + StructType structType = new ParquetToSparkSchemaConverter(conf).convert(messageType); + return SparkAdapterSupport$.MODULE$.sparkAdapter().getAvroSchemaConverters() + .toAvroType(structType, true, messageType.getName(), StringUtils.EMPTY_STRING); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java index 86a984dd83517..d008d7cf9fd96 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java @@ -71,9 +71,13 @@ void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, wrapper = new BoundedInMemoryExecutor(config.getWriteBufferLimitBytes(), reader.getRecordIterator(), new BootstrapRecordConsumer(bootstrapHandle), record -> { try { - HoodieRecord recordCopy = record.copy(); - String recKey = recordCopy.getRecordKey(reader.getSchema(), Option.of(keyGenerator)); - HoodieRecord hoodieRecord = recordCopy.rewriteRecord(reader.getSchema(), config.getProps(), HoodieAvroUtils.RECORD_KEY_SCHEMA); + String recKey = record.getRecordKey(reader.getSchema(), Option.of(keyGenerator)); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be inserted into the queue later. + HoodieRecord hoodieRecord = record + .rewriteRecord(reader.getSchema(), config.getProps(), HoodieAvroUtils.RECORD_KEY_SCHEMA) + .copy(); MetadataValues metadataValues = new MetadataValues().setRecordKey(recKey); return hoodieRecord .updateMetadataValues(HoodieAvroUtils.RECORD_KEY_SCHEMA, new Properties(), metadataValues) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java index 65ac51a1c70b0..9a4aaa1dbc585 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java @@ -18,11 +18,10 @@ package org.apache.hudi.util; -import org.apache.hudi.HoodieInternalRowUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; -import org.apache.spark.sql.HoodieCatalystExpressionUtils$; +import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.HoodieUnsafeRowUtils; import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath; import org.apache.spark.sql.catalyst.InternalRow; @@ -43,7 +42,7 @@ public static Object getValue(StructType structType, String fieldName, InternalR * @return the string form of the field or empty if the schema does not contain the field name or the value is null */ public static Option getNullableValAsString(StructType structType, InternalRow row, String fieldName) { - String fieldVal = !HoodieCatalystExpressionUtils$.MODULE$.existField(structType, fieldName) + String fieldVal = !HoodieInternalRowUtils.existField(structType, fieldName) ? null : StringUtils.objToString(getValue(structType, fieldName, row)); return Option.ofNullable(fieldVal); } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala index 5e1d052414b8d..24e50613f50ab 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} +import scala.annotation.tailrec trait HoodieCatalystExpressionUtils { @@ -251,10 +252,6 @@ object HoodieCatalystExpressionUtils { ) } - def existField(structType: StructType, name: String): Boolean = { - structType.getFieldIndex(name).isDefined - } - private def hasUnresolvedRefs(resolvedExpr: Expression): Boolean = resolvedExpr.collectFirst { case _: UnresolvedAttribute | _: UnresolvedFunction => true diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala similarity index 78% rename from hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala index b6c331cbebc43..dfe3295cf003c 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala @@ -16,23 +16,22 @@ * limitations under the License. */ -package org.apache.hudi +package org.apache.spark.sql import java.nio.charset.StandardCharsets import java.util.HashMap import java.util.concurrent.ConcurrentHashMap import org.apache.avro.Schema import org.apache.hbase.thirdparty.com.google.common.base.Supplier +import org.apache.hudi.AvroConversionUtils import org.apache.hudi.avro.HoodieAvroUtils.{createFullName, toJavaDate} -import org.apache.hudi.client.model.HoodieInternalRow -import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField import org.apache.hudi.exception.HoodieException -import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieUnsafeRowUtils} import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String import scala.collection.mutable object HoodieInternalRowUtils { @@ -53,10 +52,12 @@ object HoodieInternalRowUtils { for ((field, pos) <- newSchema.fields.zipWithIndex) { var oldValue: AnyRef = null - if (HoodieCatalystExpressionUtils.existField(oldSchema, field.name)) { + var oldType: DataType = null + if (existField(oldSchema, field.name)) { val oldField = oldSchema(field.name) val oldPos = oldSchema.fieldIndex(field.name) - oldValue = oldRecord.get(oldPos, oldField.dataType) + oldType = oldField.dataType + oldValue = oldRecord.get(oldPos, oldType) } if (oldValue != null) { field.dataType match { @@ -72,8 +73,45 @@ object HoodieInternalRowUtils { } else { newRow.update(pos, oldValue) } - case _ => - newRow.update(pos, oldValue) + case t if t == oldType => newRow.update(pos, oldValue) + // Type promotion + case _: ShortType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toShort) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: IntegerType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toInt) + case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toInt) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: LongType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toLong) + case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toLong) + case _: IntegerType => newRow.update(pos, oldValue.asInstanceOf[Int].toLong) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: FloatType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toFloat) + case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toFloat) + case _: IntegerType => newRow.update(pos, oldValue.asInstanceOf[Int].toFloat) + case _: LongType => newRow.update(pos, oldValue.asInstanceOf[Long].toFloat) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: DoubleType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toDouble) + case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toDouble) + case _: IntegerType => newRow.update(pos, oldValue.asInstanceOf[Int].toDouble) + case _: LongType => newRow.update(pos, oldValue.asInstanceOf[Long].toDouble) + case _: FloatType => newRow.update(pos, oldValue.asInstanceOf[Float].toDouble) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: BinaryType if oldType.isInstanceOf[StringType] => newRow.update(pos, oldValue.asInstanceOf[String].getBytes) + case _ => newRow.update(pos, oldValue) } } else { // TODO default value in newSchema @@ -108,7 +146,7 @@ object HoodieInternalRowUtils { val oldStrucType = oldSchema.asInstanceOf[StructType] targetSchema.fields.zipWithIndex.foreach { case (field, i) => fieldNames.push(field.name) - if (HoodieCatalystExpressionUtils.existField(oldStrucType, field.name)) { + if (existField(oldStrucType, field.name)) { val oldField = oldStrucType(field.name) val oldPos = oldStrucType.fieldIndex(field.name) helper(i) = rewriteRecordWithNewSchema(oldRow.get(oldPos, oldField.dataType), oldField.dataType, field.dataType, renameCols, fieldNames) @@ -117,7 +155,7 @@ object HoodieInternalRowUtils { val colNamePartsFromOldSchema = renameCols.getOrDefault(fieldFullName, "").split("\\.") val lastColNameFromOldSchema = colNamePartsFromOldSchema(colNamePartsFromOldSchema.length - 1) // deal with rename - if (!HoodieCatalystExpressionUtils.existField(oldStrucType, field.name) && HoodieCatalystExpressionUtils.existField(oldStrucType, lastColNameFromOldSchema)) { + if (!existField(oldStrucType, field.name) && existField(oldStrucType, lastColNameFromOldSchema)) { // find rename val oldField = oldStrucType(lastColNameFromOldSchema) val oldPos = oldStrucType.fieldIndex(lastColNameFromOldSchema) @@ -162,7 +200,7 @@ object HoodieInternalRowUtils { val newValueArray = new GenericArrayData(Array.fill(oldMap.valueArray().numElements())(null).asInstanceOf[Array[Any]]) val newMap = new ArrayBasedMapData(newKeyArray, newValueArray) fieldNames.push("value") - oldMap.keyArray().toSeq[Any](oldKeyType).zipWithIndex.foreach { case (value, i) => newKeyArray.update(i, value) } + oldMap.keyArray().toSeq[Any](oldKeyType).zipWithIndex.foreach { case (value, i) => newKeyArray.update(i, rewritePrimaryType(value, oldKeyType, oldKeyType)) } oldMap.valueArray().toSeq[Any](oldValueType).zipWithIndex.foreach { case (value, i) => newValueArray.update(i, rewriteRecordWithNewSchema(value.asInstanceOf[AnyRef], oldValueType, newValueType, renameCols, fieldNames)) } fieldNames.pop() @@ -172,26 +210,6 @@ object HoodieInternalRowUtils { } } - /** - * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecordWithMetadata(org.apache.avro.generic.GenericRecord, org.apache.avro.Schema, java.lang.String) - */ - def rewriteRecordWithMetadata(record: InternalRow, oldSchema: StructType, newSchema: StructType, fileName: String): InternalRow = { - val newRecord = rewriteRecord(record, oldSchema, newSchema) - newRecord.update(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal, CatalystTypeConverters.convertToCatalyst(fileName)) - - newRecord - } - - /** - * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteEvolutionRecordWithMetadata(org.apache.avro.generic.GenericRecord, org.apache.avro.Schema, java.lang.String) - */ - def rewriteEvolutionRecordWithMetadata(record: InternalRow, oldSchema: StructType, newSchema: StructType, fileName: String): InternalRow = { - val newRecord = rewriteRecordWithNewSchema(record, oldSchema, newSchema, new java.util.HashMap[String, String]()) - newRecord.update(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal, CatalystTypeConverters.convertToCatalyst(fileName)) - - newRecord - } - def getCachedPosList(structType: StructType, field: String): NestedFieldPath = { val schemaPair = (structType, field) if (!orderPosListMap.containsKey(schemaPair)) { @@ -219,11 +237,22 @@ object HoodieInternalRowUtils { schemaMap.get(schema) } + def existField(structType: StructType, name: String): Boolean = { + try { + HoodieUnsafeRowUtils.composeNestedFieldPath(structType, name) + true + } catch { + case _: IllegalArgumentException => false + } + } + private def rewritePrimaryType(oldValue: Any, oldSchema: DataType, newSchema: DataType) = { if (oldSchema.equals(newSchema) || (oldSchema.isInstanceOf[DecimalType] && newSchema.isInstanceOf[DecimalType])) { oldSchema match { - case NullType | BooleanType | IntegerType | LongType | FloatType | DoubleType | StringType | DateType | TimestampType | BinaryType => + case NullType | BooleanType | IntegerType | LongType | FloatType | DoubleType | DateType | TimestampType | BinaryType => oldValue + // Copy UTF8String before putting into GenericInternalRow + case StringType => UTF8String.fromString(oldValue.toString) case DecimalType() => Decimal.fromDecimal(oldValue.asInstanceOf[Decimal].toBigDecimal.setScale(newSchema.asInstanceOf[DecimalType].scale)) case _ => diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index e634bc0710786..cc26c0be13888 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -128,6 +128,7 @@ /** * Test Cleaning related logic. */ +@Disabled public class TestCleaner extends HoodieClientTestBase { private static final int BIG_BATCH_INSERT_SIZE = 500; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java index a9e5091429d8d..7d497408e6467 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java @@ -53,8 +53,12 @@ public HoodieAvroIndexedRecord(HoodieKey key, IndexedRecord data) { super(key, data); } - public HoodieAvroIndexedRecord(HoodieKey key, IndexedRecord data, HoodieOperation operation) { - super(key, data, operation); + public HoodieAvroIndexedRecord( + HoodieKey key, + IndexedRecord data, + HoodieOperation operation, + Option> metaData) { + super(key, data, operation, metaData); } public HoodieAvroIndexedRecord(HoodieRecord record) { @@ -71,12 +75,12 @@ public HoodieRecord newInstance() { @Override public HoodieRecord newInstance(HoodieKey key, HoodieOperation op) { - return new HoodieAvroIndexedRecord(key, data, op); + return new HoodieAvroIndexedRecord(key, data, op, metaData); } @Override public HoodieRecord newInstance(HoodieKey key) { - return new HoodieAvroIndexedRecord(key, data); + return new HoodieAvroIndexedRecord(key, data, operation, metaData); } @Override @@ -104,19 +108,19 @@ public Object[] getColumnValues(Schema recordSchema, String[] columns, boolean c @Override public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { GenericRecord record = HoodieAvroUtils.stitchRecords((GenericRecord) data, (GenericRecord) other.getData(), targetSchema); - return new HoodieAvroIndexedRecord(record); + return new HoodieAvroIndexedRecord(key, record, operation, metaData); } @Override public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException { - GenericRecord genericRecord = HoodieAvroUtils.rewriteRecord((GenericRecord) data, targetSchema); - return new HoodieAvroIndexedRecord(genericRecord); + GenericRecord record = HoodieAvroUtils.rewriteRecord((GenericRecord) data, targetSchema); + return new HoodieAvroIndexedRecord(key, record, operation, metaData); } @Override public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException { - GenericRecord genericRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(data, newSchema, renameCols); - return new HoodieAvroIndexedRecord(genericRecord); + GenericRecord record = HoodieAvroUtils.rewriteRecordWithNewSchema(data, newSchema, renameCols); + return new HoodieAvroIndexedRecord(key, record, operation, metaData); } @Override @@ -127,7 +131,7 @@ public HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, } }); - return new HoodieAvroIndexedRecord(data); + return new HoodieAvroIndexedRecord(key, data, operation, metaData); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java index a1318c462c1ac..4de0470dcf879 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java @@ -44,7 +44,7 @@ public HoodieAvroRecord(HoodieKey key, T data) { } public HoodieAvroRecord(HoodieKey key, T data, HoodieOperation operation) { - super(key, data, operation); + super(key, data, operation, Option.empty()); } public HoodieAvroRecord(HoodieRecord record) { @@ -206,7 +206,9 @@ public Option> getMetadata() { public Option toIndexedRecord(Schema recordSchema, Properties props) throws IOException { Option avroData = getData().getInsertValue(recordSchema, props); if (avroData.isPresent()) { - return Option.of(new HoodieAvroIndexedRecord(avroData.get())); + HoodieAvroIndexedRecord record = + new HoodieAvroIndexedRecord(key, avroData.get(), operation, getData().getMetadata()); + return Option.of(record); } else { return Option.empty(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java index 8e80ff7c84f33..e2f9334f8a2ec 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java @@ -42,7 +42,7 @@ public HoodieEmptyRecord(HoodieKey key, HoodieRecordType type) { } public HoodieEmptyRecord(HoodieKey key, HoodieOperation operation, Comparable orderingVal, HoodieRecordType type) { - super(key, null, operation); + super(key, null, operation, Option.empty()); this.type = type; this.orderingVal = orderingVal; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java index 255a2b2a10fe7..a23316a2ff9fd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -141,17 +141,23 @@ public String getFieldName() { */ protected HoodieOperation operation; + /** + * The metaData of the record. + */ + protected Option> metaData; + public HoodieRecord(HoodieKey key, T data) { - this(key, data, null); + this(key, data, null, Option.empty()); } - public HoodieRecord(HoodieKey key, T data, HoodieOperation operation) { + public HoodieRecord(HoodieKey key, T data, HoodieOperation operation, Option> metaData) { this.key = key; this.data = data; this.currentLocation = null; this.newLocation = null; this.sealed = false; this.operation = operation; + this.metaData = metaData; } public HoodieRecord( @@ -168,7 +174,7 @@ public HoodieRecord( } public HoodieRecord(HoodieRecord record) { - this(record.key, record.data, record.operation); + this(record.key, record.data, record.operation, record.metaData); this.currentLocation = record.currentLocation; this.newLocation = record.newLocation; this.sealed = record.sealed; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index 862cea8f79af0..f90bb279e5320 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -394,7 +394,14 @@ private void processDataBlock(HoodieDataBlock dataBlock, Option keySpec while (recordIterator.hasNext()) { HoodieRecord currentRecord = recordIterator.next(); Schema schema = schemaOption.isPresent() ? schemaOption.get() : dataBlock.getSchema(); - HoodieRecord record = schemaOption.isPresent() ? currentRecord.rewriteRecordWithNewSchema(dataBlock.getSchema(), new Properties(), schemaOption.get()) : currentRecord; + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into records(Map). + HoodieRecord record = schemaOption.isPresent() + ? currentRecord + .rewriteRecordWithNewSchema(dataBlock.getSchema(), new Properties(), schemaOption.get()) + .copy() + : currentRecord; HoodieRecord completedRecord = record.wrapIntoHoodieRecordPayloadWithParams(schema, hoodieTableMetaClient.getTableConfig().getProps(), this.simpleKeyGenFields, this.withOperationField, this.partitionName, getPopulateMetaFields()); processNextRecord(completedRecord); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java index 219eb39346546..4af28c5db9537 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java @@ -227,7 +227,7 @@ private HoodieLogBlock readBlock() throws IOException { String.format("Parquet block could not be of version (%d)", HoodieLogFormatVersion.DEFAULT_VERSION)); return new HoodieParquetDataBlock(inputStream, content, readBlockLazily, logBlockContentLoc, - Option.ofNullable(readerSchema), header, footer, keyField); + Option.ofNullable(readerSchema), header, footer, keyField, internalSchema); case DELETE_BLOCK: return new HoodieDeleteBlock(content, inputStream, readBlockLazily, Option.of(logBlockContentLoc), header, footer); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index 7d2e2a654a7d5..e96890468b860 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -157,6 +157,9 @@ protected void processNextRecord(HoodieRecord newRecord) throws IOExcepti T oldValue = oldRecord.getData(); HoodieRecord combinedRecord = (HoodieRecord) recordMerger.merge(oldRecord, readerSchema, newRecord, readerSchema, this.getPayloadProps()).get().getLeft(); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into records(Map). // If combinedValue is oldValue, no need rePut oldRecord if (combinedRecord.getData() != oldValue) { records.put(key, combinedRecord.copy()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java index f93947ea0bf13..6bee61264a8d7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java @@ -54,6 +54,9 @@ public static HoodieUnMergedLogRecordScanner.Builder newBuilder() { @Override protected void processNextRecord(HoodieRecord hoodieRecord) throws Exception { + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into queue of BoundedInMemoryExecutor. // Just call callback without merging callback.apply(hoodieRecord.copy()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java index 2e499f4ae9015..22389aa521cea 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java @@ -61,7 +61,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock { private final boolean enablePointLookups; - protected final Schema readerSchema; + protected Schema readerSchema; protected InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java index 79d3b3ec86c3e..a0a6956685c94 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; +import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.io.storage.HoodieFileWriterFactory; @@ -67,8 +68,8 @@ public HoodieParquetDataBlock(FSDataInputStream inputStream, Option readerSchema, Map header, Map footer, - String keyField) { - super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, keyField, false); + String keyField, InternalSchema internalSchema) { + super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, keyField, false, internalSchema); this.compressionCodecName = Option.empty(); } @@ -150,8 +151,17 @@ protected ClosableIterator> readRecordsFromBlockPayload(Hood blockContentLoc.getContentPositionInLogFile(), blockContentLoc.getBlockSize()); + Schema writerSchema = new Schema.Parser().parse(this.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); + if (!internalSchema.isEmptySchema()) { + // we should use write schema to read log file, + // since when we have done some DDL operation, the readerSchema maybe different from writeSchema, avro reader will throw exception. + // eg: origin writeSchema is: "a String, b double" then we add a new column now the readerSchema will be: "a string, c int, b double". it's wrong to use readerSchema to read old log file. + // after we read those record by writeSchema, we rewrite those record with readerSchema in AbstractHoodieLogRecordReader + readerSchema = writerSchema; + } + ClosableIterator> iterator = HoodieFileReaderFactory.getReaderFactory(type).getFileReader(inlineConf, inlineLogFilePath, PARQUET) - .getRecordIterator(readerSchema, readerSchema); + .getRecordIterator(writerSchema, readerSchema); return iterator; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java index 8f704953230ad..b801ccc897a46 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java @@ -18,13 +18,11 @@ package org.apache.hudi.common.util; -import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodiePayloadProps; import org.apache.hudi.common.table.HoodieTableConfig; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.Properties; import java.util.stream.Collectors; @@ -58,10 +56,8 @@ public static String getPayloadClass(Properties properties) { return payloadClass; } - public static List getMergerImpls(Map optParams) { - return Arrays.stream( - optParams.getOrDefault("hoodie.datasource.write.merger.impls", - HoodieAvroRecordMerger.class.getName()).split(",")) + public static List split2List(String param) { + return Arrays.stream(param.split(",")) .map(String::trim).distinct().collect(Collectors.toList()); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/IdentityIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/IdentityIterator.java new file mode 100644 index 0000000000000..feafd0762dd64 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/IdentityIterator.java @@ -0,0 +1,44 @@ +/* + * 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.common.util; + +import java.util.Iterator; + +public class IdentityIterator implements ClosableIterator { + + private Iterator itr; + + public IdentityIterator(Iterator itr) { + this.itr = itr; + } + + @Override + public void close() { + } + + @Override + public boolean hasNext() { + return itr.hasNext(); + } + + @Override + public R next() { + return itr.next(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java index e249da0b8a835..50634d7db5794 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java @@ -105,8 +105,11 @@ public long getTotalRecords() { } private ClosableIterator getIndexedRecordIteratorInternal(Schema schema, Option requestedSchema) throws IOException { - AvroReadSupport.setAvroReadSchema(conf, schema); - if (requestedSchema.isPresent()) { + if (!requestedSchema.isPresent()) { + AvroReadSupport.setAvroReadSchema(conf, schema); + } else { + // Make record schema the same as requestedSchema(reader schema) + AvroReadSupport.setAvroReadSchema(conf, requestedSchema.get()); AvroReadSupport.setRequestedProjection(conf, requestedSchema.get()); } ParquetReader reader = AvroParquetReader.builder(path).withConf(conf).build(); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index 9bbf0a9414723..615d3d05e4dd1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -29,14 +29,14 @@ import org.apache.hudi.HoodieBaseRelation._ import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.utils.SparkInternalSchemaConverter -import org.apache.hudi.common.config.{HoodieMetadataConfig, SerializableConfiguration} +import org.apache.hudi.common.config.{ConfigProperty, HoodieMetadataConfig, SerializableConfiguration} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath import org.apache.hudi.common.model.{FileSlice, HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.util.{ConfigUtils, HoodieRecordUtils, StringUtils} +import org.apache.hudi.common.util.{ConfigUtils, StringUtils} import org.apache.hudi.common.util.StringUtils.isNullOrEmpty import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.config.HoodieWriteConfig @@ -45,7 +45,6 @@ import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema} import org.apache.hudi.io.storage.HoodieAvroHFileReader -import org.apache.spark.SerializableWritable import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -461,9 +460,10 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, } protected def getTableState: HoodieTableState = { - val mergerImpls = ConfigUtils.getMergerImpls(optParams.asJava).asScala.toList - val mergerStrategy = optParams.getOrElse(HoodieWriteConfig.MERGER_STRATEGY.key(), - sqlContext.getConf(HoodieWriteConfig.MERGER_STRATEGY.key(), HoodieWriteConfig.MERGER_STRATEGY.defaultValue())) + val mergerImpls = ConfigUtils.split2List(getConfigValue(HoodieWriteConfig.MERGER_IMPLS)).asScala.toList + + val mergerStrategy = getConfigValue(HoodieWriteConfig.MERGER_STRATEGY, + Option(metaClient.getTableConfig.getMergerStrategy)) // Subset of the state of table's configuration as of at the time of the query HoodieTableState( @@ -657,6 +657,12 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, sparkSession.conf.get(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key, DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean } + + private def getConfigValue(config: ConfigProperty[String], + defaultValueOption: Option[String]=Option.empty): String = { + optParams.getOrElse(config.key(), + sqlContext.getConf(config.key(), defaultValueOption.getOrElse(config.defaultValue()))) + } } object HoodieBaseRelation extends SparkAdapterSupport { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 2ae18bd860da8..9ea0c41650a85 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -862,22 +862,34 @@ object HoodieSparkSqlWriter { case HoodieRecord.HoodieRecordType.SPARK => // ut will use AvroKeyGenerator, so we need to cast it in spark record val sparkKeyGenerator = keyGenerator.asInstanceOf[SparkKeyGeneratorInterface] - val structType = HoodieInternalRowUtils.getCachedSchema(schema) + val targetStructType = HoodieInternalRowUtils.getCachedSchema(schema) + val sourceStructType = df.schema + val (processedSourceStructType, processedStructType) = if (dropPartitionColumns) { + (generateSparkSchemaWithoutPartitionColumns(partitionCols, sourceStructType), + generateSparkSchemaWithoutPartitionColumns(partitionCols, targetStructType)) + } else { + (sourceStructType, targetStructType) + } df.queryExecution.toRdd.mapPartitions { iter => - val projection: Function[InternalRow, InternalRow] = if (dropPartitionColumns) { - val newSchema = generateSparkSchemaWithoutPartitionColumns(partitionCols, structType) - HoodieInternalRowUtils.getCachedUnsafeProjection(structType, newSchema) + val dropProjection: Function[InternalRow, InternalRow] = if (dropPartitionColumns) { + HoodieInternalRowUtils.getCachedUnsafeProjection(sourceStructType, processedSourceStructType) } else { identity } iter.map { internalRow => - val processedRow = projection(internalRow) - val recordKey = sparkKeyGenerator.getRecordKey(internalRow, structType) - val partitionPath = sparkKeyGenerator.getPartitionPath(internalRow, structType) + val recordKey = sparkKeyGenerator.getRecordKey(internalRow, sourceStructType) + val partitionPath = sparkKeyGenerator.getPartitionPath(internalRow, sourceStructType) val key = new HoodieKey(recordKey.toString, partitionPath.toString) - new HoodieSparkRecord(key, processedRow, structType, false) + val processedRow = if (reconcileSchema) { + HoodieInternalRowUtils.getCachedUnsafeProjection(processedStructType, processedStructType) + .apply(HoodieInternalRowUtils.rewriteRecord(dropProjection(internalRow), processedSourceStructType, processedStructType)) + } else { + dropProjection(internalRow) + } + + new HoodieSparkRecord(key, processedRow, processedStructType, false) } }.toJavaRDD().asInstanceOf[JavaRDD[HoodieRecord[_]]] } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/LogFileIterator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/LogFileIterator.scala index 6ee5856d6a0ae..b8b9fab7ff822 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/LogFileIterator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/LogFileIterator.scala @@ -42,7 +42,7 @@ import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompac import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.metadata.HoodieTableMetadata.getDataTableBasePathFromMetadataTable import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata} -import org.apache.spark.sql.HoodieCatalystExpressionUtils +import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieInternalRowUtils} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeProjection import org.apache.spark.sql.types.StructType diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index 7ef0cbf51e88c..65206bad9d5b4 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -165,6 +165,7 @@ trait ProvidesHoodieConfig extends Logging { tableType == COW_TABLE_TYPE_OPT_VAL && insertMode == InsertMode.STRICT) { // Only validate duplicate key for COW, for MOR it will do the merge with the DefaultHoodieRecordPayload // on reading. + // TODO use HoodieSparkValidateDuplicateKeyRecordMerger when SparkRecordMerger is default classOf[ValidateDuplicateKeyPayload].getCanonicalName } else { classOf[OverwriteWithLatestAvroPayload].getCanonicalName diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/HoodieSparkValidateDuplicateKeyRecordMerger.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/HoodieSparkValidateDuplicateKeyRecordMerger.scala new file mode 100644 index 0000000000000..f59b62d7bec94 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/HoodieSparkValidateDuplicateKeyRecordMerger.scala @@ -0,0 +1,47 @@ +/* + * 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.command + +import org.apache.avro.Schema +import org.apache.hudi.HoodieSparkRecordMerger +import org.apache.hudi.common.config.TypedProperties +import org.apache.hudi.common.model.HoodieAvroRecordMerger.Config +import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.util.{collection, Option => HOption} +import org.apache.hudi.exception.HoodieDuplicateKeyException + +/** + * Validate the duplicate key for insert statement without enable the INSERT_DROP_DUPS_OPT + * config. + * @see org.apache.spark.sql.hudi.command.ValidateDuplicateKeyPayload + */ +class HoodieSparkValidateDuplicateKeyRecordMerger extends HoodieSparkRecordMerger { + + override def merge(older: HoodieRecord[_], oldSchema: Schema, newer: HoodieRecord[_], newSchema: Schema, props: TypedProperties): HOption[collection.Pair[HoodieRecord[_], Schema]] = { + val legacyOperatingMode = Config.LegacyOperationMode.valueOf(props.getString(Config.LEGACY_OPERATING_MODE.key, Config.LEGACY_OPERATING_MODE.defaultValue)) + legacyOperatingMode match { + case Config.LegacyOperationMode.PRE_COMBINING => + super.merge(older, oldSchema, newer, newSchema, props) + case Config.LegacyOperationMode.COMBINING => + val key = older.getRecordKey(oldSchema, HoodieRecord.RECORD_KEY_METADATA_FIELD) + throw new HoodieDuplicateKeyException(key) + case _ => + throw new UnsupportedOperationException(String.format("Unsupported legacy operating mode (%s)", legacyOperatingMode)) + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index f0394ad379e7c..7ac3c98e04af2 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hudi.command import org.apache.avro.Schema import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.util.StringUtils +import org.apache.hudi.common.model.HoodieAvroRecordMerger import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.hive.HiveSyncConfigHolder @@ -42,7 +43,6 @@ import org.apache.spark.sql.types.{BooleanType, StructType} import java.util.Base64 - /** * The Command for hoodie MergeIntoTable. * The match on condition must contain the row key fields currently, so that we can use Hoodie @@ -185,9 +185,11 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie // Create the write parameters val parameters = buildMergeIntoConfig(hoodieCatalogTable) + // TODO Remove it when we implement ExpressionPayload for SparkRecord + val parametersWithAvroRecordMerger = parameters ++ Map(HoodieWriteConfig.MERGER_IMPLS.key -> classOf[HoodieAvroRecordMerger].getName) if (mergeInto.matchedActions.nonEmpty) { // Do the upsert - executeUpsert(sourceDF, parameters) + executeUpsert(sourceDF, parametersWithAvroRecordMerger) } else { // If there is no match actions in the statement, execute insert operation only. val targetDF = Dataset.ofRows(sparkSession, mergeInto.targetTable) val primaryKeys = hoodieCatalogTable.tableConfig.getRecordKeyFieldProp.split(",") @@ -196,7 +198,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie // column order changed after left anti join , we should keep column order of source dataframe val cols = removeMetaFields(sourceDF).columns - executeInsertOnly(insertSourceDF.select(cols.head, cols.tail:_*), parameters) + executeInsertOnly(insertSourceDF.select(cols.head, cols.tail:_*), parametersWithAvroRecordMerger) } sparkSession.catalog.refreshTable(targetTableIdentify.unquotedString) Seq.empty[Row] diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieInternalRowUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieInternalRowUtils.scala index 3fabdad4a2f7c..26a343d2ff613 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieInternalRowUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieInternalRowUtils.scala @@ -23,7 +23,7 @@ import org.apache.hudi.testutils.HoodieClientTestUtils import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.types._ -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.{HoodieInternalRowUtils, Row, SparkSession} import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} class TestHoodieInternalRowUtils extends FunSuite with Matchers with BeforeAndAfterAll { @@ -85,18 +85,5 @@ class TestHoodieInternalRowUtils extends FunSuite with Matchers with BeforeAndAf assert(newRow.get(3, IntegerType) == null) } - test("test rewrite with metaDataFiled value") { - val data = sparkSession.sparkContext.parallelize(Seq(Row("like", 18))) - val oldRow = sparkSession.createDataFrame(data, schema1).queryExecution.toRdd.first() - val newRow = HoodieInternalRowUtils.rewriteRecordWithMetadata(oldRow, schema1, schema1WithMetaData, "file1") - assert(newRow.get(0, StringType) == null) - assert(newRow.get(1, StringType) == null) - assert(newRow.get(2, StringType) == null) - assert(newRow.get(3, StringType) == null) - assert(newRow.get(4, StringType).toString.equals("file1")) - assert(newRow.get(5, StringType) == null) - assert(newRow.get(6, BooleanType) == null) - assert(newRow.get(7, StringType).toString.equals("like")) - assert(newRow.get(8, IntegerType) == 18) - } + } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestStructTypeSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestStructTypeSchemaEvolutionUtils.scala index bd2bda59ee5c3..49e21d64bf226 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestStructTypeSchemaEvolutionUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestStructTypeSchemaEvolutionUtils.scala @@ -31,7 +31,7 @@ import org.apache.hudi.internal.schema.utils.SchemaChangeUtils import org.apache.hudi.testutils.HoodieClientTestUtils import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{HoodieInternalRowUtils, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/model/TestHoodieRecordSerialization.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/model/TestHoodieRecordSerialization.scala index 02cb46721c27d..8d0a8d56f5dd8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/model/TestHoodieRecordSerialization.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/model/TestHoodieRecordSerialization.scala @@ -20,13 +20,13 @@ package org.apache.hudi.common.model import org.apache.avro.generic.GenericRecord import org.apache.hudi.AvroConversionUtils.{convertStructTypeToAvroSchema, createInternalRowToAvroConverter} -import org.apache.hudi.{HoodieInternalRowUtils, SparkAdapterSupport} +import org.apache.hudi.SparkAdapterSupport import org.apache.hudi.client.model.HoodieInternalRow import org.apache.hudi.commmon.model.HoodieSparkRecord import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import org.apache.hudi.common.model.TestHoodieRecordSerialization.{OverwriteWithLatestAvroPayloadWithEquality, cloneUsingKryo, convertToAvroRecord, toUnsafeRow} import org.apache.hudi.testutils.SparkClientFunctionalTestHarness -import org.apache.spark.sql.Row +import org.apache.spark.sql.{HoodieInternalRowUtils, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.objects.SerializerSupport diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index a77a14047dfbb..5d76d2af05477 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -36,8 +36,8 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config import org.apache.hudi.metrics.Metrics import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.hudi.util.JFunction -import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieInternalRowUtils, HoodieSparkRecordMerger, QuickstartUtils} -import org.apache.spark.sql._ +import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkRecordMerger, QuickstartUtils} +import org.apache.spark.sql.{HoodieInternalRowUtils, _} import org.apache.spark.sql.functions.{col, concat, lit, udf} import org.apache.spark.sql.hudi.HoodieSparkSessionExtension import org.apache.spark.sql.types._ @@ -102,7 +102,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testShortNameStorage(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // Insert Operation val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList @@ -119,7 +119,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testNoPrecombine(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // Insert Operation val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList @@ -144,7 +144,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testHoodieIsDeletedNonBooleanField(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // Insert Operation val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList @@ -171,7 +171,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testPrunePartitionForTimestampBasedKeyGenerator(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val options = commonOpts ++ Map( "hoodie.compact.inline" -> "false", @@ -235,7 +235,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testArchivalWithBulkInsert(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) var structType : StructType = null for (i <- 1 to 4) { @@ -267,7 +267,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testCopyOnWriteDeletes(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // Insert Operation val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList @@ -302,7 +302,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testOverWriteModeUseReplaceAction(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) @@ -332,7 +332,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testReadPathsOnCopyOnWriteTable(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val records1 = dataGen.generateInsertsContainsAllPartitions("001", 20) val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2)) @@ -381,7 +381,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testOverWriteTableModeUseReplaceAction(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) @@ -411,7 +411,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testOverWriteModeUseReplaceActionOnDisJointPartitions(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // step1: Write 5 records to hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH val records1 = recordsToStrings(dataGen.generateInsertsForPartition("001", 5, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList @@ -471,7 +471,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testOverWriteTableModeUseReplaceActionOnDisJointPartitions(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // step1: Write 5 records to hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH val records1 = recordsToStrings(dataGen.generateInsertsForPartition("001", 5, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList @@ -525,7 +525,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testDropInsertDup(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val insert1Cnt = 10 val insert2DupKeyCnt = 9 @@ -573,7 +573,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testComplexDataTypeWriteAndReadConsistency(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val schema = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, true) :: StructField("timeStampValue", TimestampType, true) :: StructField("dateValue", DateType, true) @@ -610,7 +610,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testWithAutoCommitOn(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) @@ -636,7 +636,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testSparkPartitionByWithCustomKeyGenerator(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // Without fieldType, the default is SIMPLE var writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, writeOpts) @@ -692,7 +692,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @Test def testSparkPartitionByWithSimpleKeyGenerator() { - val (writeOpts, readOpts) = getOpts(HoodieRecordType.AVRO) + val (writeOpts, readOpts) = getWriterReaderOpts(HoodieRecordType.AVRO) // Use the `driver` field as the partition key var writer = getDataFrameWriter(classOf[SimpleKeyGenerator].getName, writeOpts) @@ -718,7 +718,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testSparkPartitionByWithComplexKeyGenerator(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // Use the `driver` field as the partition key var writer = getDataFrameWriter(classOf[ComplexKeyGenerator].getName, writeOpts) @@ -743,7 +743,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testSparkPartitionByWithTimestampBasedKeyGenerator(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val writer = getDataFrameWriter(classOf[TimestampBasedKeyGenerator].getName, writeOpts) writer.partitionBy("current_ts") @@ -762,7 +762,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testSparkPartitionByWithGlobalDeleteKeyGenerator(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val writer = getDataFrameWriter(classOf[GlobalDeleteKeyGenerator].getName, writeOpts) writer.partitionBy("driver") @@ -778,7 +778,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testSparkPartitionByWithNonpartitionedKeyGenerator(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // Empty string column var writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName, writeOpts) @@ -807,7 +807,7 @@ class TestCOWDataSource extends HoodieClientTestBase { "true,false,SPARK", "true,true,SPARK", "false,true,SPARK", "false,false,SPARK" )) def testQueryCOWWithBasePathAndFileIndex(partitionEncode: Boolean, isMetadataEnabled: Boolean, recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val N = 20 // Test query with partition prune if URL_ENCODE_PARTITIONING has enable @@ -864,7 +864,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testSchemaEvolution(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // open the schema validate val opts = commonOpts ++ Map("hoodie.avro.schema.validate" -> "true") ++ @@ -928,7 +928,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testSchemaNotEqualData(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val opts = commonOpts ++ Map("hoodie.avro.schema.validate" -> "true") val schema1 = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, true):: @@ -950,7 +950,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @CsvSource(Array("true, AVRO", "false, AVRO", "true, SPARK", "false, SPARK")) def testCopyOnWriteWithDroppedPartitionColumns(enableDropPartitionColumns: Boolean, recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val records1 = recordsToStrings(dataGen.generateInsertsContainsAllPartitions("000", 100)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) @@ -968,7 +968,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testHoodieIsDeletedCOW(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val numRecords = 100 val numRecordsToDelete = 2 @@ -997,13 +997,16 @@ class TestCOWDataSource extends HoodieClientTestBase { assertEquals(numRecords - numRecordsToDelete, snapshotDF2.count()) } - @Test - def testWriteSmallPrecisionDecimalTable(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testWriteSmallPrecisionDecimalTable(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) .withColumn("shortDecimal", lit(new java.math.BigDecimal(s"2090.0000"))) // create decimalType(8, 4) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) @@ -1011,11 +1014,11 @@ class TestCOWDataSource extends HoodieClientTestBase { // update the value of shortDecimal val inputDF2 = inputDF1.withColumn("shortDecimal", lit(new java.math.BigDecimal(s"3090.0000"))) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) - val readResult = spark.read.format("hudi").load(basePath) + val readResult = spark.read.format("hudi").options(readOpts).load(basePath) assert(readResult.count() == 5) // compare the test result assertEquals(inputDF2.sort("_row_key").select("shortDecimal").collect().map(_.getDecimal(0).toPlainString).mkString(","), @@ -1025,7 +1028,7 @@ class TestCOWDataSource extends HoodieClientTestBase { @ParameterizedTest @CsvSource(Array("true, AVRO", "false, AVRO", "true, SPARK", "false, SPARK")) def testPartitionColumnsProperHandling(useGlobbing: Boolean, recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val _spark = spark import _spark.implicits._ @@ -1101,16 +1104,18 @@ class TestCOWDataSource extends HoodieClientTestBase { } } - @Test - def testSaveAsTableInDifferentModes(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testSaveAsTableInDifferentModes(recordType: HoodieRecordType): Unit = { val options = scala.collection.mutable.Map.empty ++ commonOpts ++ Map("path" -> basePath) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType, options.toMap) // first use the Overwrite mode val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.Append) .saveAsTable("hoodie_test") @@ -1119,28 +1124,28 @@ class TestCOWDataSource extends HoodieClientTestBase { .setBasePath(basePath) .setConf(spark.sessionState.newHadoopConf) .build() - assertEquals(spark.read.format("hudi").load(basePath).count(), 5) + assertEquals(spark.read.format("hudi").options(readOpts).load(basePath).count(), 5) // use the Append mode val records2 = recordsToStrings(dataGen.generateInserts("002", 6)).toList val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.Append) .saveAsTable("hoodie_test") - assertEquals(spark.read.format("hudi").load(basePath).count(), 11) + assertEquals(spark.read.format("hudi").options(readOpts).load(basePath).count(), 11) // use the Ignore mode val records3 = recordsToStrings(dataGen.generateInserts("003", 7)).toList val inputDF3 = spark.read.json(spark.sparkContext.parallelize(records3, 2)) inputDF3.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.Ignore) .saveAsTable("hoodie_test") // nothing to do for the ignore mode - assertEquals(spark.read.format("hudi").load(basePath).count(), 11) + assertEquals(spark.read.format("hudi").options(readOpts).load(basePath).count(), 11) // use the ErrorIfExists mode val records4 = recordsToStrings(dataGen.generateInserts("004", 8)).toList @@ -1148,7 +1153,7 @@ class TestCOWDataSource extends HoodieClientTestBase { try { inputDF4.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.ErrorIfExists) .saveAsTable("hoodie_test") } catch { @@ -1160,20 +1165,23 @@ class TestCOWDataSource extends HoodieClientTestBase { val inputDF5 = spark.read.json(spark.sparkContext.parallelize(records5, 2)) inputDF5.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.Overwrite) .saveAsTable("hoodie_test") - assertEquals(spark.read.format("hudi").load(basePath).count(), 9) + assertEquals(spark.read.format("hudi").options(readOpts).load(basePath).count(), 9) } - @Test - def testMetricsReporterViaDataSource(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testMetricsReporterViaDataSource(recordType: HoodieRecordType): Unit = { + val (writeOpts, _) = getWriterReaderOpts(recordType, getQuickstartWriteConfigs.asScala.toMap) + val dataGenerator = new QuickstartUtils.DataGenerator() val records = convertToStringList(dataGenerator.generateInserts( 10)) val recordsRDD = spark.sparkContext.parallelize(records, 2) val inputDF = spark.read.json(sparkSession.createDataset(recordsRDD)(Encoders.STRING)) inputDF.write.format("hudi") - .options(getQuickstartWriteConfigs) + .options(writeOpts) .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "uuid") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionpath") .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts") @@ -1188,18 +1196,10 @@ class TestCOWDataSource extends HoodieClientTestBase { assertEquals(false, Metrics.isInitialized, "Metrics should be shutdown") } - def getOpts(recordType: HoodieRecordType): (Map[String, String], Map[String, String]) = { - val writeOpts = if (recordType == HoodieRecordType.SPARK) { - commonOpts ++ sparkOpts - } else { - commonOpts + def getWriterReaderOpts(recordType: HoodieRecordType, opt: Map[String, String] = commonOpts): (Map[String, String], Map[String, String]) = { + recordType match { + case HoodieRecordType.SPARK => (opt ++ sparkOpts, sparkOpts) + case _ => (opt, Map.empty[String, String]) } - val readOpts = if (recordType == HoodieRecordType.SPARK) { - sparkOpts - } else { - Map.empty[String, String] - } - - (writeOpts, readOpts) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index be865476d4721..a96b7ba0a5778 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -35,9 +35,9 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config import org.apache.hudi.table.action.compact.CompactionTriggerStrategy import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase} import org.apache.hudi.util.JFunction -import org.apache.hudi.{DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieInternalRowUtils, HoodieSparkRecordMerger, SparkDatasetMixin} +import org.apache.hudi.{DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkRecordMerger, SparkDatasetMixin} import org.apache.log4j.LogManager -import org.apache.spark.sql._ +import org.apache.spark.sql.{HoodieInternalRowUtils, _} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hudi.HoodieSparkSessionExtension import org.apache.spark.sql.types.BooleanType @@ -98,8 +98,8 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @CsvSource(Array("AVRO, AVRO, avro", "AVRO, SPARK, parquet", "SPARK, AVRO, parquet", "SPARK, SPARK, parquet")) def testCount(readType: HoodieRecordType, writeType: HoodieRecordType, logType: String) { - var (_, readOpts) = getOpts(readType) - var (writeOpts, _) = getOpts(writeType) + var (_, readOpts) = getWriterReaderOpts(readType) + var (writeOpts, _) = getWriterReaderOpts(writeType) readOpts = readOpts ++ Map(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> logType) writeOpts = writeOpts ++ Map(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> logType) @@ -318,7 +318,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @Test def testSpill() { - val (writeOpts, readOpts) = getOpts(HoodieRecordType.SPARK) + val (writeOpts, readOpts) = getWriterReaderOpts(HoodieRecordType.SPARK) val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).asScala val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) @@ -350,7 +350,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testPayloadDelete(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // First Operation: // Producing parquet files to three default partitions. @@ -426,7 +426,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testPrunedFiltered(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // First Operation: // Producing parquet files to three default partitions. @@ -533,7 +533,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testVectorizedReader(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) spark.conf.set("spark.sql.parquet.enableVectorizedReader", true) assertTrue(spark.conf.get("spark.sql.parquet.enableVectorizedReader").toBoolean) @@ -584,7 +584,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testNoPrecombine(recordType: HoodieRecordType) { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // Insert Operation val records = recordsToStrings(dataGen.generateInserts("000", 100)).asScala @@ -610,7 +610,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testPreCombineFiledForReadMOR(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) writeData((1, "a0", 10, 100, false), writeOpts) checkAnswer((1, "a0", 10, 100, false), readOpts) @@ -697,7 +697,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { "true,false,SPARK", "true,true,SPARK", "false,true,SPARK", "false,false,SPARK" )) def testQueryMORWithBasePathAndFileIndex(partitionEncode: Boolean, isMetadataEnabled: Boolean, recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val N = 20 // Test query with partition prune if URL_ENCODE_PARTITIONING has enable @@ -758,7 +758,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { "true, false, SPARK", "false, true, SPARK", "false, false, SPARK", "true, true, SPARK" )) def testMORPartitionPrune(partitionEncode: Boolean, hiveStylePartition: Boolean, recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val partitions = Array("2021/03/01", "2021/03/02", "2021/03/03", "2021/03/04", "2021/03/05") val newDataGen = new HoodieTestDataGenerator(partitions) @@ -830,7 +830,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testReadPathsForMergeOnReadTable(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) // Paths only baseFiles val records1 = dataGen.generateInserts("001", 100) @@ -880,7 +880,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testReadPathsForOnlyLogFiles(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) initMetaClient(HoodieTableType.MERGE_ON_READ) val records1 = dataGen.generateInsertsContainsAllPartitions("000", 20) @@ -927,7 +927,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testReadLogOnlyMergeOnReadTable(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) initMetaClient(HoodieTableType.MERGE_ON_READ) val records1 = dataGen.generateInsertsContainsAllPartitions("000", 20) @@ -951,7 +951,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testTempFilesCleanForClustering(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).asScala val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) @@ -972,7 +972,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testClusteringOnNullableColumn(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).asScala val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) @@ -997,7 +997,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testHoodieIsDeletedMOR(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val numRecords = 100 val numRecordsToDelete = 2 @@ -1045,7 +1045,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testPrunePartitionForTimestampBasedKeyGenerator(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getOpts(recordType) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val options = commonOpts ++ Map( "hoodie.compact.inline" -> "false", @@ -1138,8 +1138,9 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { * * The read-optimized query should read `fg1_dc1.parquet` only in this case. */ - @Test - def testReadOptimizedQueryAfterInflightCompactionAndCompletedDeltaCommit(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testReadOptimizedQueryAfterInflightCompactionAndCompletedDeltaCommit(recordType: HoodieRecordType): Unit = { val (tableName, tablePath) = ("hoodie_mor_ro_read_test_table", s"${basePath}_mor_test_table") val precombineField = "col3" val recordKeyField = "key" @@ -1156,6 +1157,8 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { "hoodie.insert.shuffle.parallelism" -> "1", "hoodie.upsert.shuffle.parallelism" -> "1") + val (writeOpts, readOpts) = getWriterReaderOpts(recordType, options) + // First batch with all inserts // Deltacommit1 (DC1, completed), writing file group 1 (fg1) // fg1_dc1.parquet written to storage @@ -1165,7 +1168,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .withColumn(dataField, expr(recordKeyField + " + 1000")) firstDf.write.format("hudi") - .options(options) + .options(writeOpts) .mode(SaveMode.Overwrite) .save(tablePath) @@ -1178,10 +1181,10 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .withColumn(dataField, expr(recordKeyField + " + 2000")) secondDf.write.format("hudi") - .options(options) + .options(writeOpts) .mode(SaveMode.Append).save(tablePath) - val compactionOptions = options ++ Map( + val compactionOptions = writeOpts ++ Map( HoodieCompactionConfig.INLINE_COMPACT_TRIGGER_STRATEGY.key -> CompactionTriggerStrategy.NUM_COMMITS.name, HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key -> "1", DataSourceWriteOptions.ASYNC_COMPACT_ENABLE.key -> "false", @@ -1211,11 +1214,12 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .withColumn(dataField, expr(recordKeyField + " + 3000")) thirdDf.write.format("hudi") - .options(options) + .options(writeOpts) .mode(SaveMode.Append).save(tablePath) // Read-optimized query on MOR val roDf = spark.read.format("org.apache.hudi") + .options(readOpts) .option( DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) @@ -1228,18 +1232,10 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { roDf.where(col(recordKeyField) === 0).select(dataField).collect()(0).getLong(0)) } - def getOpts(recordType: HoodieRecordType): (Map[String, String], Map[String, String]) = { - val writeOpts = if (recordType == HoodieRecordType.SPARK) { - commonOpts ++ sparkOpts - } else { - commonOpts + def getWriterReaderOpts(recordType: HoodieRecordType, opt: Map[String, String] = commonOpts): (Map[String, String], Map[String, String]) = { + recordType match { + case HoodieRecordType.SPARK => (opt ++ sparkOpts, sparkOpts) + case _ => (opt, Map.empty[String, String]) } - val readOpts = if (recordType == HoodieRecordType.SPARK) { - sparkOpts - } else { - Map.empty[String, String] - } - - (writeOpts, readOpts) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala index e7848320ff354..ce29e74b86aa3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala @@ -18,8 +18,12 @@ package org.apache.spark.sql.hudi import org.apache.hadoop.fs.Path -import org.apache.hudi.HoodieSparkUtils +import org.apache.hudi.{HoodieSparkRecordMerger, HoodieSparkUtils} import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.config.HoodieStorageConfig +import org.apache.hudi.common.model.HoodieAvroRecordMerger +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType +import org.apache.hudi.config.HoodieWriteConfig import org.apache.log4j.Level import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -170,4 +174,45 @@ class HoodieSparkSqlTestBase extends FunSuite with BeforeAndAfterAll { val fs = FSUtils.getFs(filePath, spark.sparkContext.hadoopConfiguration) fs.exists(path) } + + protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val conf = spark.sessionState.conf + val currentValues = pairs.unzip._1.map { k => + if (conf.contains(k)) { + Some(conf.getConfString(k)) + } else None + } + pairs.foreach { case(k, v) => conf.setConfString(k, v) } + try f finally { + pairs.unzip._1.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } + + protected def withRecordType(recordConfig: Map[HoodieRecordType, Map[String, String]]=Map.empty)(f: => Unit) { + // TODO HUDI-5264 Test parquet log with avro record in spark sql test + Seq(HoodieRecordType.AVRO, HoodieRecordType.SPARK).foreach { recordType => + val (merger, format) = recordType match { + case HoodieRecordType.SPARK => (classOf[HoodieSparkRecordMerger].getName, "parquet") + case _ => (classOf[HoodieAvroRecordMerger].getName, "avro") + } + val config = Map( + HoodieWriteConfig.MERGER_IMPLS.key -> merger, + HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> format) ++ recordConfig.getOrElse(recordType, Map.empty) + withSQLConf(config.toList:_*) { + f + } + } + } + + protected def getRecordType(): HoodieRecordType = { + val merger = spark.sessionState.conf.getConfString(HoodieWriteConfig.MERGER_IMPLS.key, HoodieWriteConfig.MERGER_IMPLS.defaultValue()) + if (merger.equals(classOf[HoodieSparkRecordMerger].getName)) { + HoodieRecordType.SPARK + } else { + HoodieRecordType.AVRO + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala index 0ef89fc5b9fe3..780d1aad1af2e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hudi class TestCompactionTable extends HoodieSparkSqlTestBase { test("Test compaction table") { - withTempDir {tmp => + withRecordType()(withTempDir {tmp => val tableName = generateTableName spark.sql( s""" @@ -67,11 +67,11 @@ class TestCompactionTable extends HoodieSparkSqlTestBase { Seq(4, "a4", 10.0, 1000) ) assertResult(0)(spark.sql(s"show compaction on $tableName").collect().length) - } + }) } test("Test compaction path") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -124,6 +124,6 @@ class TestCompactionTable extends HoodieSparkSqlTestBase { checkException(s"run compaction on '${tmp.getCanonicalPath}' at 12345")( s"Compaction instant: 12345 is not found in ${tmp.getCanonicalPath}, Available pending compaction instants are: " ) - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteFromTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteFromTable.scala index a972f835e8054..e1c30b1104d35 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteFromTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteFromTable.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hudi class TestDeleteFromTable extends HoodieSparkSqlTestBase { test("Test deleting from table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName spark.sql( @@ -78,6 +78,6 @@ class TestDeleteFromTable extends HoodieSparkSqlTestBase { Seq(3, "a3", 30.0, 3000, "2021-01-07") ) } - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala index ced6fef72d45f..b83ef2a3aeba8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala @@ -20,18 +20,19 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType +import org.apache.spark.sql.hudi.command.HoodieSparkValidateDuplicateKeyRecordMerger import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieDuplicateKeyException import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.internal.SQLConf import java.io.File class TestInsertTable extends HoodieSparkSqlTestBase { test("Test Insert Into with values") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -62,11 +63,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 20.0, 2000, "2021-01-06"), Seq(3, "a3", 30.0, 3000, "2021-01-07") ) - } + }) } test("Test Insert Into with static partition") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -115,11 +116,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 20.0, 2000, "2021-01-06"), Seq(3, "a3", 30.0, 3000, "2021-01-07") ) - } + }) } test("Test Insert Into with dynamic partition") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -169,11 +170,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 20.0, 2000, "2021-01-06"), Seq(3, "a3", 30.0, 3000, "2021-01-07") ) - } + }) } test("Test Insert Into with multi partition") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -222,83 +223,87 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(3, "a3", 30.0, 3000, "20210103", "03"), Seq(4, "a4", 40.0, 4000, "20210104", "04") ) - } + }) } test("Test Insert Into None Partitioned Table") { - withTempDir { tmp => - val tableName = generateTableName - spark.sql(s"set hoodie.sql.insert.mode=strict") - // Create none partitioned cow table - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | location '${tmp.getCanonicalPath}/$tableName' - | tblproperties ( - | type = 'cow', - | primaryKey = 'id', - | preCombineField = 'ts' - | ) - """.stripMargin) - spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") - checkAnswer(s"select id, name, price, ts from $tableName")( - Seq(1, "a1", 10.0, 1000) - ) - spark.sql(s"insert into $tableName select 2, 'a2', 12, 1000") - checkAnswer(s"select id, name, price, ts from $tableName")( - Seq(1, "a1", 10.0, 1000), - Seq(2, "a2", 12.0, 1000) - ) - - assertThrows[HoodieDuplicateKeyException] { - try { - spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") - } catch { - case e: Exception => - var root: Throwable = e - while (root.getCause != null) { - root = root.getCause - } - throw root - } - } - // Create table with dropDup is true - val tableName2 = generateTableName - spark.sql("set hoodie.datasource.write.insert.drop.duplicates = true") - spark.sql( - s""" - |create table $tableName2 ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | location '${tmp.getCanonicalPath}/$tableName2' - | tblproperties ( - | type = 'mor', - | primaryKey = 'id', - | preCombineField = 'ts' - | ) - """.stripMargin) - spark.sql(s"insert into $tableName2 select 1, 'a1', 10, 1000") - // This record will be drop when dropDup is true - spark.sql(s"insert into $tableName2 select 1, 'a1', 12, 1000") - checkAnswer(s"select id, name, price, ts from $tableName2")( - Seq(1, "a1", 10.0, 1000) - ) - // disable this config to avoid affect other test in this class. - spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false") - spark.sql(s"set hoodie.sql.insert.mode=upsert") - } + withRecordType(Map(HoodieRecordType.SPARK -> + // SparkMerger should use "HoodieSparkValidateDuplicateKeyRecordMerger" + // with "hoodie.sql.insert.mode=strict" + Map(HoodieWriteConfig.MERGER_IMPLS.key -> + classOf[HoodieSparkValidateDuplicateKeyRecordMerger].getName)))(withTempDir { tmp => + val tableName = generateTableName + spark.sql(s"set hoodie.sql.insert.mode=strict") + // Create none partitioned cow table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'cow', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 10.0, 1000) + ) + spark.sql(s"insert into $tableName select 2, 'a2', 12, 1000") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 12.0, 1000) + ) + + assertThrows[HoodieDuplicateKeyException] { + try { + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + } catch { + case e: Exception => + var root: Throwable = e + while (root.getCause != null) { + root = root.getCause + } + throw root + } + } + // Create table with dropDup is true + val tableName2 = generateTableName + spark.sql("set hoodie.datasource.write.insert.drop.duplicates = true") + spark.sql( + s""" + |create table $tableName2 ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName2' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + spark.sql(s"insert into $tableName2 select 1, 'a1', 10, 1000") + // This record will be drop when dropDup is true + spark.sql(s"insert into $tableName2 select 1, 'a1', 12, 1000") + checkAnswer(s"select id, name, price, ts from $tableName2")( + Seq(1, "a1", 10.0, 1000) + ) + // disable this config to avoid affect other test in this class. + spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false") + spark.sql(s"set hoodie.sql.insert.mode=upsert") + }) } test("Test Insert Overwrite") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -388,27 +393,27 @@ class TestInsertTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, price, ts from $tblNonPartition")( Seq(2, "a2", 10.0, 1000) ) - } + }) } test("Test Different Type of Partition Column") { - withTempDir { tmp => - val typeAndValue = Seq( - ("string", "'1000'"), - ("int", 1000), - ("bigint", 10000), - ("timestamp", "TIMESTAMP'2021-05-20 00:00:00'"), - ("date", "DATE'2021-05-20'") - ) - typeAndValue.foreach { case (partitionType, partitionValue) => - val tableName = generateTableName - validateDifferentTypesOfPartitionColumn(tmp, partitionType, partitionValue, tableName) - } - } + withRecordType()(withTempDir { tmp => + val typeAndValue = Seq( + ("string", "'1000'"), + ("int", 1000), + ("bigint", 10000), + ("timestamp", "TIMESTAMP'2021-05-20 00:00:00'"), + ("date", "DATE'2021-05-20'") + ) + typeAndValue.foreach { case (partitionType, partitionValue) => + val tableName = generateTableName + validateDifferentTypesOfPartitionColumn(tmp, partitionType, partitionValue, tableName) + } + }) } test("Test TimestampType Partition Column With Consistent Logical Timestamp Enabled") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val typeAndValue = Seq( ("timestamp", "TIMESTAMP'2021-05-20 00:00:00'"), ("date", "DATE'2021-05-20'") @@ -418,33 +423,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { spark.sql(s"set hoodie.datasource.write.keygenerator.consistent.logical.timestamp.enabled=true") validateDifferentTypesOfPartitionColumn(tmp, partitionType, partitionValue, tableName) } - } - } - - private def validateDifferentTypesOfPartitionColumn(tmp: File, partitionType: String, partitionValue: Any, tableName: String) = { - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | dt $partitionType - |) using hudi - | tblproperties (primaryKey = 'id') - | partitioned by (dt) - | location '${tmp.getCanonicalPath}/$tableName' - """.stripMargin) - // NOTE: We have to drop type-literal prefix since Spark doesn't parse type literals appropriately - spark.sql(s"insert into $tableName partition(dt = ${dropTypeLiteralPrefix(partitionValue)}) select 1, 'a1', 10") - spark.sql(s"insert into $tableName select 2, 'a2', 10, $partitionValue") - checkAnswer(s"select id, name, price, cast(dt as string) from $tableName order by id")( - Seq(1, "a1", 10, extractRawValue(partitionValue).toString), - Seq(2, "a2", 10, extractRawValue(partitionValue).toString) - ) + }) } test("Test insert for uppercase table name") { - withTempDir{ tmp => + withRecordType()(withTempDir{ tmp => val tableName = s"H_$generateTableName" spark.sql( @@ -467,82 +450,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { .setConf(spark.sessionState.newHadoopConf()) .build() assertResult(metaClient.getTableConfig.getTableName)(tableName) - } + }) } test("Test Insert Exception") { - val tableName = generateTableName - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | dt string - |) using hudi - | tblproperties (primaryKey = 'id') - | partitioned by (dt) - """.stripMargin) - checkException(s"insert into $tableName partition(dt = '2021-06-20') select 1, 'a1', 10, '2021-06-20'") ( - "Expected table's schema: " + - "[StructField(id,IntegerType,true), StructField(name,StringType,true), StructField(price,DoubleType,true), StructField(dt,StringType,true)], " + - "query's output (including static partition values): " + - "[StructField(1,IntegerType,false), StructField(a1,StringType,false), StructField(10,IntegerType,false), StructField(2021-06-20,StringType,false), StructField(dt,StringType,true)]" - ) - checkException(s"insert into $tableName select 1, 'a1', 10")( - "Expected table's schema: " + - "[StructField(id,IntegerType,true), StructField(name,StringType,true), StructField(price,DoubleType,true), StructField(dt,StringType,true)], " + - "query's output (including static partition values): " + - "[StructField(1,IntegerType,false), StructField(a1,StringType,false), StructField(10,IntegerType,false)]" - ) - spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql("set hoodie.sql.insert.mode = strict") - - val tableName2 = generateTableName - spark.sql( - s""" - |create table $tableName2 ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | tblproperties ( - | primaryKey = 'id', - | preCombineField = 'ts' - | ) - """.stripMargin) - checkException(s"insert into $tableName2 values(1, 'a1', 10, 1000)")( - "Table with primaryKey can not use bulk insert in strict mode." - ) - - spark.sql("set hoodie.sql.insert.mode = non-strict") - val tableName3 = generateTableName - spark.sql( - s""" - |create table $tableName3 ( - | id int, - | name string, - | price double, - | dt string - |) using hudi - | tblproperties (primaryKey = 'id') - | partitioned by (dt) - """.stripMargin) - checkException(s"insert overwrite table $tableName3 values(1, 'a1', 10, '2021-07-18')")( - "Insert Overwrite Partition can not use bulk insert." - ) - spark.sql("set hoodie.sql.bulk.insert.enable = false") - spark.sql("set hoodie.sql.insert.mode = upsert") - } - - - test("Test Insert timestamp when 'spark.sql.datetime.java8API.enabled' enables") { - try { - // enable spark.sql.datetime.java8API.enabled - // and use java.time.Instant to replace java.sql.Timestamp to represent TimestampType. - spark.conf.set("spark.sql.datetime.java8API.enabled", value = true) - + withRecordType() { val tableName = generateTableName spark.sql( s""" @@ -550,209 +462,276 @@ class TestInsertTable extends HoodieSparkSqlTestBase { | id int, | name string, | price double, - | dt timestamp - |) - |using hudi - |partitioned by(dt) - |options(type = 'cow', primaryKey = 'id') - |""".stripMargin + | dt string + |) using hudi + | tblproperties (primaryKey = 'id') + | partitioned by (dt) + """.stripMargin) + checkException(s"insert into $tableName partition(dt = '2021-06-20') select 1, 'a1', 10, '2021-06-20'")( + "Expected table's schema: " + + "[StructField(id,IntegerType,true), StructField(name,StringType,true), StructField(price,DoubleType,true), StructField(dt,StringType,true)], " + + "query's output (including static partition values): " + + "[StructField(1,IntegerType,false), StructField(a1,StringType,false), StructField(10,IntegerType,false), StructField(2021-06-20,StringType,false), StructField(dt,StringType,true)]" ) - - spark.sql(s"insert into $tableName values (1, 'a1', 10, cast('2021-05-07 00:00:00' as timestamp))") - checkAnswer(s"select id, name, price, cast(dt as string) from $tableName")( - Seq(1, "a1", 10, "2021-05-07 00:00:00") + checkException(s"insert into $tableName select 1, 'a1', 10")( + "Expected table's schema: " + + "[StructField(id,IntegerType,true), StructField(name,StringType,true), StructField(price,DoubleType,true), StructField(dt,StringType,true)], " + + "query's output (including static partition values): " + + "[StructField(1,IntegerType,false), StructField(a1,StringType,false), StructField(10,IntegerType,false)]" ) - - } finally { - spark.conf.set("spark.sql.datetime.java8API.enabled", value = false) - } - } - - test("Test bulk insert") { - spark.sql("set hoodie.sql.insert.mode = non-strict") - withTempDir { tmp => - Seq("cow", "mor").foreach {tableType => - // Test bulk insert for single partition - val tableName = generateTableName + withSQLConf("hoodie.sql.bulk.insert.enable" -> "true", "hoodie.sql.insert.mode" -> "strict") { + val tableName2 = generateTableName spark.sql( s""" - |create table $tableName ( + |create table $tableName2 ( | id int, | name string, | price double, - | dt string + | ts long |) using hudi | tblproperties ( - | type = '$tableType', - | primaryKey = 'id' + | primaryKey = 'id', + | preCombineField = 'ts' | ) - | partitioned by (dt) - | location '${tmp.getCanonicalPath}/$tableName' """.stripMargin) - spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false") - - // Enable the bulk insert - spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')") - - checkAnswer(s"select id, name, price, dt from $tableName")( - Seq(1, "a1", 10.0, "2021-07-18") + checkException(s"insert into $tableName2 values(1, 'a1', 10, 1000)")( + "Table with primaryKey can not use bulk insert in strict mode." ) - // Disable the bulk insert - spark.sql("set hoodie.sql.bulk.insert.enable = false") - spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')") - checkAnswer(s"select id, name, price, dt from $tableName order by id")( - Seq(1, "a1", 10.0, "2021-07-18"), - Seq(2, "a2", 10.0, "2021-07-18") - ) - - // Test bulk insert for multi-level partition - val tableMultiPartition = generateTableName + spark.sql("set hoodie.sql.insert.mode = non-strict") + val tableName3 = generateTableName spark.sql( s""" - |create table $tableMultiPartition ( + |create table $tableName3 ( | id int, | name string, | price double, - | dt string, - | hh string + | dt string |) using hudi - | tblproperties ( - | type = '$tableType', - | primaryKey = 'id' - | ) - | partitioned by (dt, hh) - | location '${tmp.getCanonicalPath}/$tableMultiPartition' + | tblproperties (primaryKey = 'id') + | partitioned by (dt) """.stripMargin) + checkException(s"insert overwrite table $tableName3 values(1, 'a1', 10, '2021-07-18')")( + "Insert Overwrite Partition can not use bulk insert." + ) + } + } + } - // Enable the bulk insert - spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql(s"insert into $tableMultiPartition values(1, 'a1', 10, '2021-07-18', '12')") - checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition")( - Seq(1, "a1", 10.0, "2021-07-18", "12") - ) - // Disable the bulk insert - spark.sql("set hoodie.sql.bulk.insert.enable = false") - spark.sql(s"insert into $tableMultiPartition " + - s"values(2, 'a2', 10, '2021-07-18','12')") - - checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition order by id")( - Seq(1, "a1", 10.0, "2021-07-18", "12"), - Seq(2, "a2", 10.0, "2021-07-18", "12") - ) - // Test bulk insert for non-partitioned table - val nonPartitionedTable = generateTableName + test("Test Insert timestamp when 'spark.sql.datetime.java8API.enabled' enables") { + withRecordType() { + withSQLConf("spark.sql.datetime.java8API.enabled" -> "true") { + val tableName = generateTableName spark.sql( s""" - |create table $nonPartitionedTable ( + |create table $tableName ( | id int, | name string, - | price double - |) using hudi - | tblproperties ( - | type = '$tableType', - | primaryKey = 'id' - | ) - | location '${tmp.getCanonicalPath}/$nonPartitionedTable' - """.stripMargin) - spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql(s"insert into $nonPartitionedTable values(1, 'a1', 10)") - checkAnswer(s"select id, name, price from $nonPartitionedTable")( - Seq(1, "a1", 10.0) - ) - spark.sql(s"insert overwrite table $nonPartitionedTable values(2, 'a2', 10)") - checkAnswer(s"select id, name, price from $nonPartitionedTable")( - Seq(2, "a2", 10.0) + | price double, + | dt timestamp + |) + |using hudi + |partitioned by(dt) + |options(type = 'cow', primaryKey = 'id') + |""".stripMargin ) - spark.sql("set hoodie.sql.bulk.insert.enable = false") - // Test CTAS for bulk insert - val tableName2 = generateTableName - spark.sql( - s""" - |create table $tableName2 - |using hudi - |tblproperties( - | type = '$tableType', - | primaryKey = 'id' - |) - | location '${tmp.getCanonicalPath}/$tableName2' - | as - | select * from $tableName - |""".stripMargin) - checkAnswer(s"select id, name, price, dt from $tableName2 order by id")( - Seq(1, "a1", 10.0, "2021-07-18"), - Seq(2, "a2", 10.0, "2021-07-18") + spark.sql(s"insert into $tableName values (1, 'a1', 10, cast('2021-05-07 00:00:00' as timestamp))") + checkAnswer(s"select id, name, price, cast(dt as string) from $tableName")( + Seq(1, "a1", 10, "2021-05-07 00:00:00") ) } } - spark.sql("set hoodie.sql.insert.mode = upsert") + } + + test("Test bulk insert") { + withSQLConf("hoodie.sql.insert.mode" -> "non-strict") { + withRecordType()(withTempDir { tmp => + Seq("cow", "mor").foreach {tableType => + // Test bulk insert for single partition + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | dt string + |) using hudi + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' + | ) + | partitioned by (dt) + | location '${tmp.getCanonicalPath}/$tableName' + """.stripMargin) + spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false") + + // Enable the bulk insert + spark.sql("set hoodie.sql.bulk.insert.enable = true") + spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')") + + checkAnswer(s"select id, name, price, dt from $tableName")( + Seq(1, "a1", 10.0, "2021-07-18") + ) + // Disable the bulk insert + spark.sql("set hoodie.sql.bulk.insert.enable = false") + spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')") + + checkAnswer(s"select id, name, price, dt from $tableName order by id")( + Seq(1, "a1", 10.0, "2021-07-18"), + Seq(2, "a2", 10.0, "2021-07-18") + ) + + // Test bulk insert for multi-level partition + val tableMultiPartition = generateTableName + spark.sql( + s""" + |create table $tableMultiPartition ( + | id int, + | name string, + | price double, + | dt string, + | hh string + |) using hudi + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' + | ) + | partitioned by (dt, hh) + | location '${tmp.getCanonicalPath}/$tableMultiPartition' + """.stripMargin) + + // Enable the bulk insert + spark.sql("set hoodie.sql.bulk.insert.enable = true") + spark.sql(s"insert into $tableMultiPartition values(1, 'a1', 10, '2021-07-18', '12')") + + checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition")( + Seq(1, "a1", 10.0, "2021-07-18", "12") + ) + // Disable the bulk insert + spark.sql("set hoodie.sql.bulk.insert.enable = false") + spark.sql(s"insert into $tableMultiPartition " + + s"values(2, 'a2', 10, '2021-07-18','12')") + + checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition order by id")( + Seq(1, "a1", 10.0, "2021-07-18", "12"), + Seq(2, "a2", 10.0, "2021-07-18", "12") + ) + // Test bulk insert for non-partitioned table + val nonPartitionedTable = generateTableName + spark.sql( + s""" + |create table $nonPartitionedTable ( + | id int, + | name string, + | price double + |) using hudi + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' + | ) + | location '${tmp.getCanonicalPath}/$nonPartitionedTable' + """.stripMargin) + spark.sql("set hoodie.sql.bulk.insert.enable = true") + spark.sql(s"insert into $nonPartitionedTable values(1, 'a1', 10)") + checkAnswer(s"select id, name, price from $nonPartitionedTable")( + Seq(1, "a1", 10.0) + ) + spark.sql(s"insert overwrite table $nonPartitionedTable values(2, 'a2', 10)") + checkAnswer(s"select id, name, price from $nonPartitionedTable")( + Seq(2, "a2", 10.0) + ) + spark.sql("set hoodie.sql.bulk.insert.enable = false") + + // Test CTAS for bulk insert + val tableName2 = generateTableName + spark.sql( + s""" + |create table $tableName2 + |using hudi + |tblproperties( + | type = '$tableType', + | primaryKey = 'id' + |) + | location '${tmp.getCanonicalPath}/$tableName2' + | as + | select * from $tableName + |""".stripMargin) + checkAnswer(s"select id, name, price, dt from $tableName2 order by id")( + Seq(1, "a1", 10.0, "2021-07-18"), + Seq(2, "a2", 10.0, "2021-07-18") + ) + } + }) + } } test("Test combine before insert") { - spark.sql("set hoodie.sql.bulk.insert.enable = false") - withTempDir{tmp => - val tableName = generateTableName - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | location '${tmp.getCanonicalPath}/$tableName' - | tblproperties ( - | primaryKey = 'id', - | preCombineField = 'ts' - | ) + withSQLConf("set hoodie.sql.bulk.insert.enable" -> "false") { + withRecordType()(withTempDir{tmp => + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) """.stripMargin) - spark.sql( - s""" - |insert overwrite table $tableName - |select * from ( - | select 1 as id, 'a1' as name, 10 as price, 1000 as ts - | union all - | select 1 as id, 'a1' as name, 11 as price, 1001 as ts - | ) - |""".stripMargin - ) - checkAnswer(s"select id, name, price, ts from $tableName")( - Seq(1, "a1", 11.0, 1001) - ) + spark.sql( + s""" + |insert overwrite table $tableName + |select * from ( + | select 1 as id, 'a1' as name, 10 as price, 1000 as ts + | union all + | select 1 as id, 'a1' as name, 11 as price, 1001 as ts + | ) + |""".stripMargin + ) + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 11.0, 1001) + ) + }) } } test("Test insert pk-table") { - spark.sql("set hoodie.sql.bulk.insert.enable = false") - withTempDir{tmp => - val tableName = generateTableName - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | location '${tmp.getCanonicalPath}/$tableName' - | tblproperties ( - | primaryKey = 'id', - | preCombineField = 'ts' - | ) + withSQLConf("hoodie.sql.bulk.insert.enable" -> "false") { + withRecordType()(withTempDir{tmp => + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) """.stripMargin) - spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") - spark.sql(s"insert into $tableName values(1, 'a1', 11, 1000)") - checkAnswer(s"select id, name, price, ts from $tableName")( - Seq(1, "a1", 11.0, 1000) - ) + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(1, 'a1', 11, 1000)") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 11.0, 1000) + ) + }) } } test("Test For read operation's field") { - withTempDir { tmp => { + withRecordType()(withTempDir { tmp => { val tableName = generateTableName val tablePath = s"${tmp.getCanonicalPath}/$tableName" import spark.implicits._ @@ -790,44 +769,45 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(1, null) ) } - } + }) } test("Test enable hoodie.datasource.write.drop.partition.columns when write") { - spark.sql("set hoodie.sql.bulk.insert.enable = false") - Seq("mor", "cow").foreach { tableType => - withTempDir { tmp => - val tableName = generateTableName - spark.sql( - s""" - | create table $tableName ( - | id int, - | name string, - | price double, - | ts long, - | dt string - | ) using hudi - | partitioned by (dt) - | location '${tmp.getCanonicalPath}/$tableName' - | tblproperties ( - | primaryKey = 'id', - | preCombineField = 'ts', - | type = '$tableType', - | hoodie.datasource.write.drop.partition.columns = 'true' - | ) + withSQLConf("hoodie.sql.bulk.insert.enable" -> "false") { + Seq("mor", "cow").foreach { tableType => + withRecordType()(withTempDir { tmp => + val tableName = generateTableName + spark.sql( + s""" + | create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | dt string + | ) using hudi + | partitioned by (dt) + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts', + | type = '$tableType', + | hoodie.datasource.write.drop.partition.columns = 'true' + | ) """.stripMargin) - spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (1, 'a1', 10, 1000)") - spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (2, 'a2', 20, 1000)") - checkAnswer(s"select id, name, price, ts, dt from $tableName")( - Seq(1, "a1", 10, 1000, "2021-12-25"), - Seq(2, "a2", 20, 1000, "2021-12-25") - ) + spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (2, 'a2', 20, 1000)") + checkAnswer(s"select id, name, price, ts, dt from $tableName")( + Seq(1, "a1", 10, 1000, "2021-12-25"), + Seq(2, "a2", 20, 1000, "2021-12-25") + ) + }) } } } test("Test nested field as primaryKey and preCombineField") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName // create table @@ -856,12 +836,12 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq("name_1", 10.0, 1000, "a", 999) ) } - } + }) } test("Test Insert Into With Catalog Identifier for spark >= 3.2.0") { Seq("hudi", "parquet").foreach { format => - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = s"spark_catalog.default.$generateTableName" // Create a partitioned table if (HoodieSparkUtils.gteqSpark3_2) { @@ -898,7 +878,29 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 10.0, 1000, "2021-01-05") ) } - } + }) } } + + private def validateDifferentTypesOfPartitionColumn(tmp: File, partitionType: String, partitionValue: Any, tableName: String) = { + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | dt $partitionType + |) using hudi + | tblproperties (primaryKey = 'id') + | partitioned by (dt) + | location '${tmp.getCanonicalPath}/$tableName' + """.stripMargin) + // NOTE: We have to drop type-literal prefix since Spark doesn't parse type literals appropriately + spark.sql(s"insert into $tableName partition(dt = ${dropTypeLiteralPrefix(partitionValue)}) select 1, 'a1', 10") + spark.sql(s"insert into $tableName select 2, 'a2', 10, $partitionValue") + checkAnswer(s"select id, name, price, cast(dt as string) from $tableName order by id")( + Seq(1, "a1", 10, extractRawValue(partitionValue).toString), + Seq(2, "a2", 10, extractRawValue(partitionValue).toString) + ) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala index 232b6bbb511c5..48ee872d4d95f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala @@ -22,7 +22,7 @@ import org.apache.hudi.testutils.DataSourceTestUtils class TestMergeIntoLogOnlyTable extends HoodieSparkSqlTestBase { test("Test Query Log Only MOR Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => // Create table with INMEMORY index to generate log only mor table. val tableName = generateTableName spark.sql( @@ -86,6 +86,6 @@ class TestMergeIntoLogOnlyTable extends HoodieSparkSqlTestBase { Seq(3, "a3", 10.0, 1000), Seq(4, "a4", 11.0, 1000) ) - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala index 4aa91498b110b..0981f74a102ca 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala @@ -23,7 +23,7 @@ import org.apache.hudi.common.fs.FSUtils class TestMergeIntoTable extends HoodieSparkSqlTestBase { test("Test MergeInto Basic") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -109,11 +109,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { """.stripMargin) val cnt = spark.sql(s"select * from $tableName where id = 1").count() assertResult(0)(cnt) - } + }) } test("Test MergeInto with ignored record") { - withTempDir {tmp => + withRecordType()(withTempDir {tmp => val sourceTable = generateTableName val targetTable = generateTableName // Create source table @@ -187,11 +187,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 20.0, 1001), Seq(3, "a3", 12.0, 1000) ) - } + }) } test("Test MergeInto for MOR table ") { - withTempDir {tmp => + withRecordType()(withTempDir {tmp => val tableName = generateTableName // Create a mor partitioned table. spark.sql( @@ -298,11 +298,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,price,dt from $tableName order by id")( Seq(1, "a1", 12, "2021-03-21") ) - } + }) } test("Test MergeInto with insert only") { - withTempDir {tmp => + withRecordType()(withTempDir {tmp => // Create a partitioned mor table val tableName = generateTableName spark.sql( @@ -352,11 +352,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 10, "2021-03-21"), Seq(2, "a2", 10, "2021-03-20") ) - } + }) } test("Test MergeInto For PreCombineField") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName1 = generateTableName // Create a mor partitioned table. @@ -425,11 +425,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 12, "2021-03-21", 1002) ) } - } + }) } test("Test MergeInto with preCombine field expression") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName1 = generateTableName spark.sql( @@ -485,11 +485,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 24, "2021-03-21", 1002) ) } - } + }) } test("Test MergeInto with primaryKey expression") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName1 = generateTableName spark.sql( s""" @@ -544,11 +544,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,price,v,dt from $tableName1 order by id")( Seq(1, "a1", 10, 1000, "2021-03-21") ) - } + }) } test("Test MergeInto with combination of delete update insert") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val sourceTable = generateTableName val targetTable = generateTableName // Create source table @@ -606,11 +606,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(11, "s11", 110, 2011, "2021-03-21"), Seq(12, "s12", 120, 2012, "2021-03-21") ) - } + }) } test("Merge Hudi to Hudi") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val sourceTable = generateTableName spark.sql( @@ -711,11 +711,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 10, 1001) ) } - } + }) } test("Test Different Type of PreCombineField") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val typeAndValue = Seq( ("string", "'1000'"), ("int", 1000), @@ -771,11 +771,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 20.0) ) } - } + }) } test("Test MergeInto For MOR With Compaction On") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -821,11 +821,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(3, "a3", 10.0, 1000), Seq(4, "a4", 11.0, 1000) ) - } + }) } test("Test MereInto With Null Fields") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val types = Seq( "string" , "int", @@ -866,11 +866,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", null, 1000) ) } - } + }) } test("Test MergeInto With All Kinds Of DataType") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val dataAndTypes = Seq( ("string", "'a1'"), ("int", "10"), @@ -912,11 +912,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", extractRawValue(dataValue), 1000) ) } - } + }) } test("Test MergeInto with no-full fields source") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -947,11 +947,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, value, ts from $tableName")( Seq(1, "a1", 10, 1001) ) - } + }) } test("Test Merge Into with target matched columns cast-ed") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -1027,6 +1027,6 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, value, ts from $tableName")( Seq(1, "a1", 10, 1004) ) - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala index 8a6aa9691d936..768ee768af4d2 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.Row class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { test("Test MergeInto for MOR table 2") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a mor partitioned table. spark.sql( @@ -137,11 +137,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { | price = s0.price + t0.price, ts = s0.ts, dt = s0.dt """.stripMargin )("assertion failed: Target table's field(price) cannot be the right-value of the update clause for MOR table.") - } + }) } test("Test Merge Into CTAS Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -172,11 +172,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name from $tableName")( Seq(1, "a1_1") ) - } + }) } test("Test Merge With Complex Data Type") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -235,11 +235,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, s_value, a_value, m_value, ts from $tableName")( Seq(1, "a1", Row(1, "12"), Seq("a0", "a1", "a2"), Map("k1" -> "v1"), 1000) ) - } + }) } test("Test column name matching for insert * and update set *") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -318,11 +318,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(3, "a3", 102.0, 1000, "2021-05-05"), Seq(4, "a4", 100.0, 1000, "2021-05-06") ) - } + }) } test("Test MergeInto For Source Table With Column Aliases") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -361,11 +361,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, "a1", 10.0, 1000) ) } - } + }) } test("Test MergeInto When PrimaryKey And PreCombineField Of Source Table And Target Table Differ In Case Only") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -430,11 +430,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, "a1", 11.0, 1001), Seq(2, "a2", 12.0, 1002) ) - } + }) } test("Test ignoring case") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -501,11 +501,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, "a1", 111.0, 1111, "2021-05-05"), Seq(2, "a2", 112.0, 1112, "2021-05-05") ) - } + }) } test("Test ignoring case for MOR table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a mor partitioned table. spark.sql( @@ -541,11 +541,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, price, ts, dt from $tableName")( Seq(1, "a1", 111.0, 1111, "2021-05-05") ) - } + }) } test("Test only insert when source table contains history") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -586,11 +586,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, "a1", 1.0, 10, "2022-08-18"), Seq(2, "a2", 10.0, 100, "2022-08-18") ) - } + }) } test("Test only insert when source table contains history and target table has multiple keys") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table with multiple keys spark.sql( @@ -633,11 +633,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, 1, "a1", 1.0, 10, "2022-08-18"), Seq(1, 2, "a2", 10.0, 100, "2022-08-18") ) - } + }) } test("Test Merge Into For Source Table With Different Column Order") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a mor partitioned table. spark.sql( @@ -671,11 +671,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,price,dt from $tableName")( Seq(1, "a1", 10, "2021-03-21") ) - } + }) } test("Test Merge into with String cast to Double") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a cow partitioned table. spark.sql( @@ -711,11 +711,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,price,dt from $tableName")( Seq(1, "a1", 10.1, "2021-03-21") ) - } + }) } test("Test Merge into where manually set DefaultHoodieRecordPayload") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a cow table with default payload class, check whether it will be overwritten by ExpressionPayload. // if not, this ut cannot pass since DefaultHoodieRecordPayload can not promotion int to long when insert a ts with Integer value @@ -749,6 +749,6 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,ts from $tableName")( Seq(1, "a2", 1000) ) - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala index 65357b903b5ba..8467e0704a2e7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala @@ -18,11 +18,13 @@ package org.apache.spark.sql.hudi import org.apache.hadoop.fs.Path +import org.apache.hudi.common.config.HoodieStorageConfig import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload import org.apache.hudi.config.HoodieWriteConfig -import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils} +import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkRecordMerger, HoodieSparkUtils} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.functions.{arrays_zip, col} import org.apache.spark.sql.{Row, SaveMode, SparkSession} @@ -64,7 +66,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } test("Test multi change data type") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -131,11 +133,11 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { spark.sessionState.catalog.refreshTable(TableIdentifier(tableName)) } } - } + }) } test("Test multi change data type2") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -168,11 +170,11 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } - } + }) } test("Test Partition Table alter ") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -270,11 +272,11 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 or id = 11 order by id").show(false) } } - } + }) } test("Test Chinese table ") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -320,12 +322,12 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } - } + }) } test("Test Alter Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -382,12 +384,12 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } - } + }) } test("Test Alter Table complex") { - withTempDir { tmp => - Seq("cow", "mor").foreach { tableType => + withRecordType()(withTempDir { tmp => + Seq("mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" if (HoodieSparkUtils.gteqSpark3_1) { @@ -470,11 +472,11 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } - } + }) } test("Test schema auto evolution") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("COPY_ON_WRITE", "MERGE_ON_READ").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -500,14 +502,15 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY -> "true" ) + val (writeOpt, readOpt) = getWriterReaderOpts(getRecordType(), hudiOptions) orgStringDf.write .format("org.apache.hudi") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) - .options(hudiOptions) + .options(writeOpt) .mode(SaveMode.Overwrite) .save(tablePath) - val oldView = spark.read.format("hudi").load(tablePath) + val oldView = spark.read.format("hudi").options(readOpt).load(tablePath) oldView.show(false) val records2 = RawTripTestPayload.recordsToStrings(dataGen.generateUpdatesAsPerSchema("002", 100, schema)).toList @@ -517,12 +520,12 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { updatedStringDf.write .format("org.apache.hudi") - .options(hudiOptions) + .options(writeOpt) .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option("hoodie.datasource.write.reconcile.schema", "true") .mode(SaveMode.Append) .save(tablePath) - spark.read.format("hudi").load(tablePath).registerTempTable("newView") + spark.read.format("hudi").options(readOpt).load(tablePath).registerTempTable("newView") val checkResult = spark.sql(s"select tip_history.amount,city_to_state,distance_in_meters,fare,height from newView where _row_key='$checkRowKey' ") .collect().map(row => (row.isNullAt(0), row.isNullAt(1), row.isNullAt(2), row.isNullAt(3), row.isNullAt(4))) assertResult((false, false, false, true, true))(checkResult(0)) @@ -531,6 +534,18 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } + }) + } + + val sparkOpts = Map( + HoodieWriteConfig.MERGER_IMPLS.key -> classOf[HoodieSparkRecordMerger].getName, + HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> "parquet" + ) + + def getWriterReaderOpts(recordType: HoodieRecordType, opt: Map[String, String]): (Map[String, String], Map[String, String]) = { + recordType match { + case HoodieRecordType.SPARK => (opt ++ sparkOpts, sparkOpts) + case _ => (opt, Map.empty[String, String]) } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala index 30d465f38d3c7..ed67b0caf9add 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala @@ -23,7 +23,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient class TestTimeTravelTable extends HoodieSparkSqlTestBase { test("Test Insert and Update Record with time travel") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName1 = generateTableName spark.sql( s""" @@ -62,13 +62,13 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { s"select id, name, price, ts from $tableName1 TIMESTAMP AS OF '$instant1'")( Seq(1, "a1", 10.0, 1000) ) - } + }) } } test("Test Insert Into Records with time travel To new Table") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => // Create Non-Partitioned table val tableName1 = generateTableName spark.sql( @@ -137,13 +137,13 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 10.0, 1000, "2022-02-14"), Seq(2, "a2", 10.0, 1000, "2022-02-15") ) - } + }) } } test("Test Two Table's Union Join with time travel") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName @@ -235,25 +235,25 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { Seq(4, "a4", 20.0, 1000) ) } - } + }) } } test("Test Unsupported syntax can be parsed") { if (HoodieSparkUtils.gteqSpark3_2) { checkAnswer("select 1 distribute by 1")(Seq(1)) - withTempDir { dir => + withRecordType()(withTempDir { dir => val path = dir.toURI.getPath spark.sql(s"insert overwrite local directory '$path' using parquet select 1") // Requires enable hive support, so didn't test it // spark.sql(s"insert overwrite local directory '$path' stored as orc select 1") - } + }) } } test("Test Select Record with time travel and Repartition") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -291,13 +291,13 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { s"select id, name, price, ts from $tableName TIMESTAMP AS OF '$instant' distribute by cast(rand() * 2 as int)")( Seq(1, "a1", 10.0, 1000) ) - } + }) } } test("Test Time Travel With Schema Evolution") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => spark.sql("set hoodie.schema.on.read.enable=true") val tableName = generateTableName spark.sql( @@ -350,7 +350,7 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 1000, null), Seq(2, "a2", 1100, "hudi") ) - } + }) } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala index 2d8d6ceca7142..8937e8595d389 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala @@ -22,7 +22,7 @@ import org.apache.hudi.HoodieSparkUtils.isSpark2 class TestUpdateTable extends HoodieSparkSqlTestBase { test("Test Update Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach {tableType => val tableName = generateTableName // create table @@ -60,11 +60,11 @@ class TestUpdateTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 40.0, 1000) ) } - } + }) } test("Test Update Table On Non-PK Condition") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach {tableType => /** non-partitioned table */ val tableName = generateTableName @@ -161,11 +161,11 @@ class TestUpdateTable extends HoodieSparkSqlTestBase { Seq(3, "a2", 33.0, 1001, "2022") ) } - } + }) } test("Test ignoring case for Update Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach {tableType => val tableName = generateTableName // create table @@ -202,6 +202,6 @@ class TestUpdateTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 40.0, 1000) ) } - } + }) } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 3cf0a3354e03c..50d7ee35f7126 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -18,30 +18,43 @@ package org.apache.hudi.utilities.deltastreamer; +import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.DataSourceUtils; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.HoodieSparkUtils; +import org.apache.hudi.SparkAdapterSupport$; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.commmon.model.HoodieSparkRecord; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.ConfigUtils; +import org.apache.hudi.common.util.HoodieRecordUtils; +import org.apache.hudi.common.util.IdentityIterator; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; @@ -57,6 +70,7 @@ import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.keygen.SparkKeyGeneratorInterface; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.metrics.HoodieMetrics; @@ -89,8 +103,12 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.avro.HoodieAvroDeserializer; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; import java.io.Closeable; import java.io.IOException; @@ -257,7 +275,7 @@ public void refreshTimeline() throws IOException { if (fs.exists(new Path(cfg.targetBasePath))) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())) .setBasePath(cfg.targetBasePath).setPayloadClassName(cfg.payloadClassName) - .setMergerStrategy(cfg.mergerStrategy) + .setMergerStrategy(props.getProperty(HoodieWriteConfig.MERGER_STRATEGY.key(), HoodieWriteConfig.MERGER_STRATEGY.defaultValue())) .build(); switch (meta.getTableType()) { case COPY_ON_WRITE: @@ -420,6 +438,15 @@ public Pair>> readFromSource( } private Pair>> fetchFromSource(Option resumeCheckpointStr) { + HoodieRecordType recordType = HoodieRecordUtils.createRecordMerger(null, EngineType.SPARK, + ConfigUtils.split2List(props.getProperty(HoodieWriteConfig.MERGER_IMPLS.key(), HoodieWriteConfig.MERGER_IMPLS.defaultValue())), + props.getProperty(HoodieWriteConfig.MERGER_STRATEGY.key(), HoodieWriteConfig.MERGER_STRATEGY.defaultValue())).getRecordType(); + if (recordType == HoodieRecordType.SPARK && HoodieTableType.valueOf(cfg.tableType) == HoodieTableType.MERGE_ON_READ + && HoodieLogBlockType.fromId(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(), "avro")) + != HoodieLogBlockType.PARQUET_DATA_BLOCK) { + throw new UnsupportedOperationException("Spark record only support parquet log."); + } + final Option> avroRDDOptional; final String checkpointStr; SchemaProvider schemaProvider; @@ -492,15 +519,40 @@ private Pair>> fetchFromSourc boolean shouldCombine = cfg.filterDupes || cfg.operation.equals(WriteOperationType.UPSERT); Set partitionColumns = getPartitionColumns(keyGenerator, props); JavaRDD avroRDD = avroRDDOptional.get(); - JavaRDD records = avroRDD.map(record -> { - GenericRecord gr = isDropPartitionColumns() ? HoodieAvroUtils.removeFields(record, partitionColumns) : record; - HoodieRecordPayload payload = shouldCombine ? DataSourceUtils.createPayload(cfg.payloadClassName, gr, - (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false, props.getBoolean( - KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), - Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())))) - : DataSourceUtils.createPayload(cfg.payloadClassName, gr); - return new HoodieAvroRecord<>(keyGenerator.getKey(record), payload); - }); + + JavaRDD records; + SerializableSchema avroSchema = new SerializableSchema(schemaProvider.getTargetSchema()); + SerializableSchema processedAvroSchema = new SerializableSchema(isDropPartitionColumns() ? HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get()); + if (recordType == HoodieRecordType.AVRO) { + records = avroRDD.map(record -> { + GenericRecord gr = isDropPartitionColumns() ? HoodieAvroUtils.removeFields(record, partitionColumns) : record; + HoodieRecordPayload payload = shouldCombine ? DataSourceUtils.createPayload(cfg.payloadClassName, gr, + (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false, props.getBoolean( + KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), + Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())))) + : DataSourceUtils.createPayload(cfg.payloadClassName, gr); + return new HoodieAvroRecord<>(keyGenerator.getKey(record), payload); + }); + } else if (recordType == HoodieRecordType.SPARK) { + // TODO we should remove it if we can read InternalRow from source. + records = avroRDD.mapPartitions(itr -> { + StructType baseStructType = AvroConversionUtils.convertAvroSchemaToStructType(processedAvroSchema.get()); + StructType targetStructType = isDropPartitionColumns() ? AvroConversionUtils + .convertAvroSchemaToStructType(HoodieAvroUtils.removeFields(processedAvroSchema.get(), partitionColumns)) : baseStructType; + HoodieAvroDeserializer deserializer = SparkAdapterSupport$.MODULE$.sparkAdapter().createAvroDeserializer(processedAvroSchema.get(), baseStructType); + + return new MappingIterator<>(new IdentityIterator<>(itr), rec -> { + InternalRow row = (InternalRow) deserializer.deserialize(rec).get(); + SparkKeyGeneratorInterface keyGenerator = (SparkKeyGeneratorInterface) this.keyGenerator; + String recordKey = keyGenerator.getRecordKey(row, baseStructType).toString(); + String partitionPath = keyGenerator.getPartitionPath(row, baseStructType).toString(); + return new HoodieSparkRecord(new HoodieKey(recordKey, partitionPath), + HoodieInternalRowUtils.getCachedUnsafeProjection(baseStructType, targetStructType).apply(row), targetStructType, false); + }); + }); + } else { + throw new UnsupportedOperationException(recordType.name()); + } return Pair.of(schemaProvider, Pair.of(checkpointStr, records)); } @@ -849,7 +901,6 @@ private Schema getSchemaForWriteConfig(Schema targetSchema) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())) .setBasePath(cfg.targetBasePath) .setPayloadClassName(cfg.payloadClassName) - .setMergerStrategy(cfg.mergerStrategy) .build(); int totalCompleted = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants(); if (totalCompleted > 0) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index deeddd96df39c..74cb3e31df079 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -33,8 +33,6 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieAvroRecordMerger; -import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -276,14 +274,6 @@ public static class Config implements Serializable { + "a GenericRecord. Implement your own, if you want to do something other than overwriting existing value") public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName(); - @Parameter(names = {"--merger-impls"}, description = "List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used. " - + "These merger impls will filter by merger-strategy " - + "Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)") - public String mergerImpls = HoodieAvroRecordMerger.class.getName(); - - @Parameter(names = {"--merger-strategy"}, description = "Id of merger strategy. Hudi will pick HoodieRecordMerger implementations in merger-impls which has the same merger strategy id") - public String mergerStrategy = HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; - @Parameter(names = {"--schemaprovider-class"}, description = "subclass of org.apache.hudi.utilities.schema" + ".SchemaProvider to attach schemas to input & target table data, built in options: " + "org.apache.hudi.utilities.schema.FilebasedSchemaProvider." diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index f05a36745b8a7..b6fb2f0f0a538 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -21,16 +21,19 @@ import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.DataSourceReadOptions; import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.HoodieSparkRecordMerger; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; import org.apache.hudi.common.config.DFSPropertiesConfiguration; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; @@ -115,6 +118,8 @@ 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.CsvSource; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; @@ -126,6 +131,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Properties; import java.util.concurrent.CountDownLatch; @@ -163,21 +169,28 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase { private static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamer.class); - protected HoodieDeltaStreamer initialHoodieDeltaStreamer(String tableBasePath, int totalRecords, String asyncCluster) throws IOException { + protected HoodieDeltaStreamer initialHoodieDeltaStreamer(String tableBasePath, int totalRecords, String asyncCluster, HoodieRecordType recordType) throws IOException { HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", asyncCluster, "")); return new HoodieDeltaStreamer(cfg, jsc); } + protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute, HoodieRecordType recordType) { + return initialHoodieClusteringJob(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, null, recordType); + } + protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute) { - return initialHoodieClusteringJob(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, null); + return initialHoodieClusteringJob(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, null, HoodieRecordType.AVRO); } - protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute, Boolean retryLastFailedClusteringJob) { + protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute, + Boolean retryLastFailedClusteringJob, HoodieRecordType recordType) { HoodieClusteringJob.Config scheduleClusteringConfig = buildHoodieClusteringUtilConfig(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, retryLastFailedClusteringJob); + TestHelpers.addRecordMerger(recordType, scheduleClusteringConfig.configs); return new HoodieClusteringJob(jsc, scheduleClusteringConfig); } @@ -285,6 +298,13 @@ static HoodieDeltaStreamer.Config makeConfigForHudiIncrSrc(String srcBasePath, S return cfg; } + static void addRecordMerger(HoodieRecordType type, List hoodieConfig) { + if (type == HoodieRecordType.SPARK) { + hoodieConfig.add(String.format("%s=%s", HoodieWriteConfig.MERGER_IMPLS.key(), HoodieSparkRecordMerger.class.getName())); + hoodieConfig.add(String.format("%s=%s", HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(),"parquet")); + } + } + static void assertRecordCount(long expected, String tablePath, SQLContext sqlContext) { sqlContext.clearCache(); long recordCount = sqlContext.read().format("org.apache.hudi").load(tablePath).count(); @@ -470,14 +490,23 @@ private static HoodieDeltaStreamer.Config getBaseConfig() { */ private static Stream schemaEvolArgs() { return Stream.of( - Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, true), - Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, false), - Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, true), - Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, false), - Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, true), - Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, false), - Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, true), - Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, false)); + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, true, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, false, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, true, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, false, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, true, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, false, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, true, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, false, HoodieRecordType.AVRO), + + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, true, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, false, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, true, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, false, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, true, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, false, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, true, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, false, HoodieRecordType.SPARK)); } private static Stream provideValidCliArgs() { @@ -606,12 +635,14 @@ public void testTableCreation() throws Exception { LOG.debug("Expected error during table creation", e); } - @Test - public void testBulkInsertsAndUpsertsWithBootstrap() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testBulkInsertsAndUpsertsWithBootstrap(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/test_table"; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); syncAndAssertRecordCount(cfg, 1000, tableBasePath, "00000", 1); // No new data => no commits. @@ -653,6 +684,9 @@ public void testBulkInsertsAndUpsertsWithBootstrap() throws Exception { assertEquals(expectedFieldNames.size(), fields.length); assertTrue(fieldNames.containsAll(HoodieRecord.HOODIE_META_COLUMNS)); assertTrue(fieldNames.containsAll(expectedFieldNames)); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, bootstrapSourcePath); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, newDatasetBasePath); } @Test @@ -694,12 +728,13 @@ private void syncAndAssertRecordCount(HoodieDeltaStreamer.Config cfg, Integer ex @ParameterizedTest @MethodSource("schemaEvolArgs") - public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, boolean useSchemaPostProcessor) throws Exception { + public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, boolean useSchemaPostProcessor, HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/test_table_schema_evolution" + tableType + "_" + useUserProvidedSchema + "_" + useSchemaPostProcessor; defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); // Insert data produced with Schema A, pass Schema A HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), PROPS_FILENAME_TEST_SOURCE, false, true, false, null, tableType); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.configs.add("hoodie.deltastreamer.schemaprovider.source.schema.file=" + dfsBasePath + "/source.avsc"); cfg.configs.add("hoodie.deltastreamer.schemaprovider.target.schema.file=" + dfsBasePath + "/source.avsc"); cfg.configs.add(DataSourceWriteOptions.RECONCILE_SCHEMA().key() + "=true"); @@ -713,6 +748,7 @@ public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, // Upsert data produced with Schema B, pass Schema B cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TripsWithEvolvedOptionalFieldTransformer.class.getName()), PROPS_FILENAME_TEST_SOURCE, false, true, false, null, tableType); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.configs.add("hoodie.deltastreamer.schemaprovider.source.schema.file=" + dfsBasePath + "/source.avsc"); cfg.configs.add("hoodie.deltastreamer.schemaprovider.target.schema.file=" + dfsBasePath + "/source_evolved.avsc"); cfg.configs.add(DataSourceWriteOptions.RECONCILE_SCHEMA().key() + "=true"); @@ -737,6 +773,7 @@ public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, } cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), PROPS_FILENAME_TEST_SOURCE, false, true, false, null, tableType); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.configs.add("hoodie.deltastreamer.schemaprovider.source.schema.file=" + dfsBasePath + "/source.avsc"); if (useUserProvidedSchema) { cfg.configs.add("hoodie.deltastreamer.schemaprovider.target.schema.file=" + dfsBasePath + "/source_evolved.avsc"); @@ -764,20 +801,24 @@ public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, assertEquals(tableSchema, expectedSchema); // clean up and reinit + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); UtilitiesTestBase.Helpers.deleteFileFromDfs(FSUtils.getFs(cfg.targetBasePath, jsc.hadoopConfiguration()), dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE); writeCommonPropsToFile(dfs, dfsBasePath); defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); } - @Test - public void testUpsertsCOWContinuousMode() throws Exception { - testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow"); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsCOWContinuousMode(HoodieRecordType recordType) throws Exception { + testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow", recordType); } - @Test - public void testUpsertsCOW_ContinuousModeDisabled() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsCOW_ContinuousModeDisabled(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/non_continuous_cow"; HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.add(String.format("%s=%s", TURN_METRICS_ON.key(), "true")); cfg.configs.add(String.format("%s=%s", METRICS_REPORTER_TYPE_VALUE.key(), "CONSOLE")); @@ -786,22 +827,27 @@ public void testUpsertsCOW_ContinuousModeDisabled() throws Exception { ds.sync(); TestHelpers.assertRecordCount(SQL_SOURCE_NUM_RECORDS, tableBasePath, sqlContext); assertFalse(Metrics.isInitialized(), "Metrics should be shutdown"); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } - @Test - public void testUpsertsCOWContinuousModeShutdownGracefully() throws Exception { - testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow", true); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsCOWContinuousModeShutdownGracefully(HoodieRecordType recordType) throws Exception { + testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow", true, recordType); } - @Test - public void testUpsertsMORContinuousMode() throws Exception { - testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor"); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsMORContinuousMode(HoodieRecordType recordType) throws Exception { + testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor", recordType); } - @Test - public void testUpsertsMOR_ContinuousModeDisabled() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsMOR_ContinuousModeDisabled(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/non_continuous_mor"; HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.tableType = HoodieTableType.MERGE_ON_READ.name(); cfg.configs.add(String.format("%s=%s", TURN_METRICS_ON.key(), "true")); cfg.configs.add(String.format("%s=%s", METRICS_REPORTER_TYPE_VALUE.key(), "CONSOLE")); @@ -810,18 +856,20 @@ public void testUpsertsMOR_ContinuousModeDisabled() throws Exception { ds.sync(); TestHelpers.assertRecordCount(SQL_SOURCE_NUM_RECORDS, tableBasePath, sqlContext); assertFalse(Metrics.isInitialized(), "Metrics should be shutdown"); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } - private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception { - testUpsertsContinuousMode(tableType, tempDir, false); + private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir, HoodieRecordType recordType) throws Exception { + testUpsertsContinuousMode(tableType, tempDir, false, recordType); } - private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir, boolean testShutdownGracefully) throws Exception { + private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir, boolean testShutdownGracefully, HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/" + tempDir; // Keep it higher than batch-size to test continuous mode int totalRecords = 3000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; if (testShutdownGracefully) { cfg.postWriteTerminationStrategyClass = NoNewDataTerminationStrategy.class.getName(); @@ -844,6 +892,7 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir } return true; }); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer.Config cfg, Function condition) throws Exception { @@ -896,14 +945,15 @@ static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, Function getAsyncServicesConfigs(int totalRecords, String autoClean, String inlineCluster, String inlineClusterMaxCommit, @@ -1106,10 +1160,11 @@ private HoodieIndexer.Config buildIndexerConfig(String basePath, return config; } - @Test - public void testHoodieIndexer() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testHoodieIndexer(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/asyncindexer"; - HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 1000, "false"); + HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 1000, "false", recordType); deltaStreamerTestRunner(ds, (r) -> { TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs); @@ -1136,13 +1191,14 @@ public void testHoodieIndexer() throws Exception { } return true; }); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } @ParameterizedTest @ValueSource(booleans = {true, false}) public void testHoodieAsyncClusteringJob(boolean shouldPassInClusteringInstantTime) throws Exception { String tableBasePath = dfsBasePath + "/asyncClusteringJob"; - HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false"); + HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false", HoodieRecordType.AVRO); CountDownLatch countDownLatch = new CountDownLatch(1); deltaStreamerTestRunner(ds, (r) -> { @@ -1178,14 +1234,16 @@ public void testHoodieAsyncClusteringJob(boolean shouldPassInClusteringInstantTi } } - @Test - public void testAsyncClusteringService() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testAsyncClusteringService(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/asyncClustering"; // Keep it higher than batch-size to test continuous mode int totalRecords = 2000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "3")); @@ -1198,6 +1256,7 @@ public void testAsyncClusteringService() throws Exception { TestHelpers.assertAtLeastNCommits(4, tableBasePath, dfs); TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs); TestHelpers.assertDistinctRecordCount(totalRecords, tableBasePath, sqlContext); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } /** @@ -1206,14 +1265,16 @@ public void testAsyncClusteringService() throws Exception { * * @throws Exception */ - @Test - public void testAsyncClusteringServiceWithConflicts() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testAsyncClusteringServiceWithConflicts(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/asyncClusteringWithConflicts"; // Keep it higher than batch-size to test continuous mode int totalRecords = 2000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "3")); @@ -1226,16 +1287,19 @@ public void testAsyncClusteringServiceWithConflicts() throws Exception { TestHelpers.assertAtLeastNCommits(4, tableBasePath, dfs); TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs); TestHelpers.assertDistinctRecordCount(1900, tableBasePath, sqlContext); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } - @Test - public void testAsyncClusteringServiceWithCompaction() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testAsyncClusteringServiceWithCompaction(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/asyncClusteringCompaction"; // Keep it higher than batch-size to test continuous mode int totalRecords = 2000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; cfg.tableType = HoodieTableType.MERGE_ON_READ.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "3")); @@ -1249,16 +1313,18 @@ public void testAsyncClusteringServiceWithCompaction() throws Exception { TestHelpers.assertAtLeastNCommits(4, tableBasePath, dfs); TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs); TestHelpers.assertDistinctRecordCount(totalRecords, tableBasePath, sqlContext); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob) throws Exception { + @CsvSource(value = {"true, AVRO", "true, SPARK", "false, AVRO", "false, SPARK"}) + public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob, HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/asyncClustering3"; // ingest data int totalRecords = 3000; HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = false; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "false", "0", "false", "0")); @@ -1285,7 +1351,7 @@ public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob // trigger a scheduleAndExecute clustering job // when retryFailedClustering true => will rollback and re-execute failed clustering plan with same instant timestamp. // when retryFailedClustering false => will make and execute a new clustering plan with new instant timestamp. - HoodieClusteringJob scheduleAndExecute = initialHoodieClusteringJob(tableBasePath, null, false, "scheduleAndExecute", retryLastFailedClusteringJob); + HoodieClusteringJob scheduleAndExecute = initialHoodieClusteringJob(tableBasePath, null, false, "scheduleAndExecute", retryLastFailedClusteringJob, recordType); scheduleAndExecute.cluster(0); String completeClusteringTimeStamp = meta.getActiveTimeline().reload().getCompletedReplaceTimeline().lastInstant().get().getTimestamp(); @@ -1295,14 +1361,15 @@ public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob } else { assertFalse(clusteringRequest.getTimestamp().equalsIgnoreCase(completeClusteringTimeStamp)); } + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } @ParameterizedTest - @ValueSource(strings = {"execute", "schedule", "scheduleAndExecute"}) - public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMode) throws Exception { + @CsvSource(value = {"execute, AVRO", "schedule, AVRO", "scheduleAndExecute, AVRO", "execute, SPARK", "schedule, SPARK", "scheduleAndExecute, SPARK"}) + public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMode, HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/asyncClustering2"; - HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false"); - HoodieClusteringJob scheduleClusteringJob = initialHoodieClusteringJob(tableBasePath, null, true, runningMode); + HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false", recordType); + HoodieClusteringJob scheduleClusteringJob = initialHoodieClusteringJob(tableBasePath, null, true, runningMode, recordType); deltaStreamerTestRunner(ds, (r) -> { Exception exception = null; @@ -1342,6 +1409,9 @@ public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMod throw new IllegalStateException("Unexpected value: " + runningMode); } }); + if (runningMode.toLowerCase(Locale.ROOT).equals(SCHEDULE_AND_EXECUTE)) { + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); + } } /** @@ -1349,14 +1419,16 @@ public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMod * step involves using a SQL template to transform a source TEST-DATA-SOURCE ============================> HUDI TABLE * 1 ===============> HUDI TABLE 2 (incr-pull with transform) (incr-pull) Hudi Table 1 is synced with Hive. */ - @Test - public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/test_table2"; String downstreamTableBasePath = dfsBasePath + "/test_downstream_table2"; // Initial bulk insert to ingest to first hudi table HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT, Collections.singletonList(SqlQueryBasedTransformer.class.getName()), PROPS_FILENAME_TEST_SOURCE, true); + TestHelpers.addRecordMerger(recordType, cfg.configs); // NOTE: We should not have need to set below config, 'datestr' should have assumed date partitioning cfg.configs.add("hoodie.datasource.hive_sync.partition_fields=year,month,day"); new HoodieDeltaStreamer(cfg, jsc, dfs, hiveServer.getHiveConf()).sync(); @@ -1369,6 +1441,7 @@ public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() t HoodieDeltaStreamer.Config downstreamCfg = TestHelpers.makeConfigForHudiIncrSrc(tableBasePath, downstreamTableBasePath, WriteOperationType.BULK_INSERT, true, null); + TestHelpers.addRecordMerger(recordType, downstreamCfg.configs); new HoodieDeltaStreamer(downstreamCfg, jsc, dfs, hiveServer.getHiveConf()).sync(); TestHelpers.assertRecordCount(1000, downstreamTableBasePath, sqlContext); TestHelpers.assertDistanceCount(1000, downstreamTableBasePath, sqlContext); @@ -1408,6 +1481,7 @@ public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() t downstreamCfg = TestHelpers.makeConfigForHudiIncrSrc(tableBasePath, downstreamTableBasePath, WriteOperationType.UPSERT, false, null); + TestHelpers.addRecordMerger(recordType, downstreamCfg.configs); downstreamCfg.sourceLimit = 2000; new HoodieDeltaStreamer(downstreamCfg, jsc).sync(); TestHelpers.assertRecordCount(2000, downstreamTableBasePath, sqlContext); @@ -1430,6 +1504,8 @@ public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() t assertEquals(lastInstantForUpstreamTable, hiveClient.getLastCommitTimeSynced(tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, downstreamTableBasePath); } @Test @@ -1516,12 +1592,14 @@ public void testPayloadClassUpdateWithCOWTable() throws Exception { assertFalse(props.containsKey(HoodieTableConfig.PAYLOAD_CLASS_NAME.key())); } - @Test - public void testFilterDupes() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testFilterDupes(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/test_dupes_table"; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); new HoodieDeltaStreamer(cfg, jsc).sync(); TestHelpers.assertRecordCount(1000, tableBasePath, sqlContext); TestHelpers.assertCommitMetadata("00000", tableBasePath, dfs, 1); @@ -1542,6 +1620,7 @@ public void testFilterDupes() throws Exception { HoodieTableMetaClient mClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build(); HoodieInstant lastFinished = mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get(); HoodieDeltaStreamer.Config cfg2 = TestHelpers.makeDropAllConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg2.configs); cfg2.filterDupes = false; cfg2.sourceLimit = 2000; cfg2.operation = WriteOperationType.UPSERT; @@ -1567,7 +1646,7 @@ public void testFilterDupes() throws Exception { } catch (IllegalArgumentException e) { assertTrue(e.getMessage().contains("'--filter-dupes' needs to be disabled when '--op' is 'UPSERT' to ensure updates are not missed.")); } - + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } @Test @@ -2137,14 +2216,16 @@ private void insertInTable(String tableBasePath, int count, WriteOperationType o } } - @Test - public void testInsertOverwrite() throws Exception { - testDeltaStreamerWithSpecifiedOperation(dfsBasePath + "/insert_overwrite", WriteOperationType.INSERT_OVERWRITE); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testInsertOverwrite(HoodieRecordType recordType) throws Exception { + testDeltaStreamerWithSpecifiedOperation(dfsBasePath + "/insert_overwrite", WriteOperationType.INSERT_OVERWRITE, recordType); } - @Test - public void testInsertOverwriteTable() throws Exception { - testDeltaStreamerWithSpecifiedOperation(dfsBasePath + "/insert_overwrite_table", WriteOperationType.INSERT_OVERWRITE_TABLE); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testInsertOverwriteTable(HoodieRecordType recordType) throws Exception { + testDeltaStreamerWithSpecifiedOperation(dfsBasePath + "/insert_overwrite_table", WriteOperationType.INSERT_OVERWRITE_TABLE, recordType); } @Disabled("Local run passing; flaky in CI environment.") @@ -2173,9 +2254,10 @@ public void testDeletePartitions() throws Exception { TestHelpers.assertNoPartitionMatch(tableBasePath, sqlContext, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); } - void testDeltaStreamerWithSpecifiedOperation(final String tableBasePath, WriteOperationType operationType) throws Exception { + void testDeltaStreamerWithSpecifiedOperation(final String tableBasePath, WriteOperationType operationType, HoodieRecordType recordType) throws Exception { // Initial insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); new HoodieDeltaStreamer(cfg, jsc).sync(); TestHelpers.assertRecordCount(1000, tableBasePath, sqlContext); TestHelpers.assertDistanceCount(1000, tableBasePath, sqlContext); @@ -2195,6 +2277,7 @@ void testDeltaStreamerWithSpecifiedOperation(final String tableBasePath, WriteOp TestHelpers.assertRecordCount(1950, tableBasePath, sqlContext); TestHelpers.assertDistanceCount(1950, tableBasePath, sqlContext); TestHelpers.assertCommitMetadata("00001", tableBasePath, dfs, 2); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } @Test @@ -2230,11 +2313,13 @@ public void testFetchingCheckpointFromPreviousCommits() throws IOException { .getCommitsTimeline()).get().getMetadata(CHECKPOINT_KEY), "def"); } - @Test - public void testDropPartitionColumns() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testDropPartitionColumns(HoodieRecordType recordType) throws Exception { String tableBasePath = dfsBasePath + "/test_drop_partition_columns" + testNum++; // ingest data with dropping partition columns enabled HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.configs.add(String.format("%s=%s", HoodieTableConfig.DROP_PARTITION_COLUMNS.key(), "true")); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); ds.sync(); @@ -2250,6 +2335,7 @@ public void testDropPartitionColumns() throws Exception { List tableFields = tableSchema.getFields().stream().map(Schema.Field::name).collect(Collectors.toList()); // now assert that the partition column is not in the target schema assertFalse(tableFields.contains("partition_path")); + UtilitiesTestBase.Helpers.deleteFileFromDfs(dfs, tableBasePath); } @Test