diff --git a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala index 9212552d8..33d014520 100644 --- a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala +++ b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala @@ -45,13 +45,14 @@ object SparkHelpers { val schema: Schema = sourceRecords.get(0).getSchema val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_FPP.defaultValue.toDouble, HoodieIndexConfig.HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_INDEX_FILTER_TYPE.defaultValue); - val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(fs.getConf).convert(schema), schema, filter) + val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(fs.getConf).convert(schema), schema, org.apache.hudi.common.util.Option.of(filter)) val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES.defaultValue.toInt, HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue.toDouble) // Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'. parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader) - val writer = new HoodieParquetWriter[HoodieJsonPayload, IndexedRecord](instantTime, destinationFile, parquetConfig, schema, new SparkTaskContextSupplier()) + val writer = new HoodieParquetWriter[HoodieJsonPayload, IndexedRecord](instantTime, destinationFile, parquetConfig, schema, new SparkTaskContextSupplier(), + true) for (rec <- sourceRecords) { val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString if (!keysToSkip.contains(key)) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 5536a7ea7..836b01dff 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1594,6 +1594,11 @@ public class HoodieWriteConfig extends HoodieConfig { return this; } + public Builder withPopulateMetaFields(boolean populateMetaFields) { + writeConfig.setValue(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS, Boolean.toString(populateMetaFields)); + return this; + } + public Builder withProperties(Properties properties) { this.writeConfig.getProps().putAll(properties); return this; 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 1b7f986bd..fe45d4b3f 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 @@ -207,9 +207,11 @@ public class HoodieAppendHandle extends avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get())); String seqId = HoodieRecord.generateSequenceId(instantTime, getPartitionId(), RECORD_COUNTER.getAndIncrement()); - HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(), - hoodieRecord.getPartitionPath(), fileId); - HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId); + if (config.populateMetaFields()) { + HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(), + hoodieRecord.getPartitionPath(), fileId); + HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId); + } if (isUpdateRecord(hoodieRecord)) { updatedRecordsWritten++; } else { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java index 003bfb337..c33931f50 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java @@ -23,12 +23,16 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.fs.Path; +import java.util.ArrayList; +import java.util.List; import java.util.stream.Stream; /** @@ -39,17 +43,25 @@ import java.util.stream.Stream; public class HoodieKeyLocationFetchHandle extends HoodieReadHandle { private final Pair partitionPathBaseFilePair; + private final Option keyGeneratorOpt; public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable hoodieTable, - Pair partitionPathBaseFilePair) { + Pair partitionPathBaseFilePair, Option keyGeneratorOpt) { super(config, null, hoodieTable, Pair.of(partitionPathBaseFilePair.getLeft(), partitionPathBaseFilePair.getRight().getFileId())); this.partitionPathBaseFilePair = partitionPathBaseFilePair; + this.keyGeneratorOpt = keyGeneratorOpt; } public Stream> locations() { HoodieBaseFile baseFile = partitionPathBaseFilePair.getRight(); - return BaseFileUtils.getInstance(baseFile.getPath()).fetchRecordKeyPartitionPath( - hoodieTable.getHadoopConf(), new Path(baseFile.getPath())).stream() + BaseFileUtils baseFileUtils = BaseFileUtils.getInstance(baseFile.getPath()); + List hoodieKeyList = new ArrayList<>(); + if (keyGeneratorOpt.isPresent()) { + hoodieKeyList = baseFileUtils.fetchRecordKeyPartitionPath(hoodieTable.getHadoopConf(), new Path(baseFile.getPath()), keyGeneratorOpt); + } else { + hoodieKeyList = baseFileUtils.fetchRecordKeyPartitionPath(hoodieTable.getHadoopConf(), new Path(baseFile.getPath())); + } + return hoodieKeyList.stream() .map(entry -> Pair.of(entry, new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId()))); } 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 e8f1b5142..84141f33d 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 @@ -32,6 +32,7 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.HoodieRecordSizeEstimator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCorruptedDataException; @@ -40,6 +41,8 @@ import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.io.storage.HoodieFileWriter; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; @@ -101,21 +104,23 @@ public class HoodieMergeHandle extends H protected long updatedRecordsWritten = 0; protected long insertRecordsWritten = 0; protected boolean useWriterSchema; + protected Option keyGeneratorOpt; private HoodieBaseFile baseFileToMerge; public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Iterator> recordItr, String partitionPath, String fileId, - TaskContextSupplier taskContextSupplier) { + TaskContextSupplier taskContextSupplier, Option keyGeneratorOpt) { this(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier, - hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get()); + hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get(), keyGeneratorOpt); } public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Iterator> recordItr, String partitionPath, String fileId, - TaskContextSupplier taskContextSupplier, HoodieBaseFile baseFile) { + TaskContextSupplier taskContextSupplier, HoodieBaseFile baseFile, Option keyGeneratorOpt) { super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); init(fileId, recordItr); init(fileId, partitionPath, baseFile); + validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields()); } /** @@ -123,11 +128,17 @@ public class HoodieMergeHandle extends H */ public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Map> keyToNewRecords, String partitionPath, String fileId, - HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) { + HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier, Option keyGeneratorOpt) { super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); this.keyToNewRecords = keyToNewRecords; this.useWriterSchema = true; init(fileId, this.partitionPath, dataFileToBeMerged); + validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields()); + } + + private void validateAndSetAndKeyGenProps(Option keyGeneratorOpt, boolean populateMetaFields) { + ValidationUtils.checkArgument(populateMetaFields == !keyGeneratorOpt.isPresent()); + this.keyGeneratorOpt = keyGeneratorOpt; } @Override @@ -278,7 +289,7 @@ public class HoodieMergeHandle extends H * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file. */ public void write(GenericRecord oldRecord) { - String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String key = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt); boolean copyOldRecord = true; if (keyToNewRecords.containsKey(key)) { // If we have duplicate records that we are updating, then the hoodie record will be deflated after diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java index a912299f2..763178dbf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java @@ -23,8 +23,10 @@ import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.table.HoodieTable; import org.apache.avro.generic.GenericRecord; @@ -47,8 +49,9 @@ public class HoodieSortedMergeHandle ext private Queue newRecordKeysSorted = new PriorityQueue<>(); public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - Iterator> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) { - super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier); + Iterator> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier, + Option keyGeneratorOpt) { + super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier, keyGeneratorOpt); newRecordKeysSorted.addAll(keyToNewRecords.keySet()); } @@ -57,9 +60,9 @@ public class HoodieSortedMergeHandle ext */ public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Map> keyToNewRecordsOrig, String partitionPath, String fileId, - HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) { + HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier, Option keyGeneratorOpt) { super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged, - taskContextSupplier); + taskContextSupplier, keyGeneratorOpt); newRecordKeysSorted.addAll(keyToNewRecords.keySet()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieConcatHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieConcatHandle.java index 4d2140313..040060886 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieConcatHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieConcatHandle.java @@ -20,11 +20,13 @@ package org.apache.hudi.io.storage; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.HoodieTable; import org.apache.avro.generic.GenericRecord; @@ -66,13 +68,14 @@ public class HoodieConcatHandle extends private static final Logger LOG = LogManager.getLogger(HoodieConcatHandle.class); public HoodieConcatHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Iterator recordItr, - String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) { - super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier); + String partitionPath, String fileId, TaskContextSupplier taskContextSupplier, Option keyGeneratorOpt) { + super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier, keyGeneratorOpt); } public HoodieConcatHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Map keyToNewRecords, String partitionPath, String fileId, HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) { - super(config, instantTime, hoodieTable, keyToNewRecords, partitionPath, fileId, dataFileToBeMerged, taskContextSupplier); + super(config, instantTime, hoodieTable, keyToNewRecords, partitionPath, fileId, dataFileToBeMerged, taskContextSupplier, + Option.empty()); } /** @@ -80,7 +83,7 @@ public class HoodieConcatHandle extends */ @Override public void write(GenericRecord oldRecord) { - String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String key = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt); try { fileWriter.writeAvro(key, oldRecord); } catch (IOException | RuntimeException e) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index 96f19caf0..56022c94e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -34,9 +35,9 @@ import org.apache.parquet.avro.AvroSchemaConverter; import java.io.IOException; +import static org.apache.hudi.common.model.HoodieFileFormat.HFILE; import static org.apache.hudi.common.model.HoodieFileFormat.ORC; import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; -import static org.apache.hudi.common.model.HoodieFileFormat.HFILE; public class HoodieFileWriterFactory { @@ -45,7 +46,7 @@ public class HoodieFileWriterFactory { TaskContextSupplier taskContextSupplier) throws IOException { final String extension = FSUtils.getFileExtension(path.getName()); if (PARQUET.getFileExtension().equals(extension)) { - return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier); + return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier, config.populateMetaFields()); } if (HFILE.getFileExtension().equals(extension)) { return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier); @@ -58,16 +59,21 @@ public class HoodieFileWriterFactory { private static HoodieFileWriter newParquetFileWriter( String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable, - TaskContextSupplier taskContextSupplier) throws IOException { - BloomFilter filter = createBloomFilter(config); - HoodieAvroWriteSupport writeSupport = - new HoodieAvroWriteSupport(new AvroSchemaConverter(hoodieTable.getHadoopConf()).convert(schema), schema, filter); + TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException { + return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier, populateMetaFields, populateMetaFields); + } + + private static HoodieFileWriter newParquetFileWriter( + String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable, + TaskContextSupplier taskContextSupplier, boolean populateMetaFields, boolean enableBloomFilter) throws IOException { + Option filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(hoodieTable.getHadoopConf()).convert(schema), schema, filter); HoodieAvroParquetConfig parquetConfig = new HoodieAvroParquetConfig(writeSupport, config.getParquetCompressionCodec(), config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(), hoodieTable.getHadoopConf(), config.getParquetCompressionRatio()); - return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier); + return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier, populateMetaFields); } private static HoodieFileWriter newHFileFileWriter( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java index b6e77bc8c..e7328fb50 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java @@ -49,9 +49,10 @@ public class HoodieParquetWriter keyGeneratorOpt) { + return keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(genericRecord) : genericRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + } + + /** + * Fetches partition path from the GenericRecord. + * @param genericRecord generic record of interest. + * @param keyGeneratorOpt Optional BaseKeyGenerator. If not, meta field will be used. + * @return the partition path for the passed in generic record. + */ + public static String getPartitionPathFromGenericRecord(GenericRecord genericRecord, Option keyGeneratorOpt) { + return keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(genericRecord) : genericRecord.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + } + /** * Extracts the record key fields in strings out of the given record key, * this is the reverse operation of {@link #getRecordKey(GenericRecord, String)}. diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java index 3bf8e4f9d..353690d4c 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java @@ -34,6 +34,7 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.io.storage.HoodieAvroParquetConfig; import org.apache.hudi.io.storage.HoodieOrcConfig; @@ -68,11 +69,13 @@ public class HoodieWriteableTestTable extends HoodieTestTable { protected final Schema schema; protected final BloomFilter filter; + protected final boolean populateMetaFields; protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { super(basePath, fs, metaClient); this.schema = schema; this.filter = filter; + this.populateMetaFields = metaClient.getTableConfig().populateMetaFields(); } @Override @@ -91,21 +94,25 @@ public class HoodieWriteableTestTable extends HoodieTestTable { if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.PARQUET)) { HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( - new AvroSchemaConverter().convert(schema), schema, filter); + new AvroSchemaConverter().convert(schema), schema, Option.of(filter)); HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, new Configuration(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue())); try (HoodieParquetWriter writer = new HoodieParquetWriter( currentInstantTime, new Path(Paths.get(basePath, partition, fileName).toString()), - config, schema, contextSupplier)) { + config, schema, contextSupplier, populateMetaFields)) { int seqId = 1; for (HoodieRecord record : records) { GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); - HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++)); - HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName); - writer.writeAvro(record.getRecordKey(), avroRecord); - filter.add(record.getRecordKey()); + if (populateMetaFields) { + HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++)); + HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName); + writer.writeAvro(record.getRecordKey(), avroRecord); + filter.add(record.getRecordKey()); + } else { + writer.writeAvro(record.getRecordKey(), avroRecord); + } } } } else if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.ORC)) { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/simple/FlinkHoodieSimpleIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/simple/FlinkHoodieSimpleIndex.java index a8f0aac50..530bdbf16 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/simple/FlinkHoodieSimpleIndex.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/simple/FlinkHoodieSimpleIndex.java @@ -130,7 +130,7 @@ public class FlinkHoodieSimpleIndex extends Flink List> latestBaseFiles) { List>, List, List>> hoodieKeyLocationFetchHandles = - context.map(latestBaseFiles, partitionPathBaseFile -> new HoodieKeyLocationFetchHandle<>(config, hoodieTable, partitionPathBaseFile), parallelism); + context.map(latestBaseFiles, partitionPathBaseFile -> new HoodieKeyLocationFetchHandle<>(config, hoodieTable, partitionPathBaseFile, Option.empty()), parallelism); Map recordLocations = new HashMap<>(); hoodieKeyLocationFetchHandles.stream() .flatMap(handle -> handle.locations()) diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java index 972e7ca85..3c304abf7 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -65,7 +66,7 @@ public class FlinkMergeAndReplaceHandle Iterator> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier, Path basePath) { super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier, - new HoodieBaseFile(basePath.toString())); + new HoodieBaseFile(basePath.toString()), Option.empty()); // delete invalid data files generated by task retry. if (getAttemptId() > 0) { deleteInvalidDataFile(getAttemptId() - 1); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java index faea1b957..55844a44f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -65,7 +66,7 @@ public class FlinkMergeHandle public FlinkMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Iterator> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) { - super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier); + super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier, Option.empty()); if (rolloverPaths == null) { // #makeOldAndNewFilePaths may already initialize it already rolloverPaths = new ArrayList<>(); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index d8bdb9fba..9776870d9 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -351,10 +351,10 @@ public class HoodieFlinkCopyOnWriteTable extends Map> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) { if (requireSortedRecords()) { return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, taskContextSupplier); + dataFileToBeMerged, taskContextSupplier, Option.empty()); } else { return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged,taskContextSupplier); + dataFileToBeMerged,taskContextSupplier, Option.empty()); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java index 765823e77..e12bfafa2 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java @@ -283,9 +283,9 @@ public abstract class BaseJavaCommitActionExecutor> recordItr) { if (table.requireSortedRecords()) { - return new HoodieSortedMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier); + return new HoodieSortedMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier, Option.empty()); } else { - return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier); + return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier, Option.empty()); } } @@ -293,7 +293,7 @@ public abstract class BaseJavaCommitActionExecutor> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) { return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords, - partitionPath, fileId, dataFileToBeMerged, taskContextSupplier); + partitionPath, fileId, dataFileToBeMerged, taskContextSupplier, Option.empty()); } @Override diff --git a/hudi-client/hudi-java-client/src/test/resources/testDataGeneratorSchema.txt b/hudi-client/hudi-java-client/src/test/resources/testDataGeneratorSchema.txt index ada01b353..eccf31d35 100644 --- a/hudi-client/hudi-java-client/src/test/resources/testDataGeneratorSchema.txt +++ b/hudi-client/hudi-java-client/src/test/resources/testDataGeneratorSchema.txt @@ -25,6 +25,9 @@ }, { "name" : "_row_key", "type" : "string" + }, { + "name" : "partition_path", + "type" : "string" }, { "name" : "rider", "type" : "string" diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java index ab6e25867..5add5825c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java @@ -21,6 +21,7 @@ package org.apache.hudi.index.simple; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -30,15 +31,19 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.index.SparkHoodieIndex; import org.apache.hudi.io.HoodieKeyLocationFetchHandle; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.HoodieTable; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import java.io.IOException; import java.util.List; import scala.Tuple2; @@ -146,8 +151,15 @@ public class SparkHoodieSimpleIndex extends Spark List> baseFiles) { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism)); - return jsc.parallelize(baseFiles, fetchParallelism) - .flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile) + + try { + Option keyGeneratorOpt = config.populateMetaFields() ? Option.empty() + : Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps()))); + return jsc.parallelize(baseFiles, fetchParallelism) + .flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile, keyGeneratorOpt) .locations().map(x -> Tuple2.apply(((Pair)x).getLeft(), ((Pair)x).getRight())).iterator()); + } catch (IOException e) { + throw new HoodieIOException("KeyGenerator instantiation failed ", e); + } } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java index 1522fbfcc..0b30f1989 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -97,6 +97,8 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp * @param structType schema of the internalRow. * @return the partition path. */ + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) public String getPartitionPath(InternalRow internalRow, StructType structType) { try { initDeserializer(structType); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java index 77abf1514..2d5d52bec 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java @@ -19,6 +19,8 @@ package org.apache.hudi.keygen; import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; /** * Spark key generator interface. @@ -28,4 +30,6 @@ public interface SparkKeyGeneratorInterface extends KeyGeneratorInterface { String getRecordKey(Row row); String getPartitionPath(Row row); + + String getPartitionPath(InternalRow internalRow, StructType structType); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 98063f6cf..6a2bd6fb2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -27,6 +27,7 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -37,11 +38,14 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.io.HoodieCreateHandle; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.HoodieSortedMergeHandle; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor; @@ -210,12 +214,21 @@ public class HoodieSparkCopyOnWriteTable extends protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId, Map> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) { + Option keyGeneratorOpt = Option.empty(); + if (!config.populateMetaFields()) { + try { + keyGeneratorOpt = Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps()))); + } catch (IOException e) { + throw new HoodieIOException("Only BaseKeyGenerator (or any key generator that extends from BaseKeyGenerator) are supported when meta " + + "columns are disabled. Please choose the right key generator if you wish to disable meta fields.", e); + } + } if (requireSortedRecords()) { return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, taskContextSupplier); + dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); } else { - return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged,taskContextSupplier); + return new HoodieMergeHandle(config, instantTime, this, keyToNewRecords, partitionPath, fileId, + dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java index 5a87ec0c3..415c89f9a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java @@ -50,6 +50,7 @@ import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.io.IOUtils; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; @@ -247,8 +248,8 @@ public class SparkExecuteClusteringCommitActionExecutor transform(IndexedRecord indexedRecord) { GenericRecord record = (GenericRecord) indexedRecord; - String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); - String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + String key = KeyGenUtils.getRecordKeyFromGenericRecord(record, keyGeneratorOpt); + String partition = KeyGenUtils.getPartitionPathFromGenericRecord(record, keyGeneratorOpt); HoodieKey hoodieKey = new HoodieKey(key, partition); HoodieRecordPayload avroPayload = ReflectionUtils.loadPayload(table.getMetaClient().getTableConfig().getPayloadClass(), diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index c8b4fff50..50f3d68a8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -20,6 +20,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.SparkMemoryUtils; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; @@ -37,6 +38,7 @@ import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.execution.SparkLazyInsertIterable; @@ -44,6 +46,8 @@ import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.HoodieSortedMergeHandle; import org.apache.hudi.io.storage.HoodieConcatHandle; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; @@ -78,6 +82,7 @@ public abstract class BaseSparkCommitActionExecutor>, JavaRDD, JavaRDD, HoodieWriteMetadata> { private static final Logger LOG = LogManager.getLogger(BaseSparkCommitActionExecutor.class); + protected Option keyGeneratorOpt = Option.empty(); public BaseSparkCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, @@ -85,6 +90,7 @@ public abstract class BaseSparkCommitActionExecutor> clusteringHandleUpdate(JavaRDD> inputRecordsRDD) { @@ -327,11 +344,12 @@ public abstract class BaseSparkCommitActionExecutor> recordItr) { if (table.requireSortedRecords()) { - return new HoodieSortedMergeHandle<>(config, instantTime, (HoodieSparkTable) table, recordItr, partitionPath, fileId, taskContextSupplier); + return new HoodieSortedMergeHandle<>(config, instantTime, (HoodieSparkTable) table, recordItr, partitionPath, fileId, taskContextSupplier, + keyGeneratorOpt); } else if (!WriteOperationType.isChangingRecords(operationType) && config.allowDuplicateInserts()) { - return new HoodieConcatHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier); + return new HoodieConcatHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier, keyGeneratorOpt); } else { - return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier); + return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier, keyGeneratorOpt); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java index e49723a21..89143db6b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; @@ -66,6 +67,9 @@ import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.KeyGenerator; +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; @@ -120,6 +124,7 @@ import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_EXECUTION import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -136,8 +141,30 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { } }; - private static Stream configParams() { - return Arrays.stream(new Boolean[][] {{true},{false}}).map(Arguments::of); + private static Stream smallInsertHandlingParams() { + return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of); + } + + private static Stream populateMetaFieldsParams() { + return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of); + } + + private static Stream rollbackFailedCommitsParams() { + return Stream.of( + Arguments.of(HoodieFailedWritesCleaningPolicy.LAZY, true), + Arguments.of(HoodieFailedWritesCleaningPolicy.LAZY, false), + Arguments.of(HoodieFailedWritesCleaningPolicy.NEVER, true), + Arguments.of(HoodieFailedWritesCleaningPolicy.NEVER, false) + ); + } + + private static Stream rollbackAfterConsistencyCheckFailureParams() { + return Stream.of( + Arguments.of(true, true), + Arguments.of(true, false), + Arguments.of(false, true), + Arguments.of(false, false) + ); } private HoodieTestTable testTable; @@ -150,50 +177,56 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Test Auto Commit behavior for HoodieWriteClient insert API. */ - @Test - public void testAutoCommitOnInsert() throws Exception { - testAutoCommit(SparkRDDWriteClient::insert, false); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testAutoCommitOnInsert(boolean populateMetaFields) throws Exception { + testAutoCommit(SparkRDDWriteClient::insert, false, populateMetaFields); } /** * Test Auto Commit behavior for HoodieWriteClient insertPrepped API. */ - @Test - public void testAutoCommitOnInsertPrepped() throws Exception { - testAutoCommit(SparkRDDWriteClient::insertPreppedRecords, true); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testAutoCommitOnInsertPrepped(boolean populateMetaFields) throws Exception { + testAutoCommit(SparkRDDWriteClient::insertPreppedRecords, true, populateMetaFields); } /** * Test Auto Commit behavior for HoodieWriteClient upsert API. */ - @Test - public void testAutoCommitOnUpsert() throws Exception { - testAutoCommit(SparkRDDWriteClient::upsert, false); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testAutoCommitOnUpsert(boolean populateMetaFields) throws Exception { + testAutoCommit(SparkRDDWriteClient::upsert, false, populateMetaFields); } /** * Test Auto Commit behavior for HoodieWriteClient upsert Prepped API. */ - @Test - public void testAutoCommitOnUpsertPrepped() throws Exception { - testAutoCommit(SparkRDDWriteClient::upsertPreppedRecords, true); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testAutoCommitOnUpsertPrepped(boolean populateMetaFields) throws Exception { + testAutoCommit(SparkRDDWriteClient::upsertPreppedRecords, true, populateMetaFields); } /** * Test Auto Commit behavior for HoodieWriteClient bulk-insert API. */ - @Test - public void testAutoCommitOnBulkInsert() throws Exception { - testAutoCommit(SparkRDDWriteClient::bulkInsert, false); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testAutoCommitOnBulkInsert(boolean populateMetaFields) throws Exception { + testAutoCommit(SparkRDDWriteClient::bulkInsert, false, populateMetaFields); } /** * Test Auto Commit behavior for HoodieWriteClient bulk-insert prepped API. */ - @Test - public void testAutoCommitOnBulkInsertPrepped() throws Exception { + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testAutoCommitOnBulkInsertPrepped(boolean populateMetaFields) throws Exception { testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.bulkInsertPreppedRecords(recordRDD, instantTime, - Option.empty()), true); + Option.empty()), true, populateMetaFields); } /** @@ -203,15 +236,16 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * @throws Exception in case of failure */ private void testAutoCommit(Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, - boolean isPrepped) throws Exception { + boolean isPrepped, boolean populateMetaFields) throws Exception { // Set autoCommit false - HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withAutoCommit(false); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + try (SparkRDDWriteClient client = getHoodieWriteClient(cfgBuilder.build());) { String prevCommitTime = "000"; String newCommitTime = "001"; int numRecords = 200; - JavaRDD result = insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, writeFn, + JavaRDD result = insertFirstBatch(cfgBuilder.build(), client, newCommitTime, prevCommitTime, numRecords, writeFn, isPrepped, false, numRecords); assertFalse(testTable.commitExists(newCommitTime), @@ -225,25 +259,28 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Test De-duplication behavior for HoodieWriteClient insert API. */ - @Test - public void testDeduplicationOnInsert() throws Exception { - testDeduplication(SparkRDDWriteClient::insert); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testDeduplicationOnInsert(boolean populateMetaFields) throws Exception { + testDeduplication(SparkRDDWriteClient::insert, populateMetaFields); } /** * Test De-duplication behavior for HoodieWriteClient bulk-insert API. */ - @Test - public void testDeduplicationOnBulkInsert() throws Exception { - testDeduplication(SparkRDDWriteClient::bulkInsert); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testDeduplicationOnBulkInsert(boolean populateMetaFields) throws Exception { + testDeduplication(SparkRDDWriteClient::bulkInsert, populateMetaFields); } /** * Test De-duplication behavior for HoodieWriteClient upsert API. */ - @Test - public void testDeduplicationOnUpsert() throws Exception { - testDeduplication(SparkRDDWriteClient::upsert); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testDeduplicationOnUpsert(boolean populateMetaFields) throws Exception { + testDeduplication(SparkRDDWriteClient::upsert, populateMetaFields); } /** @@ -253,7 +290,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * @throws Exception in case of failure */ private void testDeduplication( - Function3, SparkRDDWriteClient, JavaRDD, String> writeFn) throws Exception { + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean populateMetaFields) throws Exception { String newCommitTime = "001"; String recordKey = UUID.randomUUID().toString(); @@ -289,8 +326,11 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { // Perform write-action and check JavaRDD recordList = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1); - try (SparkRDDWriteClient client = getHoodieWriteClient(getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) - .combineInput(true, true).build());) { + HoodieWriteConfig.Builder configBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .combineInput(true, true); + addAppropriatePropsForPopulateMetaFields(configBuilder, populateMetaFields); + + try (SparkRDDWriteClient client = getHoodieWriteClient(configBuilder.build());) { client.startCommitWithTime(newCommitTime); List statuses = writeFn.apply(client, recordList, newCommitTime).collect(); assertNoWriteErrors(statuses); @@ -321,17 +361,23 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Test Upsert API. */ - @Test - public void testUpserts() throws Exception { - testUpsertsInternal(getConfig(), SparkRDDWriteClient::upsert, false); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testUpserts(boolean populateMetaFields) throws Exception { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + testUpsertsInternal(cfgBuilder.build(), SparkRDDWriteClient::upsert, false); } /** * Test UpsertPrepped API. */ - @Test - public void testUpsertsPrepped() throws Exception { - testUpsertsInternal(getConfig(), SparkRDDWriteClient::upsertPreppedRecords, true); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testUpsertsPrepped(boolean populateMetaFields) throws Exception { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + testUpsertsInternal(cfgBuilder.build(), SparkRDDWriteClient::upsertPreppedRecords, true); } /** @@ -348,10 +394,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) .withProps(config.getProps()).withTimelineLayoutVersion( VERSION_0).build(); + HoodieTableMetaClient.withPropertyBuilder() .fromMetaClient(metaClient) .setTimelineLayoutVersion(VERSION_0) - .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); + .setPopulateMetaFields(config.populateMetaFields()) + .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); @@ -360,7 +408,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { String initCommitTime = "000"; int numRecords = 200; insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, SparkRDDWriteClient::insert, - isPrepped, true, numRecords); + isPrepped, true, numRecords, config.populateMetaFields()); // Write 2 (updates) String prevCommitTime = newCommitTime; @@ -369,7 +417,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { String commitTimeBetweenPrevAndNew = "002"; updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime, numRecords, writeFn, isPrepped, true, - numRecords, 200, 2); + numRecords, 200, 2, config.populateMetaFields()); // Delete 1 prevCommitTime = newCommitTime; @@ -378,7 +426,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { deleteBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, initCommitTime, numRecords, SparkRDDWriteClient::delete, isPrepped, true, - 0, 150); + 0, 150, config.populateMetaFields()); // Now simulate an upgrade and perform a restore operation HoodieWriteConfig newConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion( @@ -440,7 +488,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { try { HoodieMergeHandle handle = new HoodieMergeHandle(cfg, instantTime, table, new HashMap<>(), - partitionPath, FSUtils.getFileId(baseFilePath.getName()), baseFile, new SparkTaskContextSupplier()); + partitionPath, FSUtils.getFileId(baseFilePath.getName()), baseFile, new SparkTaskContextSupplier(), + config.populateMetaFields() ? Option.empty() : + Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps())))); WriteStatus writeStatus = new WriteStatus(false, 0.0); writeStatus.setStat(new HoodieWriteStat()); writeStatus.getStat().setNumWrites(0); @@ -454,7 +504,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { cfg.getProps().setProperty("hoodie.merge.data.validation.enabled", "true"); HoodieWriteConfig cfg2 = HoodieWriteConfig.newBuilder().withProps(cfg.getProps()).build(); HoodieMergeHandle handle = new HoodieMergeHandle(cfg2, newInstantTime, table, new HashMap<>(), - partitionPath, FSUtils.getFileId(baseFilePath.getName()), baseFile, new SparkTaskContextSupplier()); + partitionPath, FSUtils.getFileId(baseFilePath.getName()), baseFile, new SparkTaskContextSupplier(), + config.populateMetaFields() ? Option.empty() : + Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps())))); WriteStatus writeStatus = new WriteStatus(false, 0.0); writeStatus.setStat(new HoodieWriteStat()); writeStatus.getStat().setNumWrites(0); @@ -470,17 +522,23 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Test Insert API for HoodieConcatHandle. */ - @Test - public void testInsertsWithHoodieConcatHandle() throws Exception { - testHoodieConcatHandle(getConfig(), false); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testInsertsWithHoodieConcatHandle(boolean populateMetaFields) throws Exception { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + testHoodieConcatHandle(cfgBuilder.build(), false); } /** * Test InsertPrepped API for HoodieConcatHandle. */ - @Test - public void testInsertsPreppedWithHoodieConcatHandle() throws Exception { - testHoodieConcatHandle(getConfig(), true); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testInsertsPreppedWithHoodieConcatHandle(boolean populateMetaFields) throws Exception { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + testHoodieConcatHandle(cfgBuilder.build(), true); } /** @@ -507,7 +565,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { String initCommitTime = "000"; int numRecords = 200; insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, SparkRDDWriteClient::insert, - isPrepped, true, numRecords); + isPrepped, true, numRecords, config.populateMetaFields()); // Write 2 (updates) String prevCommitTime = newCommitTime; @@ -520,15 +578,18 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { writeBatch(client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime, numRecords, recordGenFunction, SparkRDDWriteClient::insert, true, numRecords, 300, - 2, false); + 2, false, config.populateMetaFields()); } /** * Tests deletion of records. */ - @Test - public void testDeletes() throws Exception { - SparkRDDWriteClient client = getHoodieWriteClient(getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).build()); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testDeletes(boolean populateMetaFields) throws Exception { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + SparkRDDWriteClient client = getHoodieWriteClient(cfgBuilder.build()); /** * Write 1 (inserts and deletes) Write actual 200 insert records and ignore 100 delete records */ @@ -547,7 +608,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { }; writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, // unused as genFn uses hard-coded number of inserts/updates/deletes - -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 200, 200, 1, false); + -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 200, 200, 1, false, + populateMetaFields); /** * Write 2 (deletes+writes). @@ -564,7 +626,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { return recordsInSecondBatch; }; writeBatch(client, newCommitTime, prevCommitTime, Option.empty(), initCommitTime, 100, recordGenFunction, - SparkRDDWriteClient::upsert, true, 50, 150, 2, false); + SparkRDDWriteClient::upsert, true, 50, 150, 2, false, + populateMetaFields); } /** @@ -572,9 +635,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * not be available in read path. * @throws Exception */ - @Test - public void testDeletesForInsertsInSameBatch() throws Exception { - SparkRDDWriteClient client = getHoodieWriteClient(getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).build()); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testDeletesForInsertsInSameBatch(boolean populateMetaFields) throws Exception { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + SparkRDDWriteClient client = getHoodieWriteClient(cfgBuilder.build()); /** * Write 200 inserts and issue deletes to a subset(50) of inserts. */ @@ -593,7 +659,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { }; writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, - -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 150, 150, 1, false); + -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 150, 150, 1, false, + populateMetaFields); } /** @@ -793,7 +860,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { Properties props = new Properties(); props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY.key(), "true"); HoodieWriteConfig config = getSmallInsertWriteConfig(100, - TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), props); + TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), true, props); SparkRDDWriteClient client = getHoodieWriteClient(config); HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); @@ -847,7 +914,10 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { final String testPartitionPath = "2016/09/26"; final int insertSplitLimit = 100; // setup the small file handling params - HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max + // hold upto 200 records max + HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, + TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150)); + dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); SparkRDDWriteClient client = getHoodieWriteClient(config); BaseFileUtils fileUtils = BaseFileUtils.getInstance(metaClient); @@ -954,11 +1024,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * Test scenario of new file-group getting added during insert(). */ @ParameterizedTest - @MethodSource("configParams") + @MethodSource("smallInsertHandlingParams") public void testSmallInsertHandlingForInserts(boolean mergeAllowDuplicateInserts) throws Exception { final String testPartitionPath = "2016/09/26"; final int insertSplitLimit = 100; // setup the small file handling params + HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, false, mergeAllowDuplicateInserts); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); SparkRDDWriteClient client = getHoodieWriteClient(config); @@ -1039,7 +1110,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { final String testPartitionPath = "2016/09/26"; final int insertSplitLimit = 100; // setup the small file handling params - HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max + // hold upto 200 records max + HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, + TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150)); dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); SparkRDDWriteClient client = getHoodieWriteClient(config); @@ -1100,31 +1173,34 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { testDeletes(client, updateBatch3.getRight(), 10, file1, "007", 140, keysSoFar); } - @Test - public void testSimpleClustering() throws Exception { + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testSimpleClustering(boolean populateMetaFields) throws Exception { // setup clustering config HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); - testClustering(clusteringConfig); + testClustering(clusteringConfig, populateMetaFields); } - @Test - public void testClusteringWithSortColumns() throws Exception { + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testClusteringWithSortColumns(boolean populateMetaFields) throws Exception { // setup clustering config HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) - .withClusteringSortColumns("_hoodie_record_key") + .withClusteringSortColumns(populateMetaFields ? "_hoodie_record_key" : "_row_key") .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); - testClustering(clusteringConfig); + testClustering(clusteringConfig, populateMetaFields); } - @Test - public void testPendingClusteringRollback() throws Exception { + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testPendingClusteringRollback(boolean populateMetaFields) throws Exception { // setup clustering config HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); - // start clustering, but dont commit - List allRecords = testClustering(clusteringConfig, false); + // start clustering, but don't commit + List allRecords = testClustering(clusteringConfig, populateMetaFields); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); List> pendingClusteringPlans = ClusteringUtils.getAllPendingClusteringPlans(metaClient).collect(Collectors.toList()); @@ -1132,7 +1208,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { HoodieInstant pendingClusteringInstant = pendingClusteringPlans.get(0).getLeft(); // complete another commit after pending clustering - HoodieWriteConfig config = getConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER).build(); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + HoodieWriteConfig config = cfgBuilder.build(); SparkRDDWriteClient client = getHoodieWriteClient(config); dataGen = new HoodieTestDataGenerator(); String commitTime = HoodieActiveTimeline.createNewInstantTime(); @@ -1146,13 +1224,14 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { assertEquals(0, ClusteringUtils.getAllPendingClusteringPlans(metaClient).count()); } - private List testClustering(HoodieClusteringConfig clusteringConfig) throws Exception { - return testClustering(clusteringConfig, false); + private List testClustering(HoodieClusteringConfig clusteringConfig, boolean populateMetaFields) throws Exception { + return testClustering(clusteringConfig, false, populateMetaFields); } - private List testClustering(HoodieClusteringConfig clusteringConfig, boolean completeClustering) throws Exception { + private List testClustering(HoodieClusteringConfig clusteringConfig, boolean completeClustering, boolean populateMetaFields) throws Exception { // create config to not update small files. - HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false, 10); + HoodieWriteConfig config = getSmallInsertWriteConfig(2000, TRIP_EXAMPLE_SCHEMA, 10, false, populateMetaFields, + populateMetaFields ? new Properties() : getPropertiesForKeyGen()); SparkRDDWriteClient client = getHoodieWriteClient(config); dataGen = new HoodieTestDataGenerator(); String commitTime = HoodieActiveTimeline.createNewInstantTime(); @@ -1170,14 +1249,15 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { assertEquals(0, fileIdIntersection.size()); config = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).withAutoCommit(completeClustering) - .withClusteringConfig(clusteringConfig).build(); + .withClusteringConfig(clusteringConfig) + .withProps(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build(); // create client with new config. client = getHoodieWriteClient(config); String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString(); HoodieWriteMetadata> clusterMetadata = client.cluster(clusteringCommitTime, completeClustering); List allRecords = Stream.concat(records1.stream(), records2.stream()).collect(Collectors.toList()); - verifyRecordsWritten(clusteringCommitTime, allRecords, clusterMetadata.getWriteStatuses().collect()); + verifyRecordsWritten(clusteringCommitTime, allRecords, clusterMetadata.getWriteStatuses().collect(), config); Set insertedFileIds = new HashSet<>(); insertedFileIds.addAll(fileIds1); insertedFileIds.addAll(fileIds2); @@ -1197,25 +1277,28 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Test scenario of writing more file groups than existing number of file groups in partition. */ - @Test - public void testInsertOverwritePartitionHandlingWithMoreRecords() throws Exception { - verifyInsertOverwritePartitionHandling(1000, 3000); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testInsertOverwritePartitionHandlingWithMoreRecords(boolean populateMetaFields) throws Exception { + verifyInsertOverwritePartitionHandling(1000, 3000, populateMetaFields); } /** * Test scenario of writing fewer file groups than existing number of file groups in partition. */ - @Test - public void testInsertOverwritePartitionHandlingWithFewerRecords() throws Exception { - verifyInsertOverwritePartitionHandling(3000, 1000); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testInsertOverwritePartitionHandlingWithFewerRecords(boolean populateMetaFields) throws Exception { + verifyInsertOverwritePartitionHandling(3000, 1000, populateMetaFields); } /** * Test scenario of writing similar number file groups in partition. */ - @Test - public void testInsertOverwritePartitionHandlingWithSimilarNumberOfRecords() throws Exception { - verifyInsertOverwritePartitionHandling(3000, 3000); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testInsertOverwritePartitionHandlingWithSimilarNumberOfRecords(boolean populateMetaFields) throws Exception { + verifyInsertOverwritePartitionHandling(3000, 3000, populateMetaFields); } /** @@ -1224,9 +1307,11 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * * Verify that all records in step1 are overwritten */ - private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int batch2RecordsCount) throws Exception { + private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int batch2RecordsCount, boolean populateMetaFields) throws Exception { final String testPartitionPath = "americas"; - HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false); + HoodieWriteConfig config = getSmallInsertWriteConfig(2000, + TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), populateMetaFields, populateMetaFields + ? new Properties() : getPropertiesForKeyGen()); SparkRDDWriteClient client = getHoodieWriteClient(config); dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); @@ -1247,7 +1332,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { assertNoWriteErrors(statuses); assertEquals(batch1Buckets, new HashSet<>(writeResult.getPartitionToReplaceFileIds().get(testPartitionPath))); - verifyRecordsWritten(commitTime2, inserts2, statuses); + verifyRecordsWritten(commitTime2, inserts2, statuses, config); } private Set getFileIdsFromWriteStatus(List statuses) { @@ -1257,35 +1342,38 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Test scenario of writing fewer file groups for first partition than second an third partition. */ - @Test - public void verifyDeletePartitionsHandlingWithFewerRecordsFirstPartition() throws Exception { - verifyDeletePartitionsHandling(1000, 3000, 3000); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void verifyDeletePartitionsHandlingWithFewerRecordsFirstPartition(boolean populateMetaFields) throws Exception { + verifyDeletePartitionsHandling(1000, 3000, 3000, populateMetaFields); } /** * Test scenario of writing similar number file groups in partition. */ - @Test - public void verifyDeletePartitionsHandlingWithSimilarNumberOfRecords() throws Exception { - verifyDeletePartitionsHandling(3000, 3000, 3000); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void verifyDeletePartitionsHandlingWithSimilarNumberOfRecords(boolean populateMetaFields) throws Exception { + verifyDeletePartitionsHandling(3000, 3000, 3000, populateMetaFields); } /** * Test scenario of writing more file groups for first partition than second an third partition. */ - @Test - public void verifyDeletePartitionsHandlingHandlingWithFewerRecordsSecondThirdPartition() throws Exception { - verifyDeletePartitionsHandling(3000, 1000, 1000); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void verifyDeletePartitionsHandlingHandlingWithFewerRecordsSecondThirdPartition(boolean populateMetaFields) throws Exception { + verifyDeletePartitionsHandling(3000, 1000, 1000, populateMetaFields); } - private Set insertPartitionRecordsWithCommit(SparkRDDWriteClient client, int recordsCount, String commitTime1, String partitionPath) { + private Set insertPartitionRecordsWithCommit(SparkRDDWriteClient client, int recordsCount, String commitTime1, String partitionPath) throws IOException { client.startCommitWithTime(commitTime1); List inserts1 = dataGen.generateInsertsForPartition(commitTime1, recordsCount, partitionPath); JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 2); List statuses = client.upsert(insertRecordsRDD1, commitTime1).collect(); assertNoWriteErrors(statuses); Set batchBuckets = statuses.stream().map(s -> s.getFileId()).collect(Collectors.toSet()); - verifyRecordsWritten(commitTime1, inserts1, statuses); + verifyRecordsWritten(commitTime1, inserts1, statuses, client.config); return batchBuckets; } @@ -1306,8 +1394,11 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { * 5) delete second and third partition and check result. * */ - private void verifyDeletePartitionsHandling(int batch1RecordsCount, int batch2RecordsCount, int batch3RecordsCount) throws Exception { - HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false); + private void verifyDeletePartitionsHandling(int batch1RecordsCount, int batch2RecordsCount, int batch3RecordsCount, + boolean populateMetaFields) throws Exception { + HoodieWriteConfig config = getSmallInsertWriteConfig(2000, + TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), populateMetaFields, populateMetaFields + ? new Properties() : getPropertiesForKeyGen()); SparkRDDWriteClient client = getHoodieWriteClient(config); dataGen = new HoodieTestDataGenerator(); @@ -1360,7 +1451,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Verify data in base files matches expected records and commit time. */ - private void verifyRecordsWritten(String commitTime, List expectedRecords, List allStatus) { + private void verifyRecordsWritten(String commitTime, List expectedRecords, List allStatus, + HoodieWriteConfig config) throws IOException { List records = new ArrayList<>(); for (WriteStatus status : allStatus) { Path filePath = new Path(basePath, status.getStat().getPath()); @@ -1369,20 +1461,29 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { Set expectedKeys = recordsToRecordKeySet(expectedRecords); assertEquals(records.size(), expectedKeys.size()); - for (GenericRecord record : records) { - String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); - assertEquals(commitTime, - record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); - assertTrue(expectedKeys.contains(recordKey)); + if (config.populateMetaFields()) { + for (GenericRecord record : records) { + String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + assertEquals(commitTime, + record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); + assertTrue(expectedKeys.contains(recordKey)); + } + } else { + KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps())); + for (GenericRecord record : records) { + String recordKey = keyGenerator.getKey(record).getRecordKey(); + assertNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)); + assertTrue(expectedKeys.contains(recordKey)); + } } } - private List writeAndVerifyBatch(SparkRDDWriteClient client, List inserts, String commitTime) { + private List writeAndVerifyBatch(SparkRDDWriteClient client, List inserts, String commitTime) throws IOException { client.startCommitWithTime(commitTime); JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts, 2); List statuses = client.upsert(insertRecordsRDD1, commitTime).collect(); assertNoWriteErrors(statuses); - verifyRecordsWritten(commitTime, inserts, statuses); + verifyRecordsWritten(commitTime, inserts, statuses, client.config); return statuses; } @@ -1449,12 +1550,15 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Test delete with delete api. */ - @Test - public void testDeletesWithoutInserts() { + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testDeletesWithoutInserts(boolean populateMetaFields) { final String testPartitionPath = "2016/09/26"; final int insertSplitLimit = 100; // setup the small file handling params - HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, true); // hold upto 200 records max + HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, + TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), populateMetaFields, populateMetaFields + ? new Properties() : getPropertiesForKeyGen()); dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); SparkRDDWriteClient client = getHoodieWriteClient(config); @@ -1473,13 +1577,15 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Test to ensure commit metadata points to valid files. */ - @Test - public void testCommitWritesRelativePaths() throws Exception { + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testCommitWritesRelativePaths(boolean populateMetaFields) throws Exception { - HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withAutoCommit(false); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + try (SparkRDDWriteClient client = getHoodieWriteClient(cfgBuilder.build());) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); - HoodieSparkTable table = HoodieSparkTable.create(cfg, context, metaClient); + HoodieSparkTable table = HoodieSparkTable.create(cfgBuilder.build(), context, metaClient); String instantTime = "000"; client.startCommitWithTime(instantTime); @@ -1518,9 +1624,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { /** * Test to ensure commit metadata points to valid files.10. */ - @Test - public void testMetadataStatsOnCommit() throws Exception { - HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testMetadataStatsOnCommit(boolean populateMetaFields) throws Exception { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withAutoCommit(false); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + HoodieWriteConfig cfg = cfgBuilder.build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); String instantTime0 = "000"; @@ -1607,16 +1716,24 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { } } - private void testRollbackAfterConsistencyCheckFailureUsingFileList(boolean rollbackUsingMarkers, boolean enableOptimisticConsistencyGuard) throws Exception { + private void testRollbackAfterConsistencyCheckFailureUsingFileList(boolean rollbackUsingMarkers, boolean enableOptimisticConsistencyGuard, + boolean populateMetaFields) throws Exception { String instantTime = "000"; HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); + + Properties properties = new Properties(); + if (!populateMetaFields) { + properties = getPropertiesForKeyGen(); + } + HoodieWriteConfig cfg = !enableOptimisticConsistencyGuard ? getConfigBuilder().withRollbackUsingMarkers(rollbackUsingMarkers).withAutoCommit(false) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true) .withMaxConsistencyCheckIntervalMs(1).withInitialConsistencyCheckIntervalMs(1).withEnableOptimisticConsistencyGuard(enableOptimisticConsistencyGuard).build()).build() : getConfigBuilder().withRollbackUsingMarkers(rollbackUsingMarkers).withAutoCommit(false) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder() .withConsistencyCheckEnabled(true) - .withOptimisticConsistencyGuardSleepTimeMs(1).build()).build(); + .withOptimisticConsistencyGuardSleepTimeMs(1).build()) + .withProperties(properties).build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); testConsistencyCheck(metaClient, instantTime, enableOptimisticConsistencyGuard); @@ -1651,28 +1768,28 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testRollbackAfterConsistencyCheckFailureUsingFileList(boolean enableOptimisticConsistencyGuard) throws Exception { - testRollbackAfterConsistencyCheckFailureUsingFileList(false, enableOptimisticConsistencyGuard); + @MethodSource("rollbackAfterConsistencyCheckFailureParams") + public void testRollbackAfterConsistencyCheckFailureUsingFileList(boolean enableOptimisticConsistencyGuard, boolean populateMetCols) throws Exception { + testRollbackAfterConsistencyCheckFailureUsingFileList(false, enableOptimisticConsistencyGuard, populateMetCols); } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testRollbackAfterConsistencyCheckFailureUsingMarkers(boolean enableOptimisticConsistencyGuard) throws Exception { - testRollbackAfterConsistencyCheckFailureUsingFileList(true, enableOptimisticConsistencyGuard); + @MethodSource("rollbackAfterConsistencyCheckFailureParams") + public void testRollbackAfterConsistencyCheckFailureUsingMarkers(boolean enableOptimisticConsistencyGuard, boolean populateMetCols) throws Exception { + testRollbackAfterConsistencyCheckFailureUsingFileList(true, enableOptimisticConsistencyGuard, populateMetCols); } @ParameterizedTest - @EnumSource(value = HoodieFailedWritesCleaningPolicy.class, names = {"LAZY", "NEVER"}) - public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningPolicy) throws Exception { + @MethodSource("rollbackFailedCommitsParams") + public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningPolicy, boolean populateMetaFields) throws Exception { HoodieTestUtils.init(hadoopConf, basePath); // Perform 2 failed writes to table - SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); @@ -1680,7 +1797,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { // refresh data generator to delete records generated from failed commits dataGen = new HoodieTestDataGenerator(); // Perform 1 successful write - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, true); @@ -1696,7 +1813,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200"); Thread.sleep(2000); } - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); // Perform 1 successful write writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, @@ -1732,11 +1849,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { } } - @Test - public void testRollbackFailedCommitsToggleCleaningPolicy() throws Exception { + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testRollbackFailedCommitsToggleCleaningPolicy(boolean populateMetaFields) throws Exception { HoodieTestUtils.init(hadoopConf, basePath); HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.EAGER; - SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); // Perform 1 failed writes to table writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, @@ -1745,12 +1863,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { // Toggle cleaning policy to LAZY cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY; // Perform 2 failed writes to table - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); @@ -1766,19 +1884,19 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { assertTrue(timeline.getTimelineOfActions( CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 3); // Perform 2 failed commits - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); writeBatch(client, "500", "400", Option.of(Arrays.asList("500")), "500", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); // Toggle cleaning policy to EAGER cleaningPolicy = HoodieFailedWritesCleaningPolicy.EAGER; - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); client.startCommit(); timeline = metaClient.getActiveTimeline().reload(); assertTrue(timeline.getTimelineOfActions( @@ -1786,18 +1904,19 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 0); } - @Test - public void testParallelInsertAndCleanPreviousFailedCommits() throws Exception { + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testParallelInsertAndCleanPreviousFailedCommits(boolean populateMetaFields) throws Exception { HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY; ExecutorService service = Executors.newFixedThreadPool(2); HoodieTestUtils.init(hadoopConf, basePath); // Perform 2 failed writes to table - SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, false); client.close(); - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); writeBatch(client, "200", "200", Option.of(Arrays.asList("200")), "200", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, false); @@ -1805,7 +1924,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { // refresh data generator to delete records generated from failed commits dataGen = new HoodieTestDataGenerator(); // Create a succesful commit - Future> commit3 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)), + Future> commit3 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)), "300", "200", Option.of(Arrays.asList("300")), "200", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true)); commit3.get(); @@ -1815,17 +1934,17 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 0); assertTrue(metaClient.getActiveTimeline().filterInflights().countInstants() == 2); assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 1); - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); // Await till enough time passes such that the first 2 failed commits heartbeats are expired boolean conditionMet = false; while (!conditionMet) { conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200"); Thread.sleep(2000); } - Future> commit4 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)), + Future> commit4 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)), "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true)); - Future clean1 = service.submit(() -> new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)).clean()); + Future clean1 = service.submit(() -> new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)).clean()); commit4.get(); clean1.get(); HoodieActiveTimeline timeline = metaClient.getActiveTimeline().reload(); @@ -1878,11 +1997,13 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { return Pair.of(markerFilePath, result); } - @Test - public void testMultiOperationsPerCommit() throws IOException { - HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false) - .withAllowMultiWriteOnSameInstant(true) - .build(); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testMultiOperationsPerCommit(boolean populateMetaFields) throws IOException { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withAutoCommit(false) + .withAllowMultiWriteOnSameInstant(true); + addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields); + HoodieWriteConfig cfg = cfgBuilder.build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); String firstInstantTime = "0000"; client.startCommitWithTime(firstInstantTime); @@ -1957,16 +2078,19 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { } private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, boolean mergeAllowDuplicateInserts) { - return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, mergeAllowDuplicateInserts, new Properties()); + return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, mergeAllowDuplicateInserts, true, new Properties()); } - private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, Properties props) { - return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, false, props); + private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, boolean populateMetaFields, Properties props) { + return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, false, populateMetaFields, props); } private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, boolean mergeAllowDuplicateInserts, - Properties props) { + boolean populateMetaFields, Properties props) { HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr); + if (!populateMetaFields) { + builder.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.SIMPLE).build()); + } return builder .withCompactionConfig( HoodieCompactionConfig.newBuilder() @@ -1994,7 +2118,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { return clusteringInstant; } - private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleaningPolicy cleaningPolicy) { + private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleaningPolicy cleaningPolicy, boolean populateMetaFields) { return getConfigBuilder() .withEmbeddedTimelineServerEnabled(false) .withCompactionConfig(HoodieCompactionConfig.newBuilder() @@ -2002,7 +2126,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { .withAutoClean(false).build()) .withTimelineLayoutVersion(1) .withHeartbeatIntervalInMs(3 * 1000) - .withAutoCommit(false).build(); + .withAutoCommit(false) + .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build(); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java index 921bc3403..96782f494 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.io.HoodieCreateHandle; @@ -121,7 +122,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness { jsc.parallelize(Arrays.asList(1)).map(x -> { Executable executable = () -> { HoodieMergeHandle mergeHandle = new HoodieMergeHandle(updateTable.getConfig(), "101", updateTable, - updateRecords.iterator(), updateRecords.get(0).getPartitionPath(), insertResult.getFileId(), supplier); + updateRecords.iterator(), updateRecords.get(0).getPartitionPath(), insertResult.getFileId(), supplier, Option.empty()); List oldRecords = BaseFileUtils.getInstance(updateTable.getBaseFileFormat()) .readAvroRecords(updateTable.getHadoopConf(), new Path(updateTable.getConfig().getBasePath() + "/" + insertResult.getStat().getPath()), diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java index a18313b27..d792f088f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java @@ -23,10 +23,12 @@ import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -47,8 +49,10 @@ import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; import java.util.Arrays; @@ -56,8 +60,10 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Random; import java.util.UUID; +import java.util.stream.Stream; import scala.Tuple2; @@ -69,16 +75,34 @@ import static org.junit.jupiter.api.Assertions.fail; public class TestHoodieIndex extends HoodieClientTestHarness { + private static Stream indexTypeParams() { + Object[][] data = new Object[][] { + {IndexType.BLOOM, true}, + {IndexType.GLOBAL_BLOOM, true}, + {IndexType.SIMPLE, true}, + {IndexType.GLOBAL_SIMPLE, true}, + {IndexType.SIMPLE, false}, + {IndexType.GLOBAL_SIMPLE, false} + }; + return Stream.of(data).map(Arguments::of); + } + private static final Schema SCHEMA = getSchemaFromResource(TestHoodieIndex.class, "/exampleSchema.avsc", true); private final Random random = new Random(); private IndexType indexType; private HoodieIndex index; private HoodieWriteConfig config; - private void setUp(IndexType indexType) throws Exception { + private void setUp(IndexType indexType, boolean populateMetaFields) throws Exception { this.indexType = indexType; - initResources(); + initPath(); + initSparkContexts(); + initTestDataGenerator(); + initFileSystem(); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE, populateMetaFields ? new Properties() + : getPropertiesForKeyGen()); config = getConfigBuilder() + .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType) .build()).withAutoCommit(false).build(); writeClient = getHoodieWriteClient(config); @@ -91,9 +115,9 @@ public class TestHoodieIndex extends HoodieClientTestHarness { } @ParameterizedTest - @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE"}) - public void testSimpleTagLocationAndUpdate(IndexType indexType) throws Exception { - setUp(indexType); + @MethodSource("indexTypeParams") + public void testSimpleTagLocationAndUpdate(IndexType indexType, boolean populateMetaFields) throws Exception { + setUp(indexType, populateMetaFields); String newCommitTime = "001"; int totalRecords = 10 + random.nextInt(20); List records = dataGen.generateInserts(newCommitTime, totalRecords); @@ -141,9 +165,9 @@ public class TestHoodieIndex extends HoodieClientTestHarness { } @ParameterizedTest - @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE"}) - public void testTagLocationAndDuplicateUpdate(IndexType indexType) throws Exception { - setUp(indexType); + @MethodSource("indexTypeParams") + public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean populateMetaFields) throws Exception { + setUp(indexType, populateMetaFields); String newCommitTime = "001"; int totalRecords = 10 + random.nextInt(20); List records = dataGen.generateInserts(newCommitTime, totalRecords); @@ -191,9 +215,9 @@ public class TestHoodieIndex extends HoodieClientTestHarness { } @ParameterizedTest - @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE"}) - public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType) throws Exception { - setUp(indexType); + @MethodSource("indexTypeParams") + public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields) throws Exception { + setUp(indexType, populateMetaFields); String newCommitTime = writeClient.startCommit(); int totalRecords = 20 + random.nextInt(20); List records = dataGen.generateInserts(newCommitTime, totalRecords); @@ -242,10 +266,18 @@ public class TestHoodieIndex extends HoodieClientTestHarness { assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); } + private static Stream regularIndexTypeParams() { + Object[][] data = new Object[][] { + {IndexType.BLOOM, true}, + {IndexType.SIMPLE, true} + }; + return Stream.of(data).map(Arguments::of); + } + @ParameterizedTest - @EnumSource(value = IndexType.class, names = {"BLOOM", "SIMPLE",}) - public void testTagLocationAndFetchRecordLocations(IndexType indexType) throws Exception { - setUp(indexType); + @MethodSource("regularIndexTypeParams") + public void testTagLocationAndFetchRecordLocations(IndexType indexType, boolean populateMetaFields) throws Exception { + setUp(indexType, populateMetaFields); String p1 = "2016/01/31"; String p2 = "2015/01/31"; String rowKey1 = UUID.randomUUID().toString(); @@ -325,10 +357,9 @@ public class TestHoodieIndex extends HoodieClientTestHarness { } } - @ParameterizedTest - @EnumSource(value = IndexType.class, names = {"GLOBAL_SIMPLE"}) - public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath(IndexType indexType) throws Exception { - setUp(indexType); + @Test + public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath() throws Exception { + setUp(IndexType.GLOBAL_SIMPLE, true); config = getConfigBuilder() .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType) .withGlobalSimpleIndexUpdatePartitionPath(true) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java index ff012b049..e02e61364 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java @@ -18,20 +18,26 @@ package org.apache.hudi.io; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; @@ -40,7 +46,8 @@ import org.apache.hudi.testutils.MetadataMergeWriteStatus; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.util.ArrayList; @@ -48,6 +55,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import scala.Tuple2; @@ -71,10 +79,6 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness { initPath(); initTestDataGenerator(); initFileSystem(); - initMetaClient(); - config = getConfigBuilder() - .withIndexConfig(HoodieIndexConfig.newBuilder() - .build()).build(); } @AfterEach @@ -82,8 +86,15 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness { cleanupResources(); } - @Test - public void testFetchHandle() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testFetchHandle(boolean populateMetaFields) throws Exception { + metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE, populateMetaFields ? new Properties() : getPropertiesForKeyGen()); + config = getConfigBuilder() + .withProperties(getPropertiesForKeyGen()) + .withIndexConfig(HoodieIndexConfig.newBuilder() + .build()).build(); + List records = dataGen.generateInserts(makeNewCommitTime(), 100); Map> partitionRecordsMap = recordsToPartitionRecordsMap(records); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); @@ -93,8 +104,11 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness { List> partitionPathFileIdPairs = loadAllFilesForPartitions(new ArrayList<>(partitionRecordsMap.keySet()), context, hoodieTable); + BaseKeyGenerator keyGenerator = (BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(getPropertiesForKeyGen())); + for (Tuple2 entry : partitionPathFileIdPairs) { - HoodieKeyLocationFetchHandle fetcherHandle = new HoodieKeyLocationFetchHandle(config, hoodieTable, Pair.of(entry._1, entry._2)); + HoodieKeyLocationFetchHandle fetcherHandle = new HoodieKeyLocationFetchHandle(config, hoodieTable, Pair.of(entry._1, entry._2), + populateMetaFields ? Option.empty() : Option.of(keyGenerator)); Iterator> result = fetcherHandle.locations().iterator(); List> actualList = new ArrayList<>(); result.forEachRemaining(x -> actualList.add(new Tuple2<>(x.getLeft(), x.getRight()))); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java index fd9b1ddb1..da57fa85e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java @@ -43,6 +43,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieCompactionConfig; @@ -60,10 +61,10 @@ import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; import java.nio.file.Files; @@ -71,7 +72,9 @@ import java.nio.file.Paths; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Properties; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -91,16 +94,15 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { private HoodieTableType tableType; - public void init(HoodieTableType tableType) throws IOException { + public void init(HoodieTableType tableType, boolean populateMetaFields) throws IOException { this.tableType = tableType; initPath(); initSparkContexts("TestHoodieMetadata"); initFileSystem(); fs.mkdirs(new Path(basePath)); - initMetaClient(tableType); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType, populateMetaFields ? new Properties() : getPropertiesForKeyGen()); initTestDataGenerator(); metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); - } @AfterEach @@ -108,12 +110,25 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { cleanupResources(); } + private static Stream populateMetaFieldsParams() { + return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of); + } + + private static Stream tableTypePopulateMetaFieldsParams() { + return Stream.of( + Arguments.of(HoodieTableType.COPY_ON_WRITE, true), + Arguments.of(HoodieTableType.COPY_ON_WRITE, false), + Arguments.of(HoodieTableType.MERGE_ON_READ, true) + ); + } + /** * Metadata Table bootstrap scenarios. */ - @Test - public void testMetadataTableBootstrap() throws Exception { - init(HoodieTableType.COPY_ON_WRITE); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testMetadataTableBootstrap(boolean populateMetaFields) throws Exception { + init(HoodieTableType.COPY_ON_WRITE, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); // Metadata table should not exist until created for the first time @@ -122,7 +137,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { // Metadata table is not created if disabled by config String firstCommitTime = HoodieActiveTimeline.createNewInstantTime(); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) { client.startCommitWithTime(firstCommitTime); client.insert(jsc.parallelize(dataGen.generateInserts(firstCommitTime, 5)), firstCommitTime); assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not be created"); @@ -131,7 +146,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { // Metadata table should not be created if any non-complete instants are present String secondCommitTime = HoodieActiveTimeline.createNewInstantTime(); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(false, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(false, true, populateMetaFields), true)) { client.startCommitWithTime(secondCommitTime); client.insert(jsc.parallelize(dataGen.generateUpdates(secondCommitTime, 2)), secondCommitTime); // AutoCommit is false so no bootstrap @@ -144,7 +159,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { // Metadata table created when enabled by config & sync is called secondCommitTime = HoodieActiveTimeline.createNewInstantTime(); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields), true)) { client.startCommitWithTime(secondCommitTime); client.insert(jsc.parallelize(dataGen.generateUpdates(secondCommitTime, 2)), secondCommitTime); client.syncTableMetadata(); @@ -167,7 +182,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { }); String thirdCommitTime = HoodieActiveTimeline.createNewInstantTime(); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields), true)) { client.startCommitWithTime(thirdCommitTime); client.insert(jsc.parallelize(dataGen.generateUpdates(thirdCommitTime, 2)), thirdCommitTime); client.syncTableMetadata(); @@ -184,10 +199,11 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { /** * Only valid partition directories are added to the metadata. */ - @Test - public void testOnlyValidPartitionsAdded() throws Exception { + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testOnlyValidPartitionsAdded(boolean populateMetaFields) throws Exception { // This test requires local file system - init(HoodieTableType.COPY_ON_WRITE); + init(HoodieTableType.COPY_ON_WRITE, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); // Create an empty directory which is not a partition directory (lacks partition metadata) @@ -207,7 +223,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { .addCommit("002").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10, 10); final HoodieWriteConfig writeConfig = - getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.NEVER, true, true, false) + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.NEVER, true, true, false, populateMetaFields) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).withDirectoryFilterRegex(filterDirRegex).build()).build(); try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { client.startCommitWithTime("005"); @@ -237,12 +253,12 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { * Test various table operations sync to Metadata Table correctly. */ @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testTableOperations(HoodieTableType tableType) throws Exception { - init(tableType); + @MethodSource("tableTypePopulateMetaFieldsParams") + public void testTableOperations(HoodieTableType tableType, boolean populateMetaFields) throws Exception { + init(tableType, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) { // Write 1 (Bulk insert) String newCommitTime = "001"; @@ -325,12 +341,12 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { * Test rollback of various table operations sync to Metadata Table correctly. */ @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testRollbackOperations(HoodieTableType tableType) throws Exception { - init(tableType); + @MethodSource("tableTypePopulateMetaFieldsParams") + public void testRollbackOperations(HoodieTableType tableType, boolean populateMetaFields) throws Exception { + init(tableType, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) { // Write 1 (Bulk insert) String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); List records = dataGen.generateInserts(newCommitTime, 20); @@ -403,7 +419,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { // Rollback of partial commits try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, - getWriteConfigBuilder(false, true, false).withRollbackUsingMarkers(false).build())) { + getWriteConfigBuilder(false, true, false, populateMetaFields).withRollbackUsingMarkers(false).build())) { // Write updates and inserts String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.startCommitWithTime(newCommitTime); @@ -417,7 +433,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { // Marker based rollback of partial commits try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, - getWriteConfigBuilder(false, true, false).withRollbackUsingMarkers(true).build())) { + getWriteConfigBuilder(false, true, false, populateMetaFields).withRollbackUsingMarkers(true).build())) { // Write updates and inserts String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.startCommitWithTime(newCommitTime); @@ -435,12 +451,12 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { * Once explicit sync is called, metadata should match. */ @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testRollbackUnsyncedCommit(HoodieTableType tableType) throws Exception { - init(tableType); + @MethodSource("tableTypePopulateMetaFieldsParams") + public void testRollbackUnsyncedCommit(HoodieTableType tableType, boolean populateMetaFields) throws Exception { + init(tableType, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) { // Initialize table with metadata String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); List records = dataGen.generateInserts(newCommitTime, 20); @@ -450,7 +466,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { validateMetadata(client); } String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) { // Commit with metadata disabled client.startCommitWithTime(newCommitTime); List records = dataGen.generateUpdates(newCommitTime, 10); @@ -459,7 +475,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { client.rollback(newCommitTime); } - try (SparkRDDWriteClient client = new SparkRDDWriteClient<>(engineContext, getWriteConfig(true, true))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient<>(engineContext, getWriteConfig(true, true, populateMetaFields))) { assertFalse(metadata(client).isInSync()); client.syncTableMetadata(); validateMetadata(client); @@ -470,10 +486,10 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { * Test sync of table operations. */ @ParameterizedTest - @EnumSource(HoodieTableType.class) + @MethodSource("tableTypePopulateMetaFieldsParams") @Disabled - public void testSync(HoodieTableType tableType) throws Exception { - init(tableType); + public void testSync(HoodieTableType tableType, boolean populateMetaFields) throws Exception { + init(tableType, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); String newCommitTime; @@ -481,7 +497,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { List writeStatuses; // Initial commits without metadata table enabled - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) { newCommitTime = HoodieActiveTimeline.createNewInstantTime(); records = dataGen.generateInserts(newCommitTime, 5); client.startCommitWithTime(newCommitTime); @@ -496,7 +512,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { } // Enable metadata table so it initialized by listing from file system - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) { // inserts newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.startCommitWithTime(newCommitTime); @@ -512,7 +528,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { String restoreToInstant; String inflightActionTimestamp; String beforeInflightActionTimestamp; - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) { // updates newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.startCommitWithTime(newCommitTime); @@ -584,7 +600,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { Path inflightCleanPath = new Path(metaClient.getMetaPath(), HoodieTimeline.makeInflightCleanerFileName(inflightActionTimestamp)); fs.create(inflightCleanPath).close(); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) { // Restore cannot be done until the metadata table is in sync. See HUDI-1502 for details client.syncTableMetadata(); @@ -613,7 +629,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { } // Enable metadata table and ensure it is synced - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) { client.restoreToInstant(restoreToInstant); assertFalse(metadata(client).isInSync()); @@ -629,13 +645,14 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { /** * Instants on Metadata Table should be archived as per config. Metadata Table should be automatically compacted as per config. */ - @Test - public void testCleaningArchivingAndCompaction() throws Exception { - init(HoodieTableType.COPY_ON_WRITE); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testCleaningArchivingAndCompaction(boolean populateMetaFields) throws Exception { + init(HoodieTableType.COPY_ON_WRITE, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); final int maxDeltaCommitsBeforeCompaction = 4; - HoodieWriteConfig config = getWriteConfigBuilder(true, true, false) + HoodieWriteConfig config = getWriteConfigBuilder(true, true, false, populateMetaFields) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) .archiveCommitsWith(6, 8).retainCommits(1) .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) @@ -676,14 +693,15 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { /** * Test various error scenarios. */ - @Test - public void testErrorCases() throws Exception { - init(HoodieTableType.COPY_ON_WRITE); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testErrorCases(boolean populateMetaFields) throws Exception { + init(HoodieTableType.COPY_ON_WRITE, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); // TESTCASE: If commit on the metadata table succeeds but fails on the dataset, then on next init the metadata table // should be rolled back to last valid commit. - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields), true)) { String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 10); @@ -704,7 +722,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { commitInstantFileName), false)); } - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields), true)) { String newCommitTime = client.startCommit(); // Next insert List records = dataGen.generateInserts(newCommitTime, 5); @@ -721,11 +739,11 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { */ //@Test public void testNonPartitioned() throws Exception { - init(HoodieTableType.COPY_ON_WRITE); + init(HoodieTableType.COPY_ON_WRITE, true); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); HoodieTestDataGenerator nonPartitionedGenerator = new HoodieTestDataGenerator(new String[] {""}); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, true))) { // Write 1 (Bulk insert) String newCommitTime = "001"; List records = nonPartitionedGenerator.generateInserts(newCommitTime, 10); @@ -741,12 +759,13 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { /** * Test various metrics published by metadata table. */ - @Test - public void testMetadataMetrics() throws Exception { - init(HoodieTableType.COPY_ON_WRITE); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testMetadataMetrics(boolean populateMetaFields) throws Exception { + init(HoodieTableType.COPY_ON_WRITE, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(true, true, true).build())) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(true, true, true, populateMetaFields).build())) { // Write String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); List records = dataGen.generateInserts(newCommitTime, 20); @@ -769,15 +788,16 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { /** * Test when reading from metadata table which is out of sync with dataset that results are still consistent. */ - @Test - public void testMetadataOutOfSync() throws Exception { - init(HoodieTableType.COPY_ON_WRITE); + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testMetadataOutOfSync(boolean populateMetaFields) throws Exception { + init(HoodieTableType.COPY_ON_WRITE, populateMetaFields); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - SparkRDDWriteClient unsyncedClient = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true)); + SparkRDDWriteClient unsyncedClient = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields)); // Enable metadata so table is initialized - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) { // Perform Bulk Insert String newCommitTime = "001"; client.startCommitWithTime(newCommitTime); @@ -786,7 +806,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { } // Perform commit operations with metadata disabled - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) { // Perform Insert String newCommitTime = "002"; client.startCommitWithTime(newCommitTime); @@ -811,7 +831,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { validateMetadata(unsyncedClient); // Perform clean operation with metadata disabled - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) { // One more commit needed to trigger clean so upsert and compact String newCommitTime = "005"; client.startCommitWithTime(newCommitTime); @@ -833,7 +853,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { validateMetadata(unsyncedClient); // Perform restore with metadata disabled - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) { client.restoreToInstant("004"); } @@ -1008,18 +1028,20 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { } } - private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileListingMetadata) { - return getWriteConfigBuilder(autoCommit, useFileListingMetadata, false).build(); + private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileListingMetadata, boolean populateMetaFields) { + return getWriteConfigBuilder(autoCommit, useFileListingMetadata, false, populateMetaFields).build(); } - private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { - return getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, autoCommit, useFileListingMetadata, enableMetrics); + private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics, boolean populateMetaFields) { + return getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, autoCommit, useFileListingMetadata, enableMetrics, populateMetaFields); } - private HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { + private HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, + boolean enableMetrics, boolean populateMetaFields) { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2) .withAutoCommit(autoCommit) + .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) .withFailedWritesCleaningPolicy(policy) @@ -1028,7 +1050,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness { .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() .withEnableBackupForRemoteFileSystemView(false).build()) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(populateMetaFields ? HoodieIndex.IndexType.BLOOM : HoodieIndex.IndexType.SIMPLE).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(useFileListingMetadata) .enableMetrics(enableMetrics).build()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 300feb855..0c88777be 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -301,6 +301,13 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { } } + public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, + String initCommitTime, int numRecordsInThisCommit, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, + boolean assertForCommit, int expRecordsInThisCommit) throws Exception { + return insertFirstBatch(writeConfig, client, newCommitTime, initCommitTime, numRecordsInThisCommit, writeFn, isPreppedAPI, assertForCommit, expRecordsInThisCommit, true); + } + /** * Helper to insert first batch of records and do regular assertions on the state after successful completion. * @@ -319,12 +326,12 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, boolean filterForCommitTimeWithAssert) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, - recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1, false); + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1, false, filterForCommitTimeWithAssert); } /** @@ -355,6 +362,15 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false); } + public JavaRDD updateBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, + String prevCommitTime, Option> commitTimesBetweenPrevAndNew, String initCommitTime, + int numRecordsInThisCommit, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { + return updateBatch(writeConfig, client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, writeFn, + isPreppedAPI, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, true); + } + /** * Helper to upsert batch of records and do regular assertions on the state after successful completion. * @@ -378,13 +394,23 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { String prevCommitTime, Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, + boolean filterForCommitTimeWithAssert) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates); return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits, false); + expTotalCommits, false, filterForCommitTimeWithAssert); + } + + public JavaRDD deleteBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, + String prevCommitTime, String initCommitTime, + int numRecordsInThisCommit, + Function3, SparkRDDWriteClient, JavaRDD, String> deleteFn, boolean isPreppedAPI, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception { + return deleteBatch(writeConfig, client, newCommitTime, prevCommitTime, initCommitTime, numRecordsInThisCommit, deleteFn, isPreppedAPI, + assertForCommit, expRecordsInThisCommit, expTotalRecords, true); } /** @@ -408,13 +434,22 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> deleteFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, boolean filterForCommitTimeWithAssert) throws Exception { final Function> keyGenFunction = generateWrapDeleteKeysFn(isPreppedAPI, writeConfig, dataGen::generateUniqueDeletes); return deleteBatch(client, newCommitTime, prevCommitTime, initCommitTime, numRecordsInThisCommit, keyGenFunction, - deleteFn, assertForCommit, expRecordsInThisCommit, expTotalRecords); + deleteFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert); + } + + public JavaRDD writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, + Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, + Function2, String, Integer> recordGenFunction, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit) throws Exception { + return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, + writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true); } /** @@ -439,7 +474,8 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function2, String, Integer> recordGenFunction, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, + boolean filterForCommitTimeWithAssert) throws Exception { // Write 1 (only inserts) client.startCommitWithTime(newCommitTime); @@ -466,8 +502,10 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { "Expecting " + expTotalCommits + " commits."); assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), "Latest commit should be " + newCommitTime); - assertEquals(expRecordsInThisCommit, HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), - "Must contain " + expRecordsInThisCommit + " records"); + if (filterForCommitTimeWithAssert) { // when meta cols are disabled, we can't really do per commit assertion. + assertEquals(expRecordsInThisCommit, HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + "Must contain " + expRecordsInThisCommit + " records"); + } // Check the entire dataset has all records still String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; @@ -477,16 +515,18 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { assertEquals(expTotalRecords, HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count(), "Must contain " + expTotalRecords + " records"); - // Check that the incremental consumption from prevCommitTime - assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), - HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime), - "Incremental consumption from " + prevCommitTime + " should give all records in latest commit"); - if (commitTimesBetweenPrevAndNew.isPresent()) { - commitTimesBetweenPrevAndNew.get().forEach(ct -> { - assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), - HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, ct), - "Incremental consumption from " + ct + " should give all records in latest commit"); - }); + if (filterForCommitTimeWithAssert) { + // Check that the incremental consumption from prevCommitTime + assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime), + "Incremental consumption from " + prevCommitTime + " should give all records in latest commit"); + if (commitTimesBetweenPrevAndNew.isPresent()) { + commitTimesBetweenPrevAndNew.get().forEach(ct -> { + assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, ct), + "Incremental consumption from " + ct + " should give all records in latest commit"); + }); + } } } return result; @@ -510,7 +550,7 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { String initCommitTime, int numRecordsInThisCommit, Function> keyGenFunction, Function3, SparkRDDWriteClient, JavaRDD, String> deleteFn, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, boolean filerForCommitTimeWithAssert) throws Exception { // Delete 1 (only deletes) client.startCommitWithTime(newCommitTime); @@ -534,8 +574,10 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { "Expecting 3 commits."); assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), "Latest commit should be " + newCommitTime); - assertEquals(expRecordsInThisCommit, HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), - "Must contain " + expRecordsInThisCommit + " records"); + if (filerForCommitTimeWithAssert) { // if meta cols are disabled, we can't do assertion based on assertion time + assertEquals(expRecordsInThisCommit, HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + "Must contain " + expRecordsInThisCommit + " records"); + } // Check the entire dataset has all records still String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; @@ -545,11 +587,13 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { assertEquals(expTotalRecords, HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count(), "Must contain " + expTotalRecords + " records"); - // Check that the incremental consumption from prevCommitTime - assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), - HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime), - "Incremental consumption from " + prevCommitTime + " should give no records in latest commit," - + " since it is a delete operation"); + if (filerForCommitTimeWithAssert) { + // Check that the incremental consumption from prevCommitTime + assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime), + "Incremental consumption from " + prevCommitTime + " should give no records in latest commit," + + " since it is a delete operation"); + } } return result; } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index 951e17ca3..e618fa833 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; @@ -40,7 +41,9 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.WorkloadStat; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -56,6 +59,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.HashMap; import java.util.Map; +import java.util.Properties; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -225,6 +229,21 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType); } + protected Properties getPropertiesForKeyGen() { + Properties properties = new Properties(); + properties.put(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), "false"); + properties.put("hoodie.datasource.write.recordkey.field","_row_key"); + properties.put("hoodie.datasource.write.partitionpath.field","partition_path"); + return properties; + } + + protected void addAppropriatePropsForPopulateMetaFields(HoodieWriteConfig.Builder configBuilder, boolean populateMetaFields) { + if (!populateMetaFields) { + configBuilder.withProperties(getPropertiesForKeyGen()) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.SIMPLE).build()); + } + } + /** * Cleanups hoodie clients. */ diff --git a/hudi-client/hudi-spark-client/src/test/resources/testDataGeneratorSchema.txt b/hudi-client/hudi-spark-client/src/test/resources/testDataGeneratorSchema.txt index ada01b353..24d424237 100644 --- a/hudi-client/hudi-spark-client/src/test/resources/testDataGeneratorSchema.txt +++ b/hudi-client/hudi-spark-client/src/test/resources/testDataGeneratorSchema.txt @@ -25,6 +25,9 @@ }, { "name" : "_row_key", "type" : "string" + }, { + "name" : "partition_path", + "type" : "string" }, { "name" : "rider", "type" : "string" diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index 0f193c527..695f97e8a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -244,6 +244,22 @@ public class HoodieAvroUtils { return recordSchema; } + /** + * Fetch schema for record key and partition path. + */ + public static Schema getSchemaForFields(Schema fileSchema, List fields) { + List toBeAddedFields = new ArrayList<>(); + Schema recordSchema = Schema.createRecord("HoodieRecordKey", "", "", false); + + for (Schema.Field schemaField: fileSchema.getFields()) { + if (fields.contains(schemaField.name())) { + toBeAddedFields.add(new Schema.Field(schemaField.name(), schemaField.schema(), schemaField.doc(), schemaField.defaultValue())); + } + } + recordSchema.setFields(toBeAddedFields); + return recordSchema; + } + public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String recordKey, String partitionPath, String fileName) { record.put(HoodieRecord.FILENAME_METADATA_FIELD, fileName); diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java index c6160a8f1..9a6f26ac9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java @@ -20,6 +20,7 @@ package org.apache.hudi.avro; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter; +import org.apache.hudi.common.util.Option; import org.apache.avro.Schema; import org.apache.parquet.avro.AvroWriteSupport; @@ -33,7 +34,7 @@ import java.util.HashMap; */ public class HoodieAvroWriteSupport extends AvroWriteSupport { - private BloomFilter bloomFilter; + private Option bloomFilterOpt; private String minRecordKey; private String maxRecordKey; @@ -44,39 +45,41 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport { public static final String HOODIE_BLOOM_FILTER_TYPE_CODE = "hoodie_bloom_filter_type_code"; public static final String HOODIE_AVRO_SCHEMA_METADATA_KEY = "orc.avro.schema"; - public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) { + public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, Option bloomFilterOpt) { super(schema, avroSchema); - this.bloomFilter = bloomFilter; + this.bloomFilterOpt = bloomFilterOpt; } @Override public WriteSupport.FinalizedWriteContext finalizeWrite() { HashMap extraMetaData = new HashMap<>(); - if (bloomFilter != null) { - extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString()); + if (bloomFilterOpt.isPresent()) { + extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilterOpt.get().serializeToString()); if (minRecordKey != null && maxRecordKey != null) { extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey); extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey); } - if (bloomFilter.getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) { - extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilter.getBloomFilterTypeCode().name()); + if (bloomFilterOpt.get().getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) { + extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilterOpt.get().getBloomFilterTypeCode().name()); } } return new WriteSupport.FinalizedWriteContext(extraMetaData); } public void add(String recordKey) { - this.bloomFilter.add(recordKey); - if (minRecordKey != null) { - minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : recordKey; - } else { - minRecordKey = recordKey; - } + if (bloomFilterOpt.isPresent()) { + this.bloomFilterOpt.get().add(recordKey); + if (minRecordKey != null) { + minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : recordKey; + } else { + minRecordKey = recordKey; + } - if (maxRecordKey != null) { - maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey; - } else { - maxRecordKey = recordKey; + if (maxRecordKey != null) { + maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey; + } else { + maxRecordKey = recordKey; + } } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 01dd04082..74c985965 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -319,12 +319,6 @@ public class HoodieTableMetaClient implements Serializable { * @param operationType operation type to be executed. */ public void validateTableProperties(Properties properties, WriteOperationType operationType) { - // disabling meta fields are allowed only for bulk_insert operation - if (!Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue())) - && operationType != WriteOperationType.BULK_INSERT) { - throw new HoodieException(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() + " can only be disabled for " + WriteOperationType.BULK_INSERT - + " operation"); - } // once meta fields are disabled, it cant be re-enabled for a given table. if (!getTableConfig().populateMetaFields() && Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()))) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java index 9b95e1638..25a2bec5b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/BaseFileUtils.java @@ -34,6 +34,7 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.keygen.BaseKeyGenerator; public abstract class BaseFileUtils { @@ -170,6 +171,15 @@ public abstract class BaseFileUtils { */ public abstract List fetchRecordKeyPartitionPath(Configuration configuration, Path filePath); + /** + * Fetch {@link HoodieKey}s from the given data file. + * @param configuration configuration to build fs object + * @param filePath The data file path + * @param keyGeneratorOpt instance of KeyGenerator. + * @return {@link List} of {@link HoodieKey}s fetched from the parquet file + */ + public abstract List fetchRecordKeyPartitionPath(Configuration configuration, Path filePath, Option keyGeneratorOpt); + /** * Read the Avro schema of the data file. * @param configuration Configuration diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java index b5d11cb8b..bf2473913 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java @@ -41,6 +41,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.MetadataNotFoundException; +import org.apache.hudi.keygen.BaseKeyGenerator; + import org.apache.orc.OrcFile; import org.apache.orc.OrcProto.UserMetadataItem; import org.apache.orc.Reader; @@ -109,6 +111,11 @@ public class OrcUtils extends BaseFileUtils { return hoodieKeys; } + @Override + public List fetchRecordKeyPartitionPath(Configuration configuration, Path filePath, Option keyGeneratorOpt) { + throw new HoodieIOException("UnsupportedOperation : Disabling meta fields not yet supported for Orc"); + } + /** * NOTE: This literally reads the entire file contents, thus should be used with caution. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index bd44724e7..ebe361025 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.MetadataNotFoundException; +import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -115,23 +116,36 @@ public class ParquetUtils extends BaseFileUtils { */ @Override public List fetchRecordKeyPartitionPath(Configuration configuration, Path filePath) { + return fetchRecordKeyPartitionPathInternal(configuration, filePath, Option.empty()); + } + + private List fetchRecordKeyPartitionPathInternal(Configuration configuration, Path filePath, Option keyGeneratorOpt) { List hoodieKeys = new ArrayList<>(); try { - if (!filePath.getFileSystem(configuration).exists(filePath)) { - return new ArrayList<>(); - } - Configuration conf = new Configuration(configuration); conf.addResource(FSUtils.getFs(filePath.toString(), conf).getConf()); - Schema readSchema = HoodieAvroUtils.getRecordKeyPartitionPathSchema(); + Schema readSchema = keyGeneratorOpt.map(keyGenerator -> { + List fields = new ArrayList<>(); + fields.addAll(keyGenerator.getRecordKeyFields()); + fields.addAll(keyGenerator.getPartitionPathFields()); + return HoodieAvroUtils.getSchemaForFields(readAvroSchema(conf, filePath), fields); + }) + .orElse(HoodieAvroUtils.getRecordKeyPartitionPathSchema()); AvroReadSupport.setAvroReadSchema(conf, readSchema); AvroReadSupport.setRequestedProjection(conf, readSchema); ParquetReader reader = AvroParquetReader.builder(filePath).withConf(conf).build(); Object obj = reader.read(); while (obj != null) { if (obj instanceof GenericRecord) { - String recordKey = ((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); - String partitionPath = ((GenericRecord) obj).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + String recordKey = null; + String partitionPath = null; + if (keyGeneratorOpt.isPresent()) { + recordKey = keyGeneratorOpt.get().getRecordKey((GenericRecord) obj); + partitionPath = keyGeneratorOpt.get().getPartitionPath((GenericRecord) obj); + } else { + recordKey = ((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + partitionPath = ((GenericRecord) obj).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + } hoodieKeys.add(new HoodieKey(recordKey, partitionPath)); obj = reader.read(); } @@ -142,6 +156,19 @@ public class ParquetUtils extends BaseFileUtils { return hoodieKeys; } + /** + * Fetch {@link HoodieKey}s from the given parquet file. + * + * @param configuration configuration to build fs object + * @param filePath The parquet file path. + * @param keyGeneratorOpt + * @return {@link List} of {@link HoodieKey}s fetched from the parquet file + */ + @Override + public List fetchRecordKeyPartitionPath(Configuration configuration, Path filePath, Option keyGeneratorOpt) { + return fetchRecordKeyPartitionPathInternal(configuration, filePath, keyGeneratorOpt); + } + public ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) { ParquetMetadata footer; try { @@ -223,7 +250,7 @@ public class ParquetUtils extends BaseFileUtils { /** * Returns the number of records in the parquet file. * - * @param conf Configuration + * @param conf Configuration * @param parquetFilePath path of the file */ @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java b/hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java similarity index 100% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java rename to hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-common/src/main/java/org/apache/hudi/keygen/KeyGenerator.java similarity index 100% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenerator.java rename to hudi-common/src/main/java/org/apache/hudi/keygen/KeyGenerator.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java b/hudi-common/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java similarity index 100% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java rename to hudi-common/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java similarity index 100% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java rename to hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorType.java b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorType.java similarity index 100% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorType.java rename to hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorType.java diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroWriteSupport.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroWriteSupport.java index 72f1453d5..16a77c145 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroWriteSupport.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroWriteSupport.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; @@ -52,7 +53,7 @@ public class TestHoodieAvroWriteSupport { 1000, 0.0001, 10000, BloomFilterTypeCode.SIMPLE.name()); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( - new AvroSchemaConverter().convert(schema), schema, filter); + new AvroSchemaConverter().convert(schema), schema, Option.of(filter)); ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); for (String rowKey : rowKeys) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index 7499894c2..76f64b46b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -90,6 +90,7 @@ public class HoodieTestDataGenerator { public static final int DEFAULT_PARTITION_DEPTH = 3; public static final String TRIP_SCHEMA_PREFIX = "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ " + "{\"name\": \"timestamp\",\"type\": \"long\"}," + "{\"name\": \"_row_key\", \"type\": \"string\"}," + + "{\"name\": \"partition_path\", \"type\": \"string\"}," + "{\"name\": \"rider\", \"type\": \"string\"}," + "{\"name\": \"driver\", \"type\": \"string\"}," + "{\"name\": \"begin_lat\", \"type\": \"double\"}," + "{\"name\": \"begin_lon\", \"type\": \"double\"}," + "{\"name\": \"end_lat\", \"type\": \"double\"}," + "{\"name\": \"end_lon\", \"type\": \"double\"},"; @@ -123,7 +124,7 @@ public class HoodieTestDataGenerator { + "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}"; public static final String NULL_SCHEMA = Schema.create(Schema.Type.NULL).toString(); - public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,double,double,double,double,int,bigint,float,binary,int,bigint,decimal(10,6)," + public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,string,double,double,double,double,int,bigint,float,binary,int,bigint,decimal(10,6)," + "map,struct,array>,boolean"; @@ -208,7 +209,7 @@ public class HoodieTestDataGenerator { public static RawTripTestPayload generateRandomValue( HoodieKey key, String instantTime, boolean isFlattened) throws IOException { GenericRecord rec = generateGenericRecord( - key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0, + key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0, false, isFlattened); return new RawTripTestPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA); } @@ -230,7 +231,7 @@ public class HoodieTestDataGenerator { * Generates a new avro record of the above schema format for a delete. */ public static RawTripTestPayload generateRandomDeleteValue(HoodieKey key, String instantTime) throws IOException { - GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0, + GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0, true, false); return new RawTripTestPayload(Option.of(rec.toString()), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA, true); } @@ -239,21 +240,22 @@ public class HoodieTestDataGenerator { * Generates a new avro record of the above schema format, retaining the key if optionally provided. */ public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String instantTime) { - GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0); + GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0); return new HoodieAvroPayload(Option.of(rec)); } - public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName, + public static GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName, long timestamp) { - return generateGenericRecord(rowKey, riderName, driverName, timestamp, false, false); + return generateGenericRecord(rowKey, partitionPath, riderName, driverName, timestamp, false, false); } - public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName, + public static GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName, long timestamp, boolean isDeleteRecord, boolean isFlattened) { GenericRecord rec = new GenericData.Record(isFlattened ? FLATTENED_AVRO_SCHEMA : AVRO_SCHEMA); rec.put("_row_key", rowKey); rec.put("timestamp", timestamp); + rec.put("partition_path", partitionPath); rec.put("rider", riderName); rec.put("driver", driverName); rec.put("begin_lat", RAND.nextDouble()); @@ -807,7 +809,7 @@ public class HoodieTestDataGenerator { public List generateGenericRecords(int numRecords) { List list = new ArrayList<>(); IntStream.range(0, numRecords).forEach(i -> { - list.add(generateGenericRecord(UUID.randomUUID().toString(), UUID.randomUUID().toString(), UUID.randomUUID() + list.add(generateGenericRecord(UUID.randomUUID().toString(), "0", UUID.randomUUID().toString(), UUID.randomUUID() .toString(), RAND.nextLong())); }); return list; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestAvroOrcUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestAvroOrcUtils.java index b775a377b..692aa1ed1 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestAvroOrcUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestAvroOrcUtils.java @@ -40,7 +40,7 @@ public class TestAvroOrcUtils extends HoodieCommonTestHarness { // The following types are tested: // DATE, DECIMAL, LONG, INT, BYTES, ARRAY, RECORD, MAP, STRING, FLOAT, DOUBLE TypeDescription orcSchema = TypeDescription.fromString("struct<" - + "timestamp:bigint,_row_key:string,rider:string,driver:string,begin_lat:double," + + "timestamp:bigint,_row_key:string,partition_path:string,rider:string,driver:string,begin_lat:double," + "begin_lon:double,end_lat:double,end_lon:double," + "distance_in_meters:int,seconds_since_epoch:bigint,weight:float,nation:binary," + "current_date:date,current_ts:bigint,height:decimal(10,6)," diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java index 6735c1041..e07c0fad3 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java @@ -23,11 +23,14 @@ import org.apache.hudi.avro.HoodieAvroWriteSupport; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.avro.JsonProperties; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; @@ -50,6 +53,7 @@ import java.util.List; import java.util.Set; import java.util.UUID; +import static org.apache.hudi.avro.HoodieAvroUtils.METADATA_FIELD_SCHEMA; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -151,6 +155,33 @@ public class TestParquetUtils extends HoodieCommonTestHarness { } } + @Test + public void testFetchRecordKeyPartitionPathVirtualKeysFromParquet() throws Exception { + List rowKeys = new ArrayList<>(); + List expected = new ArrayList<>(); + String partitionPath = "path1"; + for (int i = 0; i < 1000; i++) { + String rowKey = UUID.randomUUID().toString(); + rowKeys.add(rowKey); + expected.add(new HoodieKey(rowKey, partitionPath)); + } + + String filePath = Paths.get(basePath, "test.parquet").toUri().toString(); + Schema schema = getSchemaWithFields(Arrays.asList(new String[]{"abc", "def"})); + writeParquetFile(BloomFilterTypeCode.SIMPLE.name(), filePath, rowKeys, schema, true, partitionPath, + false, "abc", "def"); + + // Read and verify + List fetchedRows = + parquetUtils.fetchRecordKeyPartitionPath(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath), + Option.of(new TestBaseKeyGen("abc","def"))); + assertEquals(rowKeys.size(), fetchedRows.size(), "Total count does not match"); + + for (HoodieKey entry : fetchedRows) { + assertTrue(expected.contains(entry), "Record key must be in the given filter"); + } + } + @Test public void testReadCounts() throws Exception { String filePath = Paths.get(basePath, "test.parquet").toUri().toString(); @@ -168,22 +199,73 @@ public class TestParquetUtils extends HoodieCommonTestHarness { } private void writeParquetFile(String typeCode, String filePath, List rowKeys, Schema schema, boolean addPartitionPathField, String partitionPath) throws Exception { + writeParquetFile(typeCode, filePath, rowKeys, schema, addPartitionPathField, partitionPath, + true, null, null); + } + + private void writeParquetFile(String typeCode, String filePath, List rowKeys, Schema schema, boolean addPartitionPathField, String partitionPathValue, + boolean useMetaFields, String recordFieldName, String partitionFieldName) throws Exception { // Write out a parquet file BloomFilter filter = BloomFilterFactory .createBloomFilter(1000, 0.0001, 10000, typeCode); HoodieAvroWriteSupport writeSupport = - new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter); + new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, Option.of(filter)); ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); for (String rowKey : rowKeys) { GenericRecord rec = new GenericData.Record(schema); - rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey); + rec.put(useMetaFields ? HoodieRecord.RECORD_KEY_METADATA_FIELD : recordFieldName, rowKey); if (addPartitionPathField) { - rec.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, partitionPath); + rec.put(useMetaFields ? HoodieRecord.PARTITION_PATH_METADATA_FIELD : partitionFieldName, partitionPathValue); } writer.write(rec); writeSupport.add(rowKey); } writer.close(); } + + private static Schema getSchemaWithFields(List fields) { + List toBeAddedFields = new ArrayList<>(); + Schema recordSchema = Schema.createRecord("HoodieRecordKey", "", "", false); + + for (String field: fields) { + Schema.Field schemaField = + new Schema.Field(field, METADATA_FIELD_SCHEMA, "", JsonProperties.NULL_VALUE); + toBeAddedFields.add(schemaField); + } + recordSchema.setFields(toBeAddedFields); + return recordSchema; + } + + class TestBaseKeyGen extends BaseKeyGenerator { + + private String recordKeyField; + private String partitionField; + + public TestBaseKeyGen(String recordKeyField, String partitionField) { + super(new TypedProperties()); + this.recordKeyField = recordKeyField; + this.partitionField = partitionField; + } + + @Override + public String getRecordKey(GenericRecord record) { + return record.get(recordKeyField).toString(); + } + + @Override + public String getPartitionPath(GenericRecord record) { + return record.get(partitionField).toString(); + } + + @Override + public List getRecordKeyFields() { + return Arrays.asList(new String[]{recordKeyField}); + } + + @Override + public List getPartitionPathFields() { + return Arrays.asList(new String[]{partitionField}); + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java index 2facfc9b4..2df811374 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java @@ -562,7 +562,7 @@ public class TestBootstrap extends HoodieClientTestBase { final List records = new ArrayList<>(); IntStream.range(from, to).forEach(i -> { String id = "" + i; - records.add(generateGenericRecord("trip_" + id, "rider_" + id, "driver_" + id, + records.add(generateGenericRecord("trip_" + id, Long.toString(timestamp), "rider_" + id, "driver_" + id, timestamp, false, false).toString()); }); if (isPartitioned) { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala index b2417dd75..1fbc13c1e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala @@ -29,6 +29,8 @@ import org.apache.hudi.exception.{HoodieException, HoodieKeyException} import org.apache.hudi.keygen._ import org.apache.hudi.testutils.KeyGeneratorTestUtilities import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types.StructType import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{BeforeEach, Test} import org.scalatest.Assertions.fail @@ -259,6 +261,8 @@ class TestDataSourceDefaults { val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord] getKey(genericRecord).getPartitionPath } + + override def getPartitionPath(internalRow: InternalRow, structType: StructType): String = null } @Test def testComplexKeyGenerator() = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala index 54fc2dcfd..512760a11 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala @@ -392,18 +392,19 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers { } } - List((DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, HoodieFileFormat.PARQUET.name()), (DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, HoodieFileFormat.ORC.name()), - (DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, HoodieFileFormat.PARQUET.name()), (DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, HoodieFileFormat.ORC.name())) + List((DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, HoodieFileFormat.PARQUET.name(), true), (DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, HoodieFileFormat.ORC.name(), true), + (DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, HoodieFileFormat.PARQUET.name(), true), (DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, HoodieFileFormat.ORC.name(), true), + (DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, HoodieFileFormat.PARQUET.name(), false)) .foreach(t => { val tableType = t._1 val baseFileFormat = t._2 - test("test basic HoodieSparkSqlWriter functionality with datasource insert for " + tableType + " with " + baseFileFormat + "as the base file format") { - initSparkContext("test_insert_datasource") + val populateMetaFields = t._3 + test("test basic HoodieSparkSqlWriter functionality with datasource insert for " + tableType + " with " + baseFileFormat + " as the base file format " + + " with populate meta fields " + populateMetaFields) { + initSparkContext("test_insert_base_file_format_datasource") val path = java.nio.file.Files.createTempDirectory("hoodie_test_path") try { - val hoodieFooTableName = "hoodie_foo_tbl" - //create a new table val fooTableModifier = Map("path" -> path.toAbsolutePath.toString, HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName, @@ -413,6 +414,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers { DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL, DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key", DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition", + HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() -> String.valueOf(populateMetaFields), DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> classOf[SimpleKeyGenerator].getCanonicalName) val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier) diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 7b9f1d127..4b916bc4b 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -40,6 +40,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.minicluster.ZookeeperTestService; import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.hive.ddl.HiveQueryDDLExecutor; @@ -351,7 +352,7 @@ public class HiveTestUtil { org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, BloomFilterTypeCode.SIMPLE.name()); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, filter); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, Option.of(filter)); ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf()); @@ -373,7 +374,7 @@ public class HiveTestUtil { org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, BloomFilterTypeCode.SIMPLE.name()); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, filter); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, Option.of(filter)); ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf()); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java index 6a077e10a..ce513c183 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.Option; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -226,7 +227,7 @@ public class TestCluster implements BeforeAllCallback, AfterAllCallback, org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, BloomFilterTypeCode.SIMPLE.name()); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, filter); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, Option.of(filter)); ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, dfsCluster.getFileSystem().getConf()); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java index 1fe6800eb..c72cf721c 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java @@ -231,7 +231,7 @@ public class TestHDFSParquetImporter extends FunctionalTestHarness implements Se long startTime = HoodieActiveTimeline.COMMIT_FORMATTER.parse("20170203000000").getTime() / 1000; List records = new ArrayList(); for (long recordNum = 0; recordNum < 96; recordNum++) { - records.add(HoodieTestDataGenerator.generateGenericRecord(Long.toString(recordNum), "rider-" + recordNum, + records.add(HoodieTestDataGenerator.generateGenericRecord(Long.toString(recordNum), "0", "rider-" + recordNum, "driver-" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); } try (ParquetWriter writer = AvroParquetWriter.builder(srcFile) @@ -249,12 +249,12 @@ public class TestHDFSParquetImporter extends FunctionalTestHarness implements Se List records = new ArrayList(); // 10 for update for (long recordNum = 0; recordNum < 11; recordNum++) { - records.add(HoodieTestDataGenerator.generateGenericRecord(Long.toString(recordNum), "rider-upsert-" + recordNum, + records.add(HoodieTestDataGenerator.generateGenericRecord(Long.toString(recordNum), "0", "rider-upsert-" + recordNum, "driver-upsert" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); } // 4 for insert for (long recordNum = 96; recordNum < 100; recordNum++) { - records.add(HoodieTestDataGenerator.generateGenericRecord(Long.toString(recordNum), "rider-upsert-" + recordNum, + records.add(HoodieTestDataGenerator.generateGenericRecord(Long.toString(recordNum), "0", "rider-upsert-" + recordNum, "driver-upsert" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); } try (ParquetWriter writer = AvroParquetWriter.builder(srcFile) diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/source-flattened.avsc b/hudi-utilities/src/test/resources/delta-streamer-config/source-flattened.avsc index 7b3ee50d9..d03178ab3 100644 --- a/hudi-utilities/src/test/resources/delta-streamer-config/source-flattened.avsc +++ b/hudi-utilities/src/test/resources/delta-streamer-config/source-flattened.avsc @@ -26,6 +26,9 @@ "name" : "_row_key", "type" : "string" }, { + "name" : "partition_path", + "type" : "string" + }, { "name" : "rider", "type" : "string" }, { diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/source.avsc b/hudi-utilities/src/test/resources/delta-streamer-config/source.avsc index 4b4beb35d..a10ab74f9 100644 --- a/hudi-utilities/src/test/resources/delta-streamer-config/source.avsc +++ b/hudi-utilities/src/test/resources/delta-streamer-config/source.avsc @@ -26,6 +26,9 @@ "name" : "_row_key", "type" : "string" }, { + "name" : "partition_path", + "type" : "string" + }, { "name" : "rider", "type" : "string" }, { diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/sql-transformer.properties b/hudi-utilities/src/test/resources/delta-streamer-config/sql-transformer.properties index dc735e803..93f95d3dc 100644 --- a/hudi-utilities/src/test/resources/delta-streamer-config/sql-transformer.properties +++ b/hudi-utilities/src/test/resources/delta-streamer-config/sql-transformer.properties @@ -16,4 +16,4 @@ # limitations under the License. ### include=base.properties -hoodie.deltastreamer.transformer.sql=SELECT a.timestamp, a._row_key, a.rider, a.driver, a.begin_lat, a.begin_lon, a.end_lat, a.end_lon, a.distance_in_meters, a.seconds_since_epoch, a.weight, a.nation, a.current_date, a.current_ts, a.height, a.city_to_state, a.fare, a.tip_history, a.`_hoodie_is_deleted`, CAST(1.0 AS DOUBLE) AS haversine_distance FROM a +hoodie.deltastreamer.transformer.sql=SELECT a.timestamp, a._row_key, a.partition_path, a.rider, a.driver, a.begin_lat, a.begin_lon, a.end_lat, a.end_lon, a.distance_in_meters, a.seconds_since_epoch, a.weight, a.nation, a.current_date, a.current_ts, a.height, a.city_to_state, a.fare, a.tip_history, a.`_hoodie_is_deleted`, CAST(1.0 AS DOUBLE) AS haversine_distance FROM a diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/target-flattened.avsc b/hudi-utilities/src/test/resources/delta-streamer-config/target-flattened.avsc index dcea4cb09..2be01709c 100644 --- a/hudi-utilities/src/test/resources/delta-streamer-config/target-flattened.avsc +++ b/hudi-utilities/src/test/resources/delta-streamer-config/target-flattened.avsc @@ -26,6 +26,9 @@ "name" : "_row_key", "type" : "string" }, { + "name" : "partition_path", + "type" : "string" + }, { "name" : "rider", "type" : "string" }, { diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/target.avsc b/hudi-utilities/src/test/resources/delta-streamer-config/target.avsc index 4252b7e66..318dfdcc0 100644 --- a/hudi-utilities/src/test/resources/delta-streamer-config/target.avsc +++ b/hudi-utilities/src/test/resources/delta-streamer-config/target.avsc @@ -25,6 +25,9 @@ }, { "name" : "_row_key", "type" : "string" + }, { + "name" : "partition_path", + "type" : "string" }, { "name" : "rider", "type" : "string"