diff --git a/azure-pipelines.yml b/azure-pipelines.yml index 278a3f09f99f6..93c5c8db0fe8b 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -155,7 +155,7 @@ stages: displayName: Top 100 long-running testcases - job: UT_FT_3 displayName: UT spark-datasource - timeoutInMinutes: '150' + timeoutInMinutes: '180' steps: - task: Maven@4 displayName: maven install @@ -179,7 +179,7 @@ stages: displayName: Top 100 long-running testcases - job: UT_FT_4 displayName: UT FT other modules - timeoutInMinutes: '150' + timeoutInMinutes: '180' steps: - task: Maven@4 displayName: maven install diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java index 7b967ad064900..9d3b92c907ad0 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java @@ -28,6 +28,8 @@ import org.apache.hudi.cli.utils.SparkUtil; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Reader; @@ -118,7 +120,7 @@ public String showArchivedCommits( // read the avro blocks while (reader.hasNext()) { HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); - blk.getRecordIterator().forEachRemaining(readRecords::add); + blk.getRecordIterator(HoodieRecordType.AVRO).forEachRemaining(r -> readRecords.add((IndexedRecord) r.getData())); } List readCommits = readRecords.stream().map(r -> (GenericRecord) r) .filter(r -> r.get("actionType").toString().equals(HoodieTimeline.COMMIT_ACTION) @@ -192,8 +194,8 @@ public String showCommits( // read the avro blocks while (reader.hasNext()) { HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); - try (ClosableIterator recordItr = blk.getRecordIterator()) { - recordItr.forEachRemaining(readRecords::add); + try (ClosableIterator> recordItr = blk.getRecordIterator(HoodieRecordType.AVRO)) { + recordItr.forEachRemaining(r -> readRecords.add(r.getData())); } } List readCommits = readRecords.stream().map(r -> (GenericRecord) r) diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java index 333eb11b0c077..7dd889e58e374 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java @@ -32,6 +32,8 @@ import org.apache.hudi.cli.HoodieCLI; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Reader; @@ -121,9 +123,9 @@ private int copyArchivedInstants(List statuses, Set actionSe // read the avro blocks while (reader.hasNext() && copyCount < limit) { HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); - try (ClosableIterator recordItr = blk.getRecordIterator()) { + try (ClosableIterator> recordItr = blk.getRecordIterator(HoodieRecordType.AVRO)) { while (recordItr.hasNext()) { - IndexedRecord ir = recordItr.next(); + IndexedRecord ir = recordItr.next().getData(); // Archived instants are saved as arvo encoded HoodieArchivedMetaEntry records. We need to get the // metadata record from the entry and convert it to json. HoodieArchivedMetaEntry archiveEntryRecord = (HoodieArchivedMetaEntry) SpecificData.get() diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java index 22f70480a3f4d..9a0e485fc9fa0 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java @@ -30,9 +30,11 @@ import org.apache.hudi.cli.TableHeader; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieLogFormat; @@ -44,6 +46,7 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieMemoryConfig; @@ -61,6 +64,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Properties; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -121,7 +125,7 @@ public String showLogFileCommits( instantTime = "dummy_instant_time_" + dummyInstantTimeCount; } if (n instanceof HoodieDataBlock) { - try (ClosableIterator recordItr = ((HoodieDataBlock) n).getRecordIterator()) { + try (ClosableIterator> recordItr = ((HoodieDataBlock) n).getRecordIterator(HoodieRecordType.AVRO)) { recordItr.forEachRemaining(r -> recordCount.incrementAndGet()); } } @@ -217,12 +221,13 @@ public String showLogFileRecords( .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue()) .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue())) .build(); - for (HoodieRecord hoodieRecord : scanner) { - Option record = hoodieRecord.getData().getInsertValue(readerSchema); + for (HoodieRecord hoodieRecord : scanner) { + Option record = hoodieRecord.toIndexedRecord(readerSchema, new Properties()); if (allRecords.size() < limit) { - allRecords.add(record.get()); + allRecords.add(record.get().getData()); } } } else { @@ -236,10 +241,10 @@ public String showLogFileRecords( HoodieLogBlock n = reader.next(); if (n instanceof HoodieDataBlock) { HoodieDataBlock blk = (HoodieDataBlock) n; - try (ClosableIterator recordItr = blk.getRecordIterator()) { + try (ClosableIterator> recordItr = blk.getRecordIterator(HoodieRecordType.AVRO)) { recordItr.forEachRemaining(record -> { if (allRecords.size() < limit) { - allRecords.add(record); + allRecords.add(record.getData()); } }); } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java index bcccb66b3716c..456610b3911de 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java @@ -21,11 +21,11 @@ import org.apache.hudi.cli.HoodieCliSparkConfig; import org.apache.hudi.cli.commands.SparkEnvCommand; import org.apache.hudi.cli.commands.SparkMain; -import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; +import org.apache.spark.HoodieSparkKryoProvider$; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.launcher.SparkLauncher; @@ -116,7 +116,7 @@ public static JavaSparkContext initJavaSparkContext(String name, Option } public static JavaSparkContext initJavaSparkContext(SparkConf sparkConf) { - SparkRDDWriteClient.registerClasses(sparkConf); + HoodieSparkKryoProvider$.MODULE$.register(sparkConf); JavaSparkContext jsc = new JavaSparkContext(sparkConf); jsc.hadoopConfiguration().setBoolean(HoodieCliSparkConfig.CLI_PARQUET_ENABLE_SUMMARY_METADATA, false); FSUtils.prepareHadoopConf(jsc.hadoopConfiguration()); 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 5c0e4a662cd4a..afbab33072c7f 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 @@ -18,20 +18,19 @@ package org.apache.hudi.cli import org.apache.avro.Schema -import org.apache.avro.generic.IndexedRecord import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hudi.avro.HoodieAvroWriteSupport import org.apache.hudi.client.SparkTaskContextSupplier import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory} +import org.apache.hudi.common.config.HoodieStorageConfig import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.util.BaseFileUtils -import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig} +import org.apache.hudi.config.HoodieIndexConfig import org.apache.hudi.io.storage.{HoodieAvroParquetWriter, HoodieParquetConfig} import org.apache.parquet.avro.AvroSchemaConverter import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.spark.sql.{DataFrame, SQLContext} - import scala.collection.JavaConversions._ import scala.collection.mutable._ @@ -58,7 +57,7 @@ object SparkHelpers { // Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'. parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader) - val writer = new HoodieAvroParquetWriter[IndexedRecord](destinationFile, parquetConfig, instantTime, new SparkTaskContextSupplier(), true) + val writer = new HoodieAvroParquetWriter(destinationFile, parquetConfig, instantTime, 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-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java index 25d54b648715f..21e6218dbe27b 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java @@ -28,9 +28,11 @@ import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; @@ -38,6 +40,7 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.SchemaTestUtil; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieMemoryConfig; @@ -111,7 +114,7 @@ public void init() throws IOException, InterruptedException, URISyntaxException .withFileId("test-log-fileid1").overBaseCommit("100").withFs(fs).build()) { // write data to file - List records = SchemaTestUtil.generateTestRecords(0, 100); + List records = SchemaTestUtil.generateTestRecords(0, 100).stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()); Map header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); @@ -191,7 +194,7 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc .withFileExtension(HoodieLogFile.DELTA_EXTENSION) .withFileId("test-log-fileid1").overBaseCommit(INSTANT_TIME).withFs(fs).withSizeThreshold(500).build(); - List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100).stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()); Map header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -228,15 +231,16 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue()) .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue())) .build(); - Iterator> records = scanner.iterator(); + Iterator records = scanner.iterator(); int num = 0; int maxSize = 10; List indexRecords = new ArrayList<>(); while (records.hasNext() && num < maxSize) { - Option hoodieRecord = records.next().getData().getInsertValue(schema); + Option hoodieRecord = ((HoodieAvroRecord)records.next()).getData().getInsertValue(schema); indexRecords.add(hoodieRecord.get()); num++; } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/functional/CLIFunctionalTestHarness.java b/hudi-cli/src/test/java/org/apache/hudi/cli/functional/CLIFunctionalTestHarness.java index 04f77df549606..b41d71cbcaeec 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/functional/CLIFunctionalTestHarness.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/functional/CLIFunctionalTestHarness.java @@ -20,7 +20,6 @@ package org.apache.hudi.cli.functional; import org.apache.hudi.client.SparkRDDReadClient; -import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.testutils.HoodieClientTestUtils; @@ -28,6 +27,7 @@ import org.apache.hudi.timeline.service.TimelineService; import org.apache.hadoop.conf.Configuration; +import org.apache.spark.HoodieSparkKryoProvider$; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; @@ -100,7 +100,7 @@ public synchronized void runBeforeEach() { initialized = spark != null; if (!initialized) { SparkConf sparkConf = conf(); - SparkRDDWriteClient.registerClasses(sparkConf); + HoodieSparkKryoProvider$.MODULE$.register(sparkConf); SparkRDDReadClient.addHoodieSupport(sparkConf); spark = SparkSession.builder().config(sparkConf).getOrCreate(); sqlContext = spark.sqlContext(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseClusterer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseClusterer.java index 648ce805b0825..a3f552e640ffc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseClusterer.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseClusterer.java @@ -19,7 +19,6 @@ package org.apache.hudi.client; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import java.io.IOException; @@ -28,7 +27,7 @@ /** * Client will run one round of clustering. */ -public abstract class BaseClusterer implements Serializable { +public abstract class BaseClusterer implements Serializable { private static final long serialVersionUID = 1L; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseCompactor.java index 88737dbcf1d7e..ba4d3f77fd9e5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseCompactor.java @@ -18,7 +18,6 @@ package org.apache.hudi.client; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import java.io.IOException; @@ -27,7 +26,7 @@ /** * Run one round of compaction. */ -public abstract class BaseCompactor implements Serializable { +public abstract class BaseCompactor implements Serializable { private static final long serialVersionUID = 1L; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 1fcaff04559ae..48382e0870a32 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -44,7 +44,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteOperationType; @@ -123,12 +122,12 @@ * Abstract Write Client providing functionality for performing commit, index updates and rollback * Reused for regular write operations like upsert/insert/bulk-insert.. as well as bootstrap * - * @param Sub type of HoodieRecordPayload + * @param Type of data * @param Type of inputs * @param Type of keys * @param Type of outputs */ -public abstract class BaseHoodieWriteClient extends BaseHoodieClient +public abstract class BaseHoodieWriteClient extends BaseHoodieClient implements RunsTableService { protected static final String LOOKUP_STR = "lookup"; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index a61a5c9008293..6d632417a42bf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -30,8 +30,11 @@ import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; @@ -341,7 +344,7 @@ public void mergeArchiveFiles(List compactCandidate) throws IOExcept // Read the avro blocks while (reader.hasNext()) { HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); - blk.getRecordIterator().forEachRemaining(records::add); + blk.getRecordIterator(HoodieRecordType.AVRO).forEachRemaining(r -> records.add((IndexedRecord) r.getData())); if (records.size() >= this.config.getCommitArchivalBatchSize()) { writeToFile(wrapperSchema, records); } @@ -668,7 +671,8 @@ private void writeToFile(Schema wrapperSchema, List records) thro Map header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); final String keyField = table.getMetaClient().getTableConfig().getRecordKeyFieldProp(); - HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header, keyField); + List indexRecords = records.stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()); + HoodieAvroDataBlock block = new HoodieAvroDataBlock(indexRecords, header, keyField); writer.appendBlock(block); records.clear(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java index 1cf1702717295..636e0e4b2603a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -49,8 +50,9 @@ public FullRecordBootstrapDataProvider(TypedProperties props, HoodieEngineContex * @param tableName Hudi Table Name * @param sourceBasePath Source Base Path * @param partitionPaths Partition Paths + * @param config config * @return input records */ public abstract I generateInputRecords(String tableName, - String sourceBasePath, List>> partitionPaths); + String sourceBasePath, List>> partitionPaths, HoodieWriteConfig config); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MergingIterator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MergingIterator.java index 47dde723e00dd..abfbc987c16eb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MergingIterator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MergingIterator.java @@ -18,20 +18,19 @@ package org.apache.hudi.client.utils; -import java.util.Iterator; -import java.util.function.Function; -import org.apache.avro.generic.GenericRecord; - +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.common.util.collection.Pair; -public class MergingIterator implements Iterator { +import java.util.Iterator; +import java.util.function.BiFunction; + +public class MergingIterator implements Iterator { - private final Iterator leftIterator; - private final Iterator rightIterator; - private final Function, T> mergeFunction; + private final Iterator leftIterator; + private final Iterator rightIterator; + private final BiFunction mergeFunction; - public MergingIterator(Iterator leftIterator, Iterator rightIterator, Function, T> mergeFunction) { + public MergingIterator(Iterator leftIterator, Iterator rightIterator, BiFunction mergeFunction) { this.leftIterator = leftIterator; this.rightIterator = rightIterator; this.mergeFunction = mergeFunction; @@ -46,7 +45,7 @@ public boolean hasNext() { } @Override - public T next() { - return mergeFunction.apply(Pair.of(leftIterator.next(), rightIterator.next())); + public HoodieRecord next() { + return mergeFunction.apply(leftIterator.next(), rightIterator.next()); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java index a042255cdcb1a..35ca3d6d5adec 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java @@ -20,62 +20,33 @@ package org.apache.hudi.common.table.log; 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.common.util.SpillableMapUtils; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodiePayloadConfig; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import java.io.IOException; import java.util.Iterator; -import java.util.stream.StreamSupport; +import java.util.Properties; /** * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice. */ -public class HoodieFileSliceReader implements Iterator> { +public class HoodieFileSliceReader implements Iterator> { + private final Iterator> recordsIterator; public static HoodieFileSliceReader getFileSliceReader( - Option baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass, - String preCombineField, Option> simpleKeyGenFieldsOpt) throws IOException { + Option baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option> simpleKeyGenFieldsOpt) throws IOException { if (baseFileReader.isPresent()) { - Iterator baseIterator = baseFileReader.get().getRecordIterator(schema); + Iterator baseIterator = baseFileReader.get().getRecordIterator(schema); while (baseIterator.hasNext()) { - GenericRecord record = (GenericRecord) baseIterator.next(); - HoodieRecord hoodieRecord = transform( - record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt); - scanner.processNextRecord(hoodieRecord); + scanner.processNextRecord(baseIterator.next().wrapIntoHoodieRecordPayloadWithParams(schema, props, + simpleKeyGenFieldsOpt, scanner.isWithOperationField(), scanner.getPartitionName(), false)); } - return new HoodieFileSliceReader(scanner.iterator()); - } else { - Iterable> iterable = () -> scanner.iterator(); - HoodiePayloadConfig payloadConfig = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(preCombineField).build(); - return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false) - .map(e -> { - try { - GenericRecord record = (GenericRecord) e.getData().getInsertValue(schema, payloadConfig.getProps()).get(); - return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt); - } catch (IOException io) { - throw new HoodieIOException("Error while creating reader for file slice with no base file.", io); - } - }).iterator()); } - } - - private static HoodieRecord transform( - GenericRecord record, HoodieMergedLogRecordScanner scanner, String payloadClass, - String preCombineField, Option> simpleKeyGenFieldsOpt) { - return simpleKeyGenFieldsOpt.isPresent() - ? SpillableMapUtils.convertToHoodieRecordPayload(record, - payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), scanner.isWithOperationField(), Option.empty()) - : SpillableMapUtils.convertToHoodieRecordPayload(record, - payloadClass, preCombineField, scanner.isWithOperationField(), scanner.getPartitionName()); + return new HoodieFileSliceReader(scanner.iterator()); } private HoodieFileSliceReader(Iterator> recordsItr) { 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 03eb44b001112..6bbcec370eb7b 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 @@ -29,13 +29,16 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieMetastoreConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FileSystemRetryConfig; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.WriteConcurrencyMode; @@ -45,6 +48,7 @@ import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; @@ -130,6 +134,18 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. " + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective"); + public static final ConfigProperty MERGER_IMPLS = ConfigProperty + .key("hoodie.datasource.write.merger.impls") + .defaultValue(HoodieAvroRecordMerger.class.getName()) + .withDocumentation("List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used. " + + "These merger impls will filter by hoodie.datasource.write.merger.strategy " + + "Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)"); + + public static final ConfigProperty MERGER_STRATEGY = ConfigProperty + .key("hoodie.datasource.write.merger.strategy") + .defaultValue(HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID) + .withDocumentation("Id of merger strategy. Hudi will pick HoodieRecordMerger implementations in hoodie.datasource.write.merger.impls which has the same merger strategy id"); + public static final ConfigProperty KEYGENERATOR_CLASS_NAME = ConfigProperty .key("hoodie.datasource.write.keygenerator.class") .noDefaultValue() @@ -542,6 +558,7 @@ public class HoodieWriteConfig extends HoodieConfig { private HoodieMetadataConfig metadataConfig; private HoodieMetastoreConfig metastoreConfig; private HoodieCommonConfig commonConfig; + private HoodieStorageConfig storageConfig; private EngineType engineType; /** @@ -934,6 +951,7 @@ protected HoodieWriteConfig(EngineType engineType, Properties props) { this.metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(props).build(); this.metastoreConfig = HoodieMetastoreConfig.newBuilder().fromProperties(props).build(); this.commonConfig = HoodieCommonConfig.newBuilder().fromProperties(props).build(); + this.storageConfig = HoodieStorageConfig.newBuilder().fromProperties(props).build(); } public static HoodieWriteConfig.Builder newBuilder() { @@ -947,6 +965,15 @@ public String getBasePath() { return getString(BASE_PATH); } + public HoodieRecordMerger getRecordMerger() { + List mergers = getSplitStringsOrDefault(MERGER_IMPLS).stream() + .map(String::trim) + .distinct() + .collect(Collectors.toList()); + String mergerStrategy = getString(MERGER_STRATEGY); + return HoodieRecordUtils.createRecordMerger(getString(BASE_PATH), engineType, mergers, mergerStrategy); + } + public String getSchema() { return getString(AVRO_SCHEMA_STRING); } @@ -955,6 +982,10 @@ public void setSchema(String schemaStr) { setValue(AVRO_SCHEMA_STRING, schemaStr); } + public void setMergerClass(String mergerStrategy) { + setValue(MERGER_STRATEGY, mergerStrategy); + } + /** * Returns schema used for writing records * @@ -2033,6 +2064,10 @@ public HoodieCommonConfig getCommonConfig() { return commonConfig; } + public HoodieStorageConfig getStorageConfig() { + return storageConfig; + } + /** * Commit call back configs. */ @@ -2340,6 +2375,16 @@ public Builder withWritePayLoad(String payload) { return this; } + public Builder withMergerImpls(String mergerImpls) { + writeConfig.setValue(MERGER_IMPLS, mergerImpls); + return this; + } + + public Builder withMergerStrategy(String mergerStrategy) { + writeConfig.setValue(MERGER_STRATEGY, mergerStrategy); + return this; + } + public Builder withKeyGenerator(String keyGeneratorClass) { writeConfig.setValue(KEYGENERATOR_CLASS_NAME, keyGeneratorClass); return this; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java index fffb5e0e775c1..55db97e87a492 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java @@ -19,9 +19,9 @@ package org.apache.hudi.execution; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.HoodieConsumer; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.execution.HoodieLazyInsertIterable.HoodieInsertValueGenResult; @@ -39,7 +39,7 @@ /** * Consumes stream of hoodie records from in-memory queue and writes to one or more create-handles. */ -public class CopyOnWriteInsertHandler +public class CopyOnWriteInsertHandler implements HoodieConsumer, List> { private final HoodieWriteConfig config; @@ -70,9 +70,9 @@ public CopyOnWriteInsertHandler(HoodieWriteConfig config, String instantTime, } @Override - public void consume(HoodieInsertValueGenResult payload) { - final HoodieRecord insertPayload = payload.record; - String partitionPath = insertPayload.getPartitionPath(); + public void consume(HoodieInsertValueGenResult genResult) { + final HoodieRecord record = genResult.getResult(); + String partitionPath = record.getPartitionPath(); HoodieWriteHandle handle = handles.get(partitionPath); if (handle == null) { // If the records are sorted, this means that we encounter a new partition path @@ -83,19 +83,19 @@ public void consume(HoodieInsertValueGenResult payload) { } // Lazily initialize the handle, for the first time handle = writeHandleFactory.create(config, instantTime, hoodieTable, - insertPayload.getPartitionPath(), idPrefix, taskContextSupplier); + record.getPartitionPath(), idPrefix, taskContextSupplier); handles.put(partitionPath, handle); } - if (!handle.canWrite(payload.record)) { + if (!handle.canWrite(genResult.getResult())) { // Handle is full. Close the handle and add the WriteStatus statuses.addAll(handle.close()); // Open new handle handle = writeHandleFactory.create(config, instantTime, hoodieTable, - insertPayload.getPartitionPath(), idPrefix, taskContextSupplier); + record.getPartitionPath(), idPrefix, taskContextSupplier); handles.put(partitionPath, handle); } - handle.write(insertPayload, payload.insertValue, payload.exception); + handle.write(record, genResult.schema, new TypedProperties(genResult.props)); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java index 1754836c91c4a..20f75f63c5234 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java @@ -18,21 +18,17 @@ package org.apache.hudi.execution; +import org.apache.avro.Schema; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.io.WriteHandleFactory; import org.apache.hudi.table.HoodieTable; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; - import java.util.Iterator; import java.util.List; import java.util.Properties; @@ -41,7 +37,7 @@ /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new files. */ -public abstract class HoodieLazyInsertIterable +public abstract class HoodieLazyInsertIterable extends LazyIterableIterator, List> { protected final HoodieWriteConfig hoodieConfig; @@ -78,34 +74,40 @@ public HoodieLazyInsertIterable(Iterator> recordItr, boolean are } // Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread. - public static class HoodieInsertValueGenResult { - public T record; - public Option insertValue; - // It caches the exception seen while fetching insert value. - public Option exception = Option.empty(); + public static class HoodieInsertValueGenResult { + private final R record; + public final Schema schema; + public final Properties props; - public HoodieInsertValueGenResult(T record, Schema schema, Properties properties) { + public HoodieInsertValueGenResult(R record, Schema schema, Properties properties) { this.record = record; - try { - this.insertValue = ((HoodieRecordPayload) record.getData()).getInsertValue(schema, properties); - } catch (Exception e) { - this.exception = Option.of(e); - } + this.schema = schema; + this.props = properties; } + + public R getResult() { + return record; + } + } + + static Function, HoodieInsertValueGenResult> getCloningTransformer(Schema schema, + HoodieWriteConfig config) { + return getCloningTransformerInternal(schema, config.getProps()); } - /** - * Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some - * expensive operations of transformation to the reader thread. - */ - static Function, HoodieInsertValueGenResult> getTransformFunction( - Schema schema, HoodieWriteConfig config) { - return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema, config.getProps()); + static Function, HoodieInsertValueGenResult> getCloningTransformer(Schema schema) { + return getCloningTransformerInternal(schema, new TypedProperties()); } - static Function, HoodieInsertValueGenResult> getTransformFunction( - Schema schema) { - return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema, CollectionUtils.emptyProps()); + private static Function, HoodieInsertValueGenResult> getCloningTransformerInternal(Schema schema, + TypedProperties props) { + return record -> { + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be subsequently buffered (w/in the in-memory queue) + HoodieRecord clonedRecord = record.copy(); + return new HoodieInsertValueGenResult(clonedRecord, schema, props); + }; } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java index 6bbea356e51d4..8a8a03e1b17ac 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; @@ -150,7 +151,8 @@ public static List filterKeysFromFile(Path filePath, List candid Configuration configuration) throws HoodieIndexException { ValidationUtils.checkArgument(FSUtils.isBaseFile(filePath)); List foundRecordKeys = new ArrayList<>(); - try (HoodieFileReader fileReader = HoodieFileReaderFactory.getFileReader(configuration, filePath)) { + try (HoodieFileReader fileReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO) + .getFileReader(configuration, filePath)) { // Load all rowKeys from the file, to double-confirm if (!candidateRecordKeys.isEmpty()) { HoodieTimer timer = HoodieTimer.start(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java index 42dcc1b97d760..8b7551361729f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; @@ -122,4 +123,11 @@ public boolean canIndexLogFiles() { public boolean isImplicitWithStorage() { return false; } + + @VisibleForTesting + public static void clear() { + if (recordLocationMap != null) { + recordLocationMap.clear(); + } + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java index b4c83c141b2bc..fdd232b55a0aa 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java @@ -19,11 +19,10 @@ package org.apache.hudi.io; import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -public class AppendHandleFactory extends WriteHandleFactory { +public class AppendHandleFactory extends WriteHandleFactory { @Override public HoodieAppendHandle create(final HoodieWriteConfig hoodieConfig, final String commitTime, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java index 09131b421f402..8dc19816fd133 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java @@ -19,13 +19,12 @@ package org.apache.hudi.io; import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import java.io.Serializable; -public class CreateHandleFactory extends WriteHandleFactory implements Serializable { +public class CreateHandleFactory extends WriteHandleFactory implements Serializable { private boolean preserveMetadata = false; 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 3a1931554330f..02053127d9c83 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 @@ -19,11 +19,10 @@ package org.apache.hudi.io; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BaseFile; @@ -36,10 +35,11 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodiePayloadProps; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.MetadataValues; import org.apache.hudi.common.table.log.AppendResult; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; @@ -67,6 +67,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Properties; @@ -79,7 +80,7 @@ /** * IO Operation to append data onto an existing file. */ -public class HoodieAppendHandle extends HoodieWriteHandle { +public class HoodieAppendHandle extends HoodieWriteHandle { private static final Logger LOG = LogManager.getLogger(HoodieAppendHandle.class); // This acts as the sequenceID for records written @@ -88,7 +89,7 @@ public class HoodieAppendHandle extends protected final String fileId; // Buffer for holding records in memory before they are flushed to disk - private final List recordList = new ArrayList<>(); + private final List recordList = new ArrayList<>(); // Buffer for holding records (to be deleted) in memory before they are flushed to disk private final List recordsToDelete = new ArrayList<>(); // Incoming records to be written to logs. @@ -145,7 +146,7 @@ public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTa super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); this.fileId = fileId; this.recordItr = recordItr; - sizeEstimator = new DefaultSizeEstimator(); + this.sizeEstimator = new DefaultSizeEstimator(); this.statuses = new ArrayList<>(); this.recordProperties.putAll(config.getProps()); } @@ -225,8 +226,9 @@ protected boolean isUpdateRecord(HoodieRecord hoodieRecord) { return hoodieRecord.getCurrentLocation() != null; } - private Option getIndexedRecord(HoodieRecord hoodieRecord) { - Option> recordMetadata = hoodieRecord.getData().getMetadata(); + private Option prepareRecord(HoodieRecord hoodieRecord) { + Option> recordMetadata = hoodieRecord.getMetadata(); + Schema schema = useWriterSchema ? writeSchemaWithMetaFields : writeSchema; try { // Pass the isUpdateRecord to the props for HoodieRecordPayload to judge // Whether it is an update or insert record. @@ -234,26 +236,21 @@ private Option getIndexedRecord(HoodieRecord hoodieRecord) { // If the format can not record the operation field, nullify the DELETE payload manually. boolean nullifyPayload = HoodieOperation.isDelete(hoodieRecord.getOperation()) && !config.allowOperationMetadataField(); recordProperties.put(HoodiePayloadProps.PAYLOAD_IS_UPDATE_RECORD_FOR_MOR, String.valueOf(isUpdateRecord)); - Option avroRecord = nullifyPayload ? Option.empty() : getInsertValue(hoodieRecord); - if (avroRecord.isPresent()) { - if (avroRecord.get().equals(IGNORE_RECORD)) { - return avroRecord; + Option finalRecord = nullifyPayload ? Option.empty() : Option.of(hoodieRecord); + // Check for delete + if (finalRecord.isPresent() && !finalRecord.get().isDelete(schema, recordProperties)) { + // Check for ignore ExpressionPayload + if (finalRecord.get().shouldIgnore(schema, recordProperties)) { + return finalRecord; } // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema - GenericRecord rewriteRecord = rewriteRecord((GenericRecord) avroRecord.get()); - avroRecord = Option.of(rewriteRecord); - String seqId = - HoodieRecord.generateSequenceId(instantTime, getPartitionId(), RECORD_COUNTER.getAndIncrement()); - if (config.populateMetaFields()) { - HoodieAvroUtils.addHoodieKeyToRecord(rewriteRecord, hoodieRecord.getRecordKey(), - hoodieRecord.getPartitionPath(), fileId); - if (!this.isLogCompaction) { - HoodieAvroUtils.addCommitMetadataToRecord(rewriteRecord, instantTime, seqId); - } - } - if (config.allowOperationMetadataField()) { - HoodieAvroUtils.addOperationToRecord(rewriteRecord, hoodieRecord.getOperation()); - } + HoodieRecord rewrittenRecord = schemaOnReadEnabled ? finalRecord.get().rewriteRecordWithNewSchema(schema, recordProperties, writeSchemaWithMetaFields) + : finalRecord.get().rewriteRecord(schema, recordProperties, writeSchemaWithMetaFields); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into the recordList(List). + HoodieRecord populatedRecord = populateMetadataFields(rewrittenRecord.copy(), writeSchemaWithMetaFields, recordProperties); + finalRecord = Option.of(populatedRecord); if (isUpdateRecord || isLogCompaction) { updatedRecordsWritten++; } else { @@ -261,6 +258,7 @@ private Option getIndexedRecord(HoodieRecord hoodieRecord) { } recordsWritten++; } else { + finalRecord = Option.empty(); recordsDeleted++; } @@ -269,7 +267,7 @@ private Option getIndexedRecord(HoodieRecord hoodieRecord) { // part of marking // record successful. hoodieRecord.deflate(); - return avroRecord; + return finalRecord; } catch (Exception e) { LOG.error("Error writing record " + hoodieRecord, e); writeStatus.markFailure(hoodieRecord, e, recordMetadata); @@ -277,12 +275,24 @@ private Option getIndexedRecord(HoodieRecord hoodieRecord) { return Option.empty(); } - private Option getInsertValue(HoodieRecord hoodieRecord) throws IOException { - if (useWriterSchema) { - return hoodieRecord.getData().getInsertValue(writeSchemaWithMetaFields, recordProperties); - } else { - return hoodieRecord.getData().getInsertValue(writeSchema, recordProperties); + private HoodieRecord populateMetadataFields(HoodieRecord hoodieRecord, Schema schema, Properties prop) throws IOException { + MetadataValues metadataValues = new MetadataValues(); + if (config.populateMetaFields()) { + String seqId = + HoodieRecord.generateSequenceId(instantTime, getPartitionId(), RECORD_COUNTER.getAndIncrement()); + metadataValues.setFileName(fileId); + metadataValues.setPartitionPath(partitionPath); + metadataValues.setRecordKey(hoodieRecord.getRecordKey()); + if (!this.isLogCompaction) { + metadataValues.setCommitTime(instantTime); + metadataValues.setCommitSeqno(seqId); + } + } + if (config.allowOperationMetadataField()) { + metadataValues.setOperation(hoodieRecord.getOperation().getName()); } + + return hoodieRecord.updateMetadataValues(schema, prop, metadataValues); } private void initNewStatus() { @@ -360,7 +370,7 @@ private void updateWriteStatus(HoodieDeltaWriteStat stat, AppendResult result) { statuses.add(this.writeStatus); } - private void processAppendResult(AppendResult result, List recordList) { + private void processAppendResult(AppendResult result, List recordList) throws IOException { HoodieDeltaWriteStat stat = (HoodieDeltaWriteStat) this.writeStatus.getStat(); if (stat.getPath() == null) { @@ -379,7 +389,9 @@ private void processAppendResult(AppendResult result, List record updateWriteStatus(stat, result); } - if (config.isMetadataColumnStatsIndexEnabled()) { + // TODO MetadataColumnStatsIndex for spark record + // https://issues.apache.org/jira/browse/HUDI-5249 + if (config.isMetadataColumnStatsIndexEnabled() && recordMerger.getRecordType() == HoodieRecordType.AVRO) { final List fieldsToIndex; // If column stats index is enabled but columns not configured then we assume that // all columns should be indexed @@ -393,8 +405,13 @@ private void processAppendResult(AppendResult result, List record .collect(Collectors.toList()); } + List indexedRecords = new LinkedList<>(); + for (HoodieRecord hoodieRecord : recordList) { + indexedRecords.add(hoodieRecord.toIndexedRecord(writeSchema, config.getProps()).get().getData()); + } + Map> columnRangesMetadataMap = - collectColumnRangeMetadata(recordList, fieldsToIndex, stat.getPath()); + collectColumnRangeMetadata(indexedRecords, fieldsToIndex, stat.getPath()); stat.putRecordsStats(columnRangesMetadataMap); } @@ -459,8 +476,8 @@ public boolean canWrite(HoodieRecord record) { } @Override - public void write(HoodieRecord record, Option insertValue) { - Option> recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata(); + protected void doWrite(HoodieRecord record, Schema schema, TypedProperties props) { + Option> recordMetadata = record.getMetadata(); try { init(record); flushToDiskIfRequired(record, false); @@ -497,10 +514,10 @@ public List close() { } } - public void write(Map> recordMap) { + public void write(Map> recordMap) { try { - for (Map.Entry> entry: recordMap.entrySet()) { - HoodieRecord record = (HoodieRecord) entry.getValue(); + for (Map.Entry> entry: recordMap.entrySet()) { + HoodieRecord record = entry.getValue(); init(record); flushToDiskIfRequired(record, false); writeToBuffer(record); @@ -532,7 +549,7 @@ private void writeToBuffer(HoodieRecord record) { if (!partitionPath.equals(record.getPartitionPath())) { HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: " + record.getPartitionPath() + " but trying to insert into partition: " + partitionPath); - writeStatus.markFailure(record, failureEx, record.getData().getMetadata()); + writeStatus.markFailure(record, failureEx, record.getMetadata()); return; } @@ -543,12 +560,17 @@ private void writeToBuffer(HoodieRecord record) { record.seal(); } // fetch the ordering val first in case the record was deflated. - final Comparable orderingVal = record.getData().getOrderingValue(); - Option indexedRecord = getIndexedRecord(record); + final Comparable orderingVal = record.getOrderingValue(writeSchema, recordProperties); + Option indexedRecord = prepareRecord(record); if (indexedRecord.isPresent()) { // Skip the ignored record. - if (!indexedRecord.get().equals(IGNORE_RECORD)) { - recordList.add(indexedRecord.get()); + try { + if (!indexedRecord.get().shouldIgnore(writeSchema, recordProperties)) { + recordList.add(indexedRecord.get()); + } + } catch (IOException e) { + writeStatus.markFailure(record, e, record.getMetadata()); + LOG.error("Error writing record " + indexedRecord.get(), e); } } else { recordsToDelete.add(DeleteRecord.create(record.getKey(), orderingVal)); @@ -560,7 +582,7 @@ private void writeToBuffer(HoodieRecord record) { * Checks if the number of records have reached the set threshold and then flushes the records to disk. */ private void flushToDiskIfRequired(HoodieRecord record, boolean appendDeleteBlocks) { - if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize) + if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize) || numberOfRecords % NUMBER_OF_RECORDS_TO_ESTIMATE_RECORD_SIZE == 0) { averageRecordSize = (long) (averageRecordSize * 0.8 + sizeEstimator.sizeEstimate(record) * 0.2); } @@ -598,18 +620,18 @@ private HoodieLogBlock.HoodieLogBlockType pickLogDataBlockFormat() { private static HoodieLogBlock getBlock(HoodieWriteConfig writeConfig, HoodieLogBlock.HoodieLogBlockType logDataBlockFormat, - List recordList, + List records, Map header, String keyField) { switch (logDataBlockFormat) { case AVRO_DATA_BLOCK: - return new HoodieAvroDataBlock(recordList, header, keyField); + return new HoodieAvroDataBlock(records, header, keyField); case HFILE_DATA_BLOCK: return new HoodieHFileDataBlock( - recordList, header, writeConfig.getHFileCompressionAlgorithm(), new Path(writeConfig.getBasePath())); + records, header, writeConfig.getHFileCompressionAlgorithm(), new Path(writeConfig.getBasePath())); case PARQUET_DATA_BLOCK: return new HoodieParquetDataBlock( - recordList, + records, header, keyField, writeConfig.getParquetCompressionCodec(), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java index 8e7f66467a6d0..f110bf585dbe3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java @@ -21,7 +21,6 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.engine.TaskContextSupplier; 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.table.HoodieTable; @@ -33,7 +32,7 @@ * writing more than 1 skeleton file for the same bootstrap file. * @param HoodieRecordPayload */ -public class HoodieBootstrapHandle extends HoodieCreateHandle { +public class HoodieBootstrapHandle extends HoodieCreateHandle { public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java index 1b62eca76142c..be646df85be4d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableConfig; @@ -179,10 +180,10 @@ public void put(HoodieRecord hoodieRecord, private void flushIfNeeded(Boolean force) { if (force || numOfCDCRecordsInMemory.get() * averageCDCRecordSize >= maxBlockSize) { try { - List records = cdcData.values().stream() + List records = cdcData.values().stream() .map(record -> { try { - return record.getInsertValue(cdcSchema).get(); + return new HoodieAvroIndexedRecord(record.getInsertValue(cdcSchema).get()); } catch (IOException e) { throw new HoodieIOException("Failed to get cdc record", e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java index ca245e0c391ba..feae0f741f0d9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java @@ -23,15 +23,13 @@ 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.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.keygen.KeyGenUtils; import org.apache.hudi.table.HoodieTable; -import org.apache.avro.generic.GenericRecord; +import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -69,7 +67,7 @@ * happen and every batch should have new records to be inserted. Above example is for illustration purposes only. */ @NotThreadSafe -public class HoodieConcatHandle extends HoodieMergeHandle { +public class HoodieConcatHandle extends HoodieMergeHandle { private static final Logger LOG = LogManager.getLogger(HoodieConcatHandle.class); // a representation of incoming records that tolerates duplicate keys @@ -94,11 +92,12 @@ public HoodieConcatHandle(HoodieWriteConfig config, String instantTime, HoodieTa * Write old record as is w/o merging with incoming record. */ @Override - public void write(GenericRecord oldRecord) { - String key = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt); + public void write(HoodieRecord oldRecord) { + Schema oldSchema = config.populateMetaFields() ? writeSchemaWithMetaFields : writeSchema; + String key = oldRecord.getRecordKey(oldSchema, keyGeneratorOpt); try { // NOTE: We're enforcing preservation of the record metadata to keep existing semantic - writeToFile(new HoodieKey(key, partitionPath), oldRecord, true); + writeToFile(new HoodieKey(key, partitionPath), oldRecord, oldSchema, config.getPayloadConfig().getProps(), true); } catch (IOException | RuntimeException e) { String errMsg = String.format("Failed to write old record into new file for key %s from old file %s to new file %s with writerSchema %s", key, getOldFilePath(), newFilePath, writeSchemaWithMetaFields.toString(true)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 2e171e1756043..35fdc247e7e50 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -19,20 +19,19 @@ package org.apache.hudi.io; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.MetadataValues; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieInsertException; @@ -51,11 +50,11 @@ import java.util.Map; @NotThreadSafe -public class HoodieCreateHandle extends HoodieWriteHandle { +public class HoodieCreateHandle extends HoodieWriteHandle { private static final Logger LOG = LogManager.getLogger(HoodieCreateHandle.class); - protected final HoodieFileWriter fileWriter; + protected final HoodieFileWriter fileWriter; protected final Path path; protected long recordsWritten = 0; protected long insertRecordsWritten = 0; @@ -101,8 +100,8 @@ public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTa hoodieTable.getPartitionMetafileFormat()); partitionMetadata.trySave(getPartitionId()); createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension())); - this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, - writeSchemaWithMetaFields, this.taskContextSupplier); + this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable.getHadoopConf(), config, + writeSchemaWithMetaFields, this.taskContextSupplier, config.getRecordMerger().getRecordType()); } catch (IOException e) { throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e); } @@ -130,22 +129,26 @@ public boolean canWrite(HoodieRecord record) { * Perform the actual writing of the given record into the backing file. */ @Override - public void write(HoodieRecord record, Option avroRecord) { - Option recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata(); - if (HoodieOperation.isDelete(record.getOperation())) { - avroRecord = Option.empty(); - } + protected void doWrite(HoodieRecord record, Schema schema, TypedProperties props) { + Option> recordMetadata = record.getMetadata(); try { - if (avroRecord.isPresent()) { - if (avroRecord.get().equals(IGNORE_RECORD)) { + if (!HoodieOperation.isDelete(record.getOperation()) && !record.isDelete(schema, config.getProps())) { + if (record.shouldIgnore(schema, config.getProps())) { return; } // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema + HoodieRecord rewriteRecord; + if (schemaOnReadEnabled) { + rewriteRecord = record.rewriteRecordWithNewSchema(schema, config.getProps(), writeSchemaWithMetaFields); + } else { + rewriteRecord = record.rewriteRecord(schema, config.getProps(), writeSchemaWithMetaFields); + } + MetadataValues metadataValues = new MetadataValues().setFileName(path.getName()); + rewriteRecord = rewriteRecord.updateMetadataValues(writeSchemaWithMetaFields, config.getProps(), metadataValues); if (preserveMetadata) { - fileWriter.writeAvro(record.getRecordKey(), - rewriteRecordWithMetadata((GenericRecord) avroRecord.get(), path.getName())); + fileWriter.write(record.getRecordKey(), rewriteRecord, writeSchemaWithMetaFields); } else { - fileWriter.writeAvroWithMetadata(record.getKey(), rewriteRecord((GenericRecord) avroRecord.get())); + fileWriter.writeWithMetadata(record.getKey(), rewriteRecord, writeSchemaWithMetaFields); } // update the new location of record, so we know where to find it next record.unseal(); @@ -180,18 +183,10 @@ public void write() { } else { keyIterator = recordMap.keySet().stream().iterator(); } - try { - while (keyIterator.hasNext()) { - final String key = keyIterator.next(); - HoodieRecord record = recordMap.get(key); - if (useWriterSchema) { - write(record, record.getData().getInsertValue(writeSchemaWithMetaFields, config.getProps())); - } else { - write(record, record.getData().getInsertValue(writeSchema, config.getProps())); - } - } - } catch (IOException io) { - throw new HoodieInsertException("Failed to insert records for path " + path, io); + while (keyIterator.hasNext()) { + final String key = keyIterator.next(); + HoodieRecord record = recordMap.get(key); + write(record, useWriterSchema ? writeSchemaWithMetaFields : writeSchema, config.getProps()); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java index 1ad28d14b3a8d..a8b7965d8007a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java @@ -18,7 +18,6 @@ package org.apache.hudi.io; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; @@ -26,7 +25,7 @@ import org.apache.hadoop.fs.FileSystem; -public abstract class HoodieIOHandle { +public abstract class HoodieIOHandle { protected final String instantTime; protected final HoodieWriteConfig config; 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 ab8b83c14aeec..f0625303687ee 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 @@ -21,7 +21,6 @@ import org.apache.hudi.common.model.HoodieBaseFile; 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; @@ -40,7 +39,7 @@ * * @param */ -public class HoodieKeyLocationFetchHandle extends HoodieReadHandle { +public class HoodieKeyLocationFetchHandle extends HoodieReadHandle { private final Pair partitionPathBaseFilePair; private final Option keyGeneratorOpt; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java index df629b83dcaa3..f7c5fb8a8781f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java @@ -18,9 +18,9 @@ package org.apache.hudi.io; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -28,8 +28,6 @@ import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.table.HoodieTable; - -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -42,7 +40,7 @@ /** * Takes a bunch of keys and returns ones that are present in the file group. */ -public class HoodieKeyLookupHandle extends HoodieReadHandle { +public class HoodieKeyLookupHandle extends HoodieReadHandle { private static final Logger LOG = LogManager.getLogger(HoodieKeyLookupHandle.class); 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 97cade66e1859..b6364aede14ca 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 @@ -19,6 +19,7 @@ package org.apache.hudi.io; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; @@ -27,15 +28,16 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.MetadataValues; 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.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.hudi.exception.HoodieIOException; @@ -45,12 +47,9 @@ import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.io.storage.HoodieFileWriterFactory; import org.apache.hudi.keygen.BaseKeyGenerator; -import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -64,6 +63,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Properties; import java.util.Set; @SuppressWarnings("Duplicates") @@ -96,13 +96,13 @@ *

*/ @NotThreadSafe -public class HoodieMergeHandle extends HoodieWriteHandle { +public class HoodieMergeHandle extends HoodieWriteHandle { private static final Logger LOG = LogManager.getLogger(HoodieMergeHandle.class); protected Map> keyToNewRecords; protected Set writtenRecordKeys; - protected HoodieFileWriter fileWriter; + protected HoodieFileWriter fileWriter; private boolean preserveMetadata = false; protected Path newFilePath; @@ -158,15 +158,6 @@ public static HoodieBaseFile getLatestBaseFile(HoodieTable hoodieTab return baseFileOp.get(); } - @Override - public Schema getWriterSchemaWithMetaFields() { - return writeSchemaWithMetaFields; - } - - public Schema getWriterSchema() { - return writeSchema; - } - /** * Extract old file path, initialize StorageWriter and WriteStatus. */ @@ -202,8 +193,8 @@ private void init(String fileId, String partitionPath, HoodieBaseFile baseFileTo createMarkerFile(partitionPath, newFilePath.getName()); // Create the writer for writing the new version file - fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, newFilePath, hoodieTable, - config, writeSchemaWithMetaFields, taskContextSupplier); + fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, newFilePath, hoodieTable.getHadoopConf(), + config, writeSchemaWithMetaFields, taskContextSupplier, recordMerger.getRecordType()); } catch (IOException io) { LOG.error("Error in update task at commit " + instantTime, io); writeStatus.setGlobalError(io); @@ -269,66 +260,65 @@ protected void init(String fileId, Iterator> newRecordsItr) { + ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); } - protected boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option indexedRecord) { + protected boolean writeUpdateRecord(HoodieRecord newRecord, HoodieRecord oldRecord, Option combineRecordOpt, Schema writerSchema) throws IOException { boolean isDelete = false; - if (indexedRecord.isPresent()) { + if (combineRecordOpt.isPresent()) { updatedRecordsWritten++; - GenericRecord record = (GenericRecord) indexedRecord.get(); - if (oldRecord != record) { + if (oldRecord.getData() != combineRecordOpt.get().getData()) { // the incoming record is chosen - isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation()); + isDelete = HoodieOperation.isDelete(newRecord.getOperation()); } else { // the incoming record is dropped return false; } } - return writeRecord(hoodieRecord, indexedRecord, isDelete); + return writeRecord(newRecord, combineRecordOpt, writerSchema, config.getPayloadConfig().getProps(), isDelete); } - protected void writeInsertRecord(HoodieRecord hoodieRecord) throws IOException { + protected void writeInsertRecord(HoodieRecord newRecord) throws IOException { Schema schema = useWriterSchemaForCompaction ? writeSchemaWithMetaFields : writeSchema; - Option insertRecord = hoodieRecord.getData().getInsertValue(schema, config.getProps()); // just skip the ignored record - if (insertRecord.isPresent() && insertRecord.get().equals(IGNORE_RECORD)) { + if (newRecord.shouldIgnore(schema, config.getProps())) { return; } - writeInsertRecord(hoodieRecord, insertRecord); + writeInsertRecord(newRecord, schema, config.getProps()); } - protected void writeInsertRecord(HoodieRecord hoodieRecord, Option insertRecord) { - if (writeRecord(hoodieRecord, insertRecord, HoodieOperation.isDelete(hoodieRecord.getOperation()))) { + protected void writeInsertRecord(HoodieRecord newRecord, Schema schema, Properties prop) + throws IOException { + if (writeRecord(newRecord, Option.of(newRecord), schema, prop, HoodieOperation.isDelete(newRecord.getOperation()))) { insertRecordsWritten++; } } - protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord) { - return writeRecord(hoodieRecord, indexedRecord, false); + protected boolean writeRecord(HoodieRecord newRecord, Option combineRecord, Schema schema, Properties prop) throws IOException { + return writeRecord(newRecord, combineRecord, schema, prop, false); } - private boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord, boolean isDelete) { - Option recordMetadata = hoodieRecord.getData().getMetadata(); - if (!partitionPath.equals(hoodieRecord.getPartitionPath())) { + private boolean writeRecord(HoodieRecord newRecord, Option combineRecord, Schema schema, Properties prop, boolean isDelete) throws IOException { + Option recordMetadata = newRecord.getMetadata(); + if (!partitionPath.equals(newRecord.getPartitionPath())) { HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: " - + hoodieRecord.getPartitionPath() + " but trying to insert into partition: " + partitionPath); - writeStatus.markFailure(hoodieRecord, failureEx, recordMetadata); + + newRecord.getPartitionPath() + " but trying to insert into partition: " + partitionPath); + writeStatus.markFailure(newRecord, failureEx, recordMetadata); return false; } try { - if (indexedRecord.isPresent() && !isDelete) { - writeToFile(hoodieRecord.getKey(), (GenericRecord) indexedRecord.get(), preserveMetadata && useWriterSchemaForCompaction); + if (combineRecord.isPresent() && !combineRecord.get().isDelete(schema, config.getProps()) && !isDelete) { + writeToFile(newRecord.getKey(), combineRecord.get(), schema, prop, preserveMetadata && useWriterSchemaForCompaction); recordsWritten++; } else { recordsDeleted++; } - writeStatus.markSuccess(hoodieRecord, recordMetadata); + writeStatus.markSuccess(newRecord, recordMetadata); // deflate record payload after recording success. This will help users access payload as a // part of marking // record successful. - hoodieRecord.deflate(); + newRecord.deflate(); return true; } catch (Exception e) { - LOG.error("Error writing record " + hoodieRecord, e); - writeStatus.markFailure(hoodieRecord, e, recordMetadata); + LOG.error("Error writing record " + newRecord, e); + writeStatus.markFailure(newRecord, e, recordMetadata); } return false; } @@ -336,23 +326,24 @@ private boolean writeRecord(HoodieRecord hoodieRecord, Option /** * 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 = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt); + public void write(HoodieRecord oldRecord) { + Schema oldSchema = config.populateMetaFields() ? writeSchemaWithMetaFields : writeSchema; + Schema newSchema = useWriterSchemaForCompaction ? writeSchemaWithMetaFields : writeSchema; boolean copyOldRecord = true; + String key = oldRecord.getRecordKey(oldSchema, keyGeneratorOpt); + TypedProperties props = config.getPayloadConfig().getProps(); if (keyToNewRecords.containsKey(key)) { // If we have duplicate records that we are updating, then the hoodie record will be deflated after // writing the first record. So make a copy of the record to be merged - HoodieRecord hoodieRecord = keyToNewRecords.get(key).newInstance(); + HoodieRecord newRecord = keyToNewRecords.get(key).newInstance(); try { - Option combinedAvroRecord = - hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, - useWriterSchemaForCompaction ? writeSchemaWithMetaFields : writeSchema, - config.getPayloadConfig().getProps()); - - if (combinedAvroRecord.isPresent() && combinedAvroRecord.get().equals(IGNORE_RECORD)) { + Option> mergeResult = recordMerger.merge(oldRecord, oldSchema, newRecord, newSchema, props); + Schema combineRecordSchema = mergeResult.map(Pair::getRight).orElse(null); + Option combinedRecord = mergeResult.map(Pair::getLeft); + if (combinedRecord.isPresent() && combinedRecord.get().shouldIgnore(combineRecordSchema, props)) { // If it is an IGNORE_RECORD, just copy the old record, and do not update the new record. copyOldRecord = true; - } else if (writeUpdateRecord(hoodieRecord, oldRecord, combinedAvroRecord)) { + } else if (writeUpdateRecord(newRecord, oldRecord, combinedRecord, combineRecordSchema)) { /* * ONLY WHEN 1) we have an update for this key AND 2) We are able to successfully * write the combined new value @@ -371,7 +362,7 @@ public void write(GenericRecord oldRecord) { if (copyOldRecord) { try { // NOTE: We're enforcing preservation of the record metadata to keep existing semantic - writeToFile(new HoodieKey(key, partitionPath), oldRecord, true); + writeToFile(new HoodieKey(key, partitionPath), oldRecord, oldSchema, props, true); } catch (IOException | RuntimeException e) { String errMsg = String.format("Failed to merge old record into new file for key %s from old file %s to new file %s with writerSchema %s", key, getOldFilePath(), newFilePath, writeSchemaWithMetaFields.toString(true)); @@ -382,13 +373,21 @@ public void write(GenericRecord oldRecord) { } } - protected void writeToFile(HoodieKey key, GenericRecord avroRecord, boolean shouldPreserveRecordMetadata) throws IOException { + protected void writeToFile(HoodieKey key, HoodieRecord record, Schema schema, Properties prop, boolean shouldPreserveRecordMetadata) throws IOException { + HoodieRecord rewriteRecord; + if (schemaOnReadEnabled) { + rewriteRecord = record.rewriteRecordWithNewSchema(schema, prop, writeSchemaWithMetaFields); + } else { + rewriteRecord = record.rewriteRecord(schema, prop, writeSchemaWithMetaFields); + } + // NOTE: `FILENAME_METADATA_FIELD` has to be rewritten to correctly point to the + // file holding this record even in cases when overall metadata is preserved + MetadataValues metadataValues = new MetadataValues().setFileName(newFilePath.getName()); + rewriteRecord = rewriteRecord.updateMetadataValues(writeSchemaWithMetaFields, prop, metadataValues); if (shouldPreserveRecordMetadata) { - // NOTE: `FILENAME_METADATA_FIELD` has to be rewritten to correctly point to the - // file holding this record even in cases when overall metadata is preserved - fileWriter.writeAvro(key.getRecordKey(), rewriteRecordWithMetadata(avroRecord, newFilePath.getName())); + fileWriter.write(key.getRecordKey(), rewriteRecord, writeSchemaWithMetaFields); } else { - fileWriter.writeAvroWithMetadata(key, rewriteRecord(avroRecord)); + fileWriter.writeWithMetadata(key, rewriteRecord, writeSchemaWithMetaFields); } } @@ -452,7 +451,7 @@ public void performMergeDataValidationCheck(WriteStatus writeStatus) { } long oldNumWrites = 0; - try (HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(hoodieTable.getHadoopConf(), oldFilePath)) { + try (HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType()).getFileReader(hoodieTable.getHadoopConf(), oldFilePath)) { oldNumWrites = reader.getTotalRecords(); } catch (IOException e) { throw new HoodieUpsertException("Failed to check for merge data validation", e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java index 436eff5dac54d..b110c2c081782 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java @@ -21,7 +21,6 @@ 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.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -38,7 +37,7 @@ public class HoodieMergeHandleFactory { /** * Creates a merge handle for normal write path. */ - public static HoodieMergeHandle create( + public static HoodieMergeHandle create( WriteOperationType operationType, HoodieWriteConfig writeConfig, String instantTime, @@ -70,7 +69,7 @@ public static HoodieMergeHandle HoodieMergeHandle create( + public static HoodieMergeHandle create( HoodieWriteConfig writeConfig, String instantTime, HoodieTable table, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleWithChangeLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleWithChangeLog.java index a19b87532635a..cd61f428ae4c0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleWithChangeLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleWithChangeLog.java @@ -18,22 +18,25 @@ package org.apache.hudi.io; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.cdc.HoodieCDCUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; - +import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -41,7 +44,10 @@ /** * A merge handle that supports logging change logs. */ -public class HoodieMergeHandleWithChangeLog extends HoodieMergeHandle { +public class HoodieMergeHandleWithChangeLog extends HoodieMergeHandle { + + private static final Logger LOG = LogManager.getLogger(HoodieMergeHandleWithChangeLog.class); + protected final HoodieCDCLogger cdcLogger; public HoodieMergeHandleWithChangeLog(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, @@ -77,19 +83,27 @@ public HoodieMergeHandleWithChangeLog(HoodieWriteConfig config, String instantTi IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config)); } - protected boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option indexedRecord) { - final boolean result = super.writeUpdateRecord(hoodieRecord, oldRecord, indexedRecord); + protected boolean writeUpdateRecord(HoodieRecord newRecord, HoodieRecord oldRecord, Option combinedRecordOpt, Schema writerSchema) + throws IOException { + // TODO [HUDI-5019] Remove these unnecessary newInstance invocations + Option savedCombineRecordOp = combinedRecordOpt.map(HoodieRecord::newInstance); + final boolean result = super.writeUpdateRecord(newRecord, oldRecord, combinedRecordOpt, writerSchema); if (result) { - boolean isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation()); - cdcLogger.put(hoodieRecord, oldRecord, isDelete ? Option.empty() : indexedRecord); + boolean isDelete = HoodieOperation.isDelete(newRecord.getOperation()); + Option avroRecordOpt = savedCombineRecordOp.flatMap(r -> + toAvroRecord(r, writerSchema, config.getPayloadConfig().getProps())); + cdcLogger.put(newRecord, (GenericRecord) oldRecord.getData(), isDelete ? Option.empty() : avroRecordOpt); } return result; } - protected void writeInsertRecord(HoodieRecord hoodieRecord, Option insertRecord) { - super.writeInsertRecord(hoodieRecord, insertRecord); - if (!HoodieOperation.isDelete(hoodieRecord.getOperation())) { - cdcLogger.put(hoodieRecord, null, insertRecord); + protected void writeInsertRecord(HoodieRecord newRecord) throws IOException { + Schema schema = useWriterSchemaForCompaction ? writeSchemaWithMetaFields : writeSchema; + // TODO Remove these unnecessary newInstance invocations + HoodieRecord savedRecord = newRecord.newInstance(); + super.writeInsertRecord(newRecord); + if (!HoodieOperation.isDelete(newRecord.getOperation())) { + cdcLogger.put(newRecord, null, savedRecord.toIndexedRecord(schema, config.getPayloadConfig().getProps()).map(HoodieAvroIndexedRecord::getData)); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java index 4440733508142..282a59466406e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java @@ -19,7 +19,6 @@ package org.apache.hudi.io; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.storage.HoodieFileReader; @@ -30,7 +29,7 @@ /** * Extract range information for a given file slice. */ -public class HoodieRangeInfoHandle extends HoodieReadHandle { +public class HoodieRangeInfoHandle extends HoodieReadHandle { public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable hoodieTable, Pair partitionPathFilePair) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java index 223241bc373d0..ebe6361fdf55f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.HoodieBaseFile; -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; @@ -34,7 +33,7 @@ /** * Base class for read operations done logically on the file group. */ -public abstract class HoodieReadHandle extends HoodieIOHandle { +public abstract class HoodieReadHandle extends HoodieIOHandle { protected final Pair partitionPathFileIDPair; @@ -63,12 +62,12 @@ protected HoodieBaseFile getLatestDataFile() { } protected HoodieFileReader createNewFileReader() throws IOException { - return HoodieFileReaderFactory.getFileReader(hoodieTable.getHadoopConf(), + return HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType()).getFileReader(hoodieTable.getHadoopConf(), new Path(getLatestDataFile().getPath())); } protected HoodieFileReader createNewFileReader(HoodieBaseFile hoodieBaseFile) throws IOException { - return HoodieFileReaderFactory.getFileReader(hoodieTable.getHadoopConf(), + return HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType()).getFileReader(hoodieTable.getHadoopConf(), new Path(hoodieBaseFile.getPath())); } } 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 45f262ba886b4..18fe6a344db98 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 @@ -18,20 +18,17 @@ package org.apache.hudi.io; +import org.apache.avro.Schema; import org.apache.hudi.client.WriteStatus; 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.keygen.KeyGenUtils; import org.apache.hudi.table.HoodieTable; -import org.apache.avro.generic.GenericRecord; - import javax.annotation.concurrent.NotThreadSafe; import java.io.IOException; @@ -48,7 +45,7 @@ * keys in newRecordKeys (sorted in-memory). */ @NotThreadSafe -public class HoodieSortedMergeHandle extends HoodieMergeHandle { +public class HoodieSortedMergeHandle extends HoodieMergeHandle { private final Queue newRecordKeysSorted = new PriorityQueue<>(); @@ -75,8 +72,10 @@ public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, Hoo * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file. */ @Override - public void write(GenericRecord oldRecord) { - String key = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt); + public void write(HoodieRecord oldRecord) { + Schema oldSchema = config.populateMetaFields() ? writeSchemaWithMetaFields : writeSchema; + Schema newSchema = useWriterSchemaForCompaction ? writeSchemaWithMetaFields : writeSchema; + String key = oldRecord.getRecordKey(oldSchema, keyGeneratorOpt); // To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than // the oldRecord's key. @@ -93,11 +92,7 @@ public void write(GenericRecord oldRecord) { throw new HoodieUpsertException("Insert/Update not in sorted order"); } try { - if (useWriterSchemaForCompaction) { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writeSchemaWithMetaFields, config.getProps())); - } else { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writeSchema, config.getProps())); - } + writeRecord(hoodieRecord, Option.of(hoodieRecord), newSchema, config.getProps()); insertRecordsWritten++; writtenRecordKeys.add(keyToPreWrite); } catch (IOException e) { @@ -117,9 +112,9 @@ public List close() { HoodieRecord hoodieRecord = keyToNewRecords.get(key); if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) { if (useWriterSchemaForCompaction) { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writeSchemaWithMetaFields, config.getProps())); + writeRecord(hoodieRecord, Option.of(hoodieRecord), writeSchemaWithMetaFields, config.getProps()); } else { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writeSchema, config.getProps())); + writeRecord(hoodieRecord, Option.of(hoodieRecord), writeSchema, config.getProps()); } insertRecordsWritten++; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandleWithChangeLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandleWithChangeLog.java index 8d317b709a4f2..819cfd0754f6e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandleWithChangeLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandleWithChangeLog.java @@ -19,23 +19,25 @@ package org.apache.hudi.io; import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; 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.keygen.BaseKeyGenerator; import org.apache.hudi.table.HoodieTable; -import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.Schema; +import java.io.IOException; +import java.util.Properties; import java.util.Iterator; import java.util.Map; /** * A sorted merge handle that supports logging change logs. */ -public class HoodieSortedMergeHandleWithChangeLog extends HoodieMergeHandleWithChangeLog { +public class HoodieSortedMergeHandleWithChangeLog extends HoodieMergeHandleWithChangeLog { public HoodieSortedMergeHandleWithChangeLog(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Iterator> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier, Option keyGeneratorOpt) { @@ -51,9 +53,10 @@ public HoodieSortedMergeHandleWithChangeLog(HoodieWriteConfig config, String ins super(config, instantTime, hoodieTable, keyToNewRecords, partitionPath, fileId, dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); } - protected boolean writeRecord(HoodieRecord hoodieRecord, Option insertRecord) { - final boolean result = super.writeRecord(hoodieRecord, insertRecord); - this.cdcLogger.put(hoodieRecord, null, insertRecord); + protected boolean writeRecord(HoodieRecord newRecord, Option insertRecord, Schema schema, Properties props) + throws IOException { + final boolean result = super.writeRecord(newRecord, insertRecord, schema, props); + this.cdcLogger.put(newRecord, null, insertRecord.map(rec -> ((HoodieAvroIndexedRecord) rec).getData())); return result; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieUnboundedCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieUnboundedCreateHandle.java index ebbc7a5c28ea1..71a19816879ae 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieUnboundedCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieUnboundedCreateHandle.java @@ -20,7 +20,6 @@ import org.apache.hudi.common.engine.TaskContextSupplier; 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.table.HoodieTable; @@ -36,7 +35,7 @@ * Please use this with caution. This can end up creating very large files if not used correctly. */ @NotThreadSafe -public class HoodieUnboundedCreateHandle extends HoodieCreateHandle { +public class HoodieUnboundedCreateHandle extends HoodieCreateHandle { private static final Logger LOG = LogManager.getLogger(HoodieUnboundedCreateHandle.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 89af038e5168b..151f2e6f99fc6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -18,14 +18,20 @@ package org.apache.hudi.io; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.util.HoodieTimer; @@ -36,18 +42,11 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.marker.WriteMarkersFactory; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.util.Collections; -import java.util.HashMap; import java.util.List; import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; @@ -55,34 +54,16 @@ /** * Base class for all write operations logically performed at the file group level. */ -public abstract class HoodieWriteHandle extends HoodieIOHandle { +public abstract class HoodieWriteHandle extends HoodieIOHandle { private static final Logger LOG = LogManager.getLogger(HoodieWriteHandle.class); - /** - * A special record returned by {@link HoodieRecordPayload}, which means - * {@link HoodieWriteHandle} should just skip this record. - * This record is only used for {@link HoodieRecordPayload} currently, so it should not - * shuffle though network, we can compare the record locally by the equal method. - * The HoodieRecordPayload#combineAndGetUpdateValue and HoodieRecordPayload#getInsertValue - * have 3 kind of return: - * 1、Option.empty - * This means we should delete this record. - * 2、IGNORE_RECORD - * This means we should not process this record,just skip. - * 3、Other non-empty record - * This means we should process this record. - * - * We can see the usage of IGNORE_RECORD in - * org.apache.spark.sql.hudi.command.payload.ExpressionPayload - */ - public static IgnoreRecord IGNORE_RECORD = new IgnoreRecord(); - /** * Schema used to write records into data files */ protected final Schema writeSchema; protected final Schema writeSchemaWithMetaFields; + protected final HoodieRecordMerger recordMerger; protected HoodieTimer timer; protected WriteStatus writeStatus; @@ -113,6 +94,7 @@ protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String this.taskContextSupplier = taskContextSupplier; this.writeToken = makeWriteToken(); schemaOnReadEnabled = !isNullOrEmpty(hoodieTable.getConfig().getInternalSchema()); + this.recordMerger = config.getRecordMerger(); } /** @@ -159,6 +141,10 @@ public Schema getWriterSchemaWithMetaFields() { return writeSchemaWithMetaFields; } + public Schema getWriterSchema() { + return writeSchema; + } + /** * Determines whether we can accept the incoming records, into the current file. Depending on *

@@ -176,35 +162,15 @@ boolean layoutControlsNumFiles() { /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record, Option insertValue) { + protected void doWrite(HoodieRecord record, Schema schema, TypedProperties props) { // NO_OP } /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record, Option avroRecord, Option exception) { - Option recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata(); - if (exception.isPresent() && exception.get() instanceof Throwable) { - // Not throwing exception from here, since we don't want to fail the entire job for a single record - writeStatus.markFailure(record, exception.get(), recordMetadata); - LOG.error("Error writing record " + record, exception.get()); - } else { - write(record, avroRecord); - } - } - - /** - * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields. - */ - protected GenericRecord rewriteRecord(GenericRecord record) { - return schemaOnReadEnabled ? HoodieAvroUtils.rewriteRecordWithNewSchema(record, writeSchemaWithMetaFields, new HashMap<>()) - : HoodieAvroUtils.rewriteRecord(record, writeSchemaWithMetaFields); - } - - protected GenericRecord rewriteRecordWithMetadata(GenericRecord record, String fileName) { - return schemaOnReadEnabled ? HoodieAvroUtils.rewriteEvolutionRecordWithMetadata(record, writeSchemaWithMetaFields, fileName) - : HoodieAvroUtils.rewriteRecordWithMetadata(record, writeSchemaWithMetaFields, fileName); + public void write(HoodieRecord record, Schema schema, TypedProperties props) { + doWrite(record, schema, props); } public abstract List close(); @@ -275,31 +241,12 @@ protected HoodieLogFormat.Writer createLogWriter(String baseCommitTime, String f } } - private static class IgnoreRecord implements GenericRecord { - - @Override - public void put(int i, Object v) { - - } - - @Override - public Object get(int i) { - return null; - } - - @Override - public Schema getSchema() { - return null; - } - - @Override - public void put(String key, Object v) { - - } - - @Override - public Object get(String key) { - return null; + protected static Option toAvroRecord(HoodieRecord record, Schema writerSchema, TypedProperties props) { + try { + return record.toIndexedRecord(writerSchema, props).map(HoodieAvroIndexedRecord::getData); + } catch (IOException e) { + LOG.error("Fail to get indexRecord from " + record, e); + return Option.empty(); } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/SingleFileHandleCreateFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/SingleFileHandleCreateFactory.java index a3f7c04ef23a8..fa5ce2c68bd88 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/SingleFileHandleCreateFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/SingleFileHandleCreateFactory.java @@ -19,7 +19,6 @@ package org.apache.hudi.io; import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; @@ -32,7 +31,7 @@ *

* Please use this with caution. This can end up creating very large files if not used correctly. */ -public class SingleFileHandleCreateFactory extends CreateHandleFactory implements Serializable { +public class SingleFileHandleCreateFactory extends CreateHandleFactory implements Serializable { private final AtomicBoolean isHandleCreated = new AtomicBoolean(false); private final String fileId; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java index c267b5969d801..46a0b1c614d1f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java @@ -20,13 +20,12 @@ 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.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import java.io.Serializable; -public abstract class WriteHandleFactory implements Serializable { +public abstract class WriteHandleFactory implements Serializable { private int numFilesWritten = 0; public abstract HoodieWriteHandle create(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, 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 deleted file mode 100644 index 9ee8571ebd066..0000000000000 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.io.storage; - -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.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; - -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -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.io.storage.HoodieHFileConfig.CACHE_DATA_IN_L1; -import static org.apache.hudi.io.storage.HoodieHFileConfig.DROP_BEHIND_CACHE_COMPACTION; -import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR; -import static org.apache.hudi.io.storage.HoodieHFileConfig.PREFETCH_ON_OPEN; - -public class HoodieFileWriterFactory { - - public static HoodieFileWriter getFileWriter( - String instantTime, Path path, HoodieTable hoodieTable, HoodieWriteConfig config, Schema schema, - TaskContextSupplier taskContextSupplier) throws IOException { - final String extension = FSUtils.getFileExtension(path.getName()); - if (PARQUET.getFileExtension().equals(extension)) { - return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier, config.populateMetaFields()); - } - if (HFILE.getFileExtension().equals(extension)) { - return newHFileFileWriter( - instantTime, path, config, schema, hoodieTable.getHadoopConf(), taskContextSupplier); - } - if (ORC.getFileExtension().equals(extension)) { - return newOrcFileWriter( - instantTime, path, config, schema, hoodieTable.getHadoopConf(), taskContextSupplier); - } - throw new UnsupportedOperationException(extension + " format not supported yet."); - } - - private static HoodieFileWriter newParquetFileWriter( - String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable, - TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException { - return newParquetFileWriter(instantTime, path, config, schema, hoodieTable.getHadoopConf(), - taskContextSupplier, populateMetaFields, populateMetaFields); - } - - private static HoodieFileWriter newParquetFileWriter( - String instantTime, Path path, HoodieWriteConfig config, Schema schema, Configuration conf, - TaskContextSupplier taskContextSupplier, boolean populateMetaFields, boolean enableBloomFilter) throws IOException { - Option filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(conf).convert(schema), schema, filter); - - HoodieParquetConfig parquetConfig = new HoodieParquetConfig<>(writeSupport, config.getParquetCompressionCodec(), - config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(), - conf, config.getParquetCompressionRatio(), config.parquetDictionaryEnabled()); - - return new HoodieAvroParquetWriter<>(path, parquetConfig, instantTime, taskContextSupplier, populateMetaFields); - } - - static HoodieFileWriter newHFileFileWriter( - String instantTime, Path path, HoodieWriteConfig config, Schema schema, Configuration conf, - TaskContextSupplier taskContextSupplier) throws IOException { - - BloomFilter filter = createBloomFilter(config); - HoodieHFileConfig hfileConfig = new HoodieHFileConfig(conf, - config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), - HoodieHFileReader.KEY_FIELD_NAME, PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, - filter, HFILE_COMPARATOR); - - return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier, config.populateMetaFields()); - } - - private static HoodieFileWriter newOrcFileWriter( - String instantTime, Path path, HoodieWriteConfig config, Schema schema, Configuration conf, - TaskContextSupplier taskContextSupplier) throws IOException { - BloomFilter filter = createBloomFilter(config); - HoodieOrcConfig orcConfig = new HoodieOrcConfig(conf, config.getOrcCompressionCodec(), - config.getOrcStripeSize(), config.getOrcBlockSize(), config.getOrcMaxFileSize(), filter); - return new HoodieOrcWriter<>(instantTime, path, orcConfig, schema, taskContextSupplier); - } - - private static BloomFilter createBloomFilter(HoodieWriteConfig config) { - return BloomFilterFactory.createBloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP(), - config.getDynamicBloomFilterMaxNumEntries(), - config.getBloomFilterType()); - } -} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 866fb432ab4ed..2408646e3900f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -994,7 +994,9 @@ protected HoodieData prepRecords(Map rddSinglePartitionRecords = records.map(r -> { FileSlice slice = finalFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), fileGroupCount)); + r.unseal(); r.setCurrentLocation(new HoodieRecordLocation(slice.getBaseInstantTime(), slice.getFileId())); + r.seal(); return r; }); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java index cddac8e2ac653..8bad9fefa5c15 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java @@ -37,7 +37,6 @@ import java.rmi.server.UnicastRemoteObject; import java.util.Objects; - /** * A reporter which publishes metric values to a JMX server. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieCompactionHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieCompactionHandler.java index 7c0ad4a4ad647..4d6a216cf2312 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieCompactionHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieCompactionHandler.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.WriteStatus; 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.table.log.block.HoodieLogBlock; import org.apache.hudi.exception.HoodieNotSupportedException; @@ -36,15 +35,15 @@ * * @param HoodieRecordPayload type. */ -public interface HoodieCompactionHandler { +public interface HoodieCompactionHandler { Iterator> handleUpdate(String instantTime, String partitionPath, String fileId, Map> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException; Iterator> handleInsert(String instantTime, String partitionPath, String fileId, - Map> recordMap); + Map> recordMap); default Iterator> handleInsertsForLogCompaction(String instantTime, String partitionPath, String fileId, - Map> recordMap, + Map> recordMap, Map header) { throw new HoodieNotSupportedException("Operation is not yet supported"); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index c1a29220857e1..3a31c80d4289e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -43,7 +43,6 @@ import org.apache.hudi.common.fs.OptimisticConsistencyGuard; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableConfig; @@ -115,7 +114,7 @@ * @param Type of keys * @param Type of outputs */ -public abstract class HoodieTable implements Serializable { +public abstract class HoodieTable implements Serializable { private static final Logger LOG = LogManager.getLogger(HoodieTable.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java index d2b2cef2f604b..b672d4d6bbb2e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java @@ -26,12 +26,11 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; -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; -public abstract class BaseActionExecutor implements Serializable { +public abstract class BaseActionExecutor implements Serializable { protected final transient HoodieEngineContext context; protected final transient Configuration hadoopConf; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java index fbdb941365b76..f9b85679fbbe1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java @@ -18,14 +18,11 @@ package org.apache.hudi.table.action.bootstrap; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.HoodieConsumer; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.HoodieBootstrapHandle; -import java.io.IOException; - /** * Consumer that dequeues records from queue and sends to Merge Handle for writing. */ @@ -39,12 +36,7 @@ public BootstrapRecordConsumer(HoodieBootstrapHandle bootstrapHandle) { @Override public void consume(HoodieRecord record) { - try { - bootstrapHandle.write(record, ((HoodieRecordPayload) record.getData()) - .getInsertValue(bootstrapHandle.getWriterSchemaWithMetaFields())); - } catch (IOException e) { - throw new HoodieIOException(e.getMessage(), e); - } + bootstrapHandle.write(record, bootstrapHandle.getWriterSchema(), new TypedProperties()); } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java index c0a4f7731689d..9137eb436bb8a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.CleanFileInfo; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.CleanerUtils; @@ -55,7 +54,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -public class CleanActionExecutor extends BaseActionExecutor { +public class CleanActionExecutor extends BaseActionExecutor { private static final long serialVersionUID = 1L; private static final Logger LOG = LogManager.getLogger(CleanActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java index 7f3b437178fd4..1f6a5a1d790b3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.CleanFileInfo; import org.apache.hudi.common.model.HoodieCleaningPolicy; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -46,7 +45,7 @@ import java.util.Map; import java.util.stream.Collectors; -public class CleanPlanActionExecutor extends BaseActionExecutor> { +public class CleanPlanActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(CleanPlanner.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index 396b47ae0a30c..5de92af3258e1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.cdc.HoodieCDCUtils; @@ -74,7 +73,7 @@ *

* 2) It bounds the growth of the files in the file system */ -public class CleanPlanner implements Serializable { +public class CleanPlanner implements Serializable { private static final Logger LOG = LogManager.getLogger(CleanPlanner.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java index e0e02bae8e14c..683be09efeec4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -41,7 +40,7 @@ import java.util.Collections; import java.util.Map; -public class ClusteringPlanActionExecutor extends BaseActionExecutor> { +public class ClusteringPlanActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(ClusteringPlanActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java index 163947fa34481..4b7240d432a43 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java @@ -20,7 +20,7 @@ import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -34,17 +34,19 @@ /** * Pluggable implementation for writing data into new file groups based on ClusteringPlan. */ -public abstract class ClusteringExecutionStrategy implements Serializable { +public abstract class ClusteringExecutionStrategy implements Serializable { private static final Logger LOG = LogManager.getLogger(ClusteringExecutionStrategy.class); private final HoodieTable hoodieTable; private final transient HoodieEngineContext engineContext; - private final HoodieWriteConfig writeConfig; + protected final HoodieWriteConfig writeConfig; + protected final HoodieRecordType recordType; public ClusteringExecutionStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { this.writeConfig = writeConfig; this.hoodieTable = table; this.engineContext = engineContext; + this.recordType = table.getConfig().getRecordMerger().getRecordType(); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java index 2df1824c5f72f..e55e900c3cedb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieFileGroupId; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -50,7 +49,7 @@ /** * Pluggable implementation for scheduling clustering and creating ClusteringPlan. */ -public abstract class ClusteringPlanStrategy implements Serializable { +public abstract class ClusteringPlanStrategy implements Serializable { private static final Logger LOG = LogManager.getLogger(ClusteringPlanStrategy.class); public static final int CLUSTERING_PLAN_VERSION_1 = 1; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java index e12d6d27aa278..915ccc2df0386 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -44,7 +43,7 @@ /** * Scheduling strategy with restriction that clustering groups can only contain files from same partition. */ -public abstract class PartitionAwareClusteringPlanStrategy extends ClusteringPlanStrategy { +public abstract class PartitionAwareClusteringPlanStrategy extends ClusteringPlanStrategy { private static final Logger LOG = LogManager.getLogger(PartitionAwareClusteringPlanStrategy.class); public PartitionAwareClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java index c08c3f312dbc8..4463f7887bb47 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java @@ -20,7 +20,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.table.HoodieTable; @@ -30,7 +29,7 @@ /** * When file groups in clustering, write records to these file group need to check. */ -public abstract class UpdateStrategy implements Serializable { +public abstract class UpdateStrategy implements Serializable { protected final transient HoodieEngineContext engineContext; protected final HoodieTable table; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java index aa7196e3dbbed..b55993856790c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java @@ -18,7 +18,6 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.WriteHandleFactory; @@ -26,7 +25,7 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public abstract class BaseBulkInsertHelper { +public abstract class BaseBulkInsertHelper { /** * Mark instant as inflight, write input records, update index and return result. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 5f406d8920165..1e92f80227482 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -31,7 +31,6 @@ import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -70,7 +69,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -public abstract class BaseCommitActionExecutor +public abstract class BaseCommitActionExecutor extends BaseActionExecutor { private static final Logger LOG = LogManager.getLogger(BaseCommitActionExecutor.class); @@ -157,7 +156,6 @@ protected String getCommitActionType() { return table.getMetaClient().getCommitActionType(); } - /** * Check if any validators are configured and run those validations. If any of the validations fail, throws HoodieValidationException. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeleteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeleteHelper.java index b119587f47535..ceeb2aeb70dee 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeleteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseDeleteHelper.java @@ -19,7 +19,6 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -29,7 +28,7 @@ * * @param */ -public abstract class BaseDeleteHelper { +public abstract class BaseDeleteHelper { /** * Deduplicate Hoodie records, using the given deduplication function. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java index f6572aae4a617..17b8620da63f6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java @@ -18,12 +18,11 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.queue.HoodieConsumer; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.table.HoodieTable; -import org.apache.avro.generic.GenericRecord; - import java.io.IOException; /** @@ -42,7 +41,7 @@ public abstract class BaseMergeHelper { /** * Consumer that dequeues records from queue and sends to Merge Handle. */ - protected static class UpdateHandler implements HoodieConsumer { + protected static class UpdateHandler implements HoodieConsumer { private final HoodieMergeHandle upsertHandle; @@ -51,7 +50,7 @@ protected UpdateHandler(HoodieMergeHandle upsertHandle) { } @Override - public void consume(GenericRecord record) { + public void consume(HoodieRecord record) { upsertHandle.write(record); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java index c69d8746d1913..adef1c44591a8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java @@ -18,8 +18,10 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; @@ -29,8 +31,9 @@ import java.time.Duration; import java.time.Instant; +import java.util.Properties; -public abstract class BaseWriteHelper { +public abstract class BaseWriteHelper { public HoodieWriteMetadata write(String instantTime, I inputRecords, @@ -80,11 +83,16 @@ public I combineOnCondition( * @param parallelism parallelism or partitions to be used while reducing/deduplicating * @return Collection of HoodieRecord already be deduplicated */ - public I deduplicateRecords( - I records, HoodieTable table, int parallelism) { - return deduplicateRecords(records, table.getIndex(), parallelism, table.getConfig().getSchema()); + public I deduplicateRecords(I records, HoodieTable table, int parallelism) { + HoodieRecordMerger recordMerger = table.getConfig().getRecordMerger(); + return deduplicateRecords(records, table.getIndex(), parallelism, table.getConfig().getSchema(), table.getConfig().getProps(), recordMerger); } - public abstract I deduplicateRecords( - I records, HoodieIndex index, int parallelism, String schema); + public I deduplicateRecords(I records, HoodieIndex index, int parallelism, String schema, Properties props, HoodieRecordMerger merger) { + TypedProperties updatedProps = HoodieAvroRecordMerger.Config.withLegacyOperatingModePreCombining(props); + return doDeduplicateRecords(records, index, parallelism, schema, updatedProps, merger); + } + + protected abstract I doDeduplicateRecords( + I records, HoodieIndex index, int parallelism, String schema, TypedProperties props, HoodieRecordMerger merger); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java index fff52eb24d736..0d212555ab228 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java @@ -24,9 +24,10 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieEmptyRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; @@ -45,7 +46,7 @@ * @param */ @SuppressWarnings("checkstyle:LineLength") -public class HoodieDeleteHelper extends +public class HoodieDeleteHelper extends BaseDeleteHelper>, HoodieData, HoodieData, R> { private HoodieDeleteHelper() { } @@ -85,8 +86,15 @@ public HoodieWriteMetadata> execute(String instantTime, dedupedKeys = keys.repartition(parallelism); } - HoodieData> dedupedRecords = - dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload())); + HoodieData dedupedRecords; + HoodieRecordType recordType = config.getRecordMerger().getRecordType(); + if (recordType == HoodieRecordType.AVRO) { + // For BWC, will remove when HoodieRecordPayload removed + dedupedRecords = + dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload())); + } else { + dedupedRecords = dedupedKeys.map(key -> new HoodieEmptyRecord<>(key, recordType)); + } Instant beginTag = Instant.now(); // perform index loop up to get existing location of records HoodieData> taggedRecords = table.getIndex().tagLocation(dedupedRecords, context, table); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java index 0b50f2a302fe0..2caa20c69c5c8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java @@ -18,17 +18,17 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.utils.MergingIterator; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.MappingIterator; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.queue.HoodieExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -46,9 +46,10 @@ import org.apache.avro.Schema; import org.apache.avro.SchemaCompatibility; -import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import java.io.IOException; import java.util.Iterator; @@ -59,9 +60,10 @@ import java.util.stream.Collectors; import static org.apache.hudi.avro.AvroSchemaUtils.isStrictProjectionOf; -import static org.apache.hudi.avro.HoodieAvroUtils.rewriteRecordWithNewSchema; -public class HoodieMergeHelper extends BaseMergeHelper { +public class HoodieMergeHelper extends BaseMergeHelper { + + private static final Logger LOG = LogManager.getLogger(HoodieMergeHelper.class); private HoodieMergeHelper() { } @@ -81,15 +83,18 @@ public void runMerge(HoodieTable table, HoodieBaseFile baseFile = mergeHandle.baseFileForMerge(); Configuration hadoopConf = new Configuration(table.getHadoopConf()); - HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf, mergeHandle.getOldFilePath()); - HoodieFileReader bootstrapFileReader = null; + HoodieRecord.HoodieRecordType recordType = table.getConfig().getRecordMerger().getRecordType(); + HoodieFileReader baseFileReader = HoodieFileReaderFactory + .getReaderFactory(recordType) + .getFileReader(hadoopConf, mergeHandle.getOldFilePath()); + HoodieFileReader bootstrapFileReader = null; Schema writerSchema = mergeHandle.getWriterSchemaWithMetaFields(); Schema readerSchema = baseFileReader.getSchema(); // In case Advanced Schema Evolution is enabled we might need to rewrite currently // persisted records to adhere to an evolved schema - Option> schemaEvolutionTransformerOpt = + Option>, Schema>> schemaEvolutionTransformerOpt = composeSchemaEvolutionTransformer(writerSchema, baseFile, writeConfig, table.getMetaClient()); // Check whether the writer schema is simply a projection of the file's one, ie @@ -103,36 +108,46 @@ public void runMerge(HoodieTable table, || !isPureProjection || baseFile.getBootstrapBaseFile().isPresent(); - HoodieExecutor wrapper = null; + HoodieExecutor wrapper = null; try { - Iterator recordIterator; + Iterator recordIterator; // In case writer's schema is simply a projection of the reader's one we can read // the records in the projected schema directly - ClosableIterator baseFileRecordIterator = baseFileReader.getRecordIterator(isPureProjection ? writerSchema : readerSchema); - + ClosableIterator baseFileRecordIterator = + baseFileReader.getRecordIterator(isPureProjection ? writerSchema : readerSchema); + Schema recordSchema; if (baseFile.getBootstrapBaseFile().isPresent()) { Path bootstrapFilePath = new Path(baseFile.getBootstrapBaseFile().get().getPath()); Configuration bootstrapFileConfig = new Configuration(table.getHadoopConf()); - bootstrapFileReader = HoodieFileReaderFactory.getFileReader(bootstrapFileConfig, bootstrapFilePath); - recordIterator = new MergingIterator<>( - baseFileRecordIterator, - bootstrapFileReader.getRecordIterator(), - (inputRecordPair) -> HoodieAvroUtils.stitchRecords(inputRecordPair.getLeft(), inputRecordPair.getRight(), mergeHandle.getWriterSchemaWithMetaFields())); - + bootstrapFileReader = + HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(bootstrapFileConfig, bootstrapFilePath); + recordIterator = new MergingIterator(baseFileRecordIterator, bootstrapFileReader.getRecordIterator(), + (left, right) -> left.joinWith(right, mergeHandle.getWriterSchemaWithMetaFields())); + recordSchema = mergeHandle.getWriterSchemaWithMetaFields(); } else if (schemaEvolutionTransformerOpt.isPresent()) { recordIterator = new MappingIterator<>(baseFileRecordIterator, - schemaEvolutionTransformerOpt.get()); + schemaEvolutionTransformerOpt.get().getLeft().apply(isPureProjection ? writerSchema : readerSchema)); + recordSchema = schemaEvolutionTransformerOpt.get().getRight(); } else { recordIterator = baseFileRecordIterator; + recordSchema = isPureProjection ? writerSchema : readerSchema; } wrapper = QueueBasedExecutorFactory.create(writeConfig, recordIterator, new UpdateHandler(mergeHandle), record -> { + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into queue of QueueBasedExecutorFactory. if (shouldRewriteInWriterSchema) { - return rewriteRecordWithNewSchema(record, writerSchema); + try { + return record.rewriteRecordWithNewSchema(recordSchema, writeConfig.getProps(), writerSchema).copy(); + } catch (IOException e) { + LOG.error("Error rewrite record with new schema", e); + throw new HoodieException(e); + } } else { - return record; + return record.copy(); } }, table.getPreExecuteRunnable()); @@ -154,7 +169,7 @@ public void runMerge(HoodieTable table, } } - private Option> composeSchemaEvolutionTransformer(Schema writerSchema, + private Option>, Schema>> composeSchemaEvolutionTransformer(Schema writerSchema, HoodieBaseFile baseFile, HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient) { @@ -195,7 +210,18 @@ private Option> composeSchemaEvolutionTra || SchemaCompatibility.checkReaderWriterCompatibility(newWriterSchema, writeSchemaFromFile).getType() == org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE; if (needToReWriteRecord) { Map renameCols = InternalSchemaUtils.collectRenameCols(writeInternalSchema, querySchema); - return Option.of(record -> rewriteRecordWithNewSchema(record, newWriterSchema, renameCols)); + return Option.of(Pair.of( + (schema) -> (record) -> { + try { + return record.rewriteRecordWithNewSchema( + schema, + writeConfig.getProps(), + newWriterSchema, renameCols); + } catch (IOException e) { + LOG.error("Error rewrite record with new schema", e); + throw new HoodieException(e); + } + }, newWriterSchema)); } else { return Option.empty(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java index b1d18573fcfc6..6557f83b24181 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java @@ -20,20 +20,21 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; -public class HoodieWriteHelper extends BaseWriteHelper>, - HoodieData, HoodieData, R> { +import java.io.IOException; +public class HoodieWriteHelper extends BaseWriteHelper>, + HoodieData, HoodieData, R> { private HoodieWriteHelper() { } @@ -52,8 +53,8 @@ protected HoodieData> tag(HoodieData> dedupedRec } @Override - public HoodieData> deduplicateRecords( - HoodieData> records, HoodieIndex index, int parallelism, String schemaStr) { + protected HoodieData> doDeduplicateRecords( + HoodieData> records, HoodieIndex index, int parallelism, String schemaStr, TypedProperties props, HoodieRecordMerger merger) { boolean isIndexingGlobal = index.isGlobal(); final SerializableSchema schema = new SerializableSchema(schemaStr); // Auto-tunes the parallelism for reduce transformation based on the number of data partitions @@ -63,13 +64,19 @@ public HoodieData> deduplicateRecords( HoodieKey hoodieKey = record.getKey(); // If index used is global, then records are expected to differ in their partitionPath Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey; - return Pair.of(key, record); + // NOTE: PLEASE READ CAREFULLY BEFORE CHANGING + // Here we have to make a copy of the incoming record, since it might be holding + // an instance of [[InternalRow]] pointing into shared, mutable buffer + return Pair.of(key, record.copy()); }).reduceByKey((rec1, rec2) -> { - @SuppressWarnings("unchecked") - T reducedData = (T) rec2.getData().preCombine(rec1.getData(), schema.get(), CollectionUtils.emptyProps()); - HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey(); - - return new HoodieAvroRecord<>(reducedKey, reducedData); + HoodieRecord reducedRecord; + try { + reducedRecord = merger.merge(rec1, schema.get(), rec2, schema.get(), props).get().getLeft(); + } catch (IOException e) { + throw new HoodieException(String.format("Error to merge two records, %s, %s", rec1, rec2), e); + } + HoodieKey reducedKey = rec1.getData().equals(reducedRecord.getData()) ? rec1.getKey() : rec2.getKey(); + return reducedRecord.newInstance(reducedKey); }, reduceParallelism).map(Pair::getRight); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java index a4daacfedbeec..f4498a82d4fb8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -43,7 +42,7 @@ * @param Type of keys * @param Type of outputs */ -public class CompactHelpers { +public class CompactHelpers { private static final CompactHelpers SINGLETON_INSTANCE = new CompactHelpers(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java index 51ed7639a4923..c6a20436c03ef 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -65,7 +64,7 @@ /** * A HoodieCompactor runs compaction on a hoodie table. */ -public abstract class HoodieCompactor implements Serializable { +public abstract class HoodieCompactor implements Serializable { private static final Logger LOG = LogManager.getLogger(HoodieCompactor.class); @@ -201,6 +200,7 @@ public List compact(HoodieCompactionHandler compactionHandler, .withOperationField(config.allowOperationMetadataField()) .withPartition(operation.getPartitionPath()) .withUseScanV2(executionHelper.useScanV2(config)) + .withRecordMerger(config.getRecordMerger()) .build(); Option oldDataFileOpt = diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java index a0e5ec22f9f5f..c26e2a9ec516e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -46,7 +45,7 @@ import static org.apache.hudi.common.util.ValidationUtils.checkArgument; @SuppressWarnings("checkstyle:LineLength") -public class RunCompactionActionExecutor extends +public class RunCompactionActionExecutor extends BaseActionExecutor>, HoodieData, HoodieData, HoodieWriteMetadata>> { private final HoodieCompactor compactor; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java index 7b4fa0fa4cf1e..328ed7d922186 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -52,7 +51,7 @@ import static org.apache.hudi.common.util.ValidationUtils.checkArgument; -public class ScheduleCompactionActionExecutor extends BaseActionExecutor> { +public class ScheduleCompactionActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class); private WriteOperationType operationType; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java index 7357054b4e233..e2ed1a06ac9a0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java @@ -92,6 +92,7 @@ private boolean isFileSliceEligibleForLogCompaction(FileSlice fileSlice, String .withLatestInstantTime(maxInstantTime) .withBufferSize(writeConfig.getMaxDFSStreamBufferSize()) .withUseScanV2(true) + .withRecordMerger(writeConfig.getRecordMerger()) .build(); scanner.scanInternal(Option.empty(), true); int totalBlocks = scanner.getCurrentInstantLogBlocks().size(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index 0d43df70c7401..2fcbfb2b2e5b6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -28,7 +28,6 @@ import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -82,7 +81,7 @@ * Reads the index plan and executes the plan. * It also reconciles updates on data timeline while indexing was in progress. */ -public class RunIndexActionExecutor extends BaseActionExecutor> { +public class RunIndexActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(RunIndexActionExecutor.class); private static final Integer INDEX_COMMIT_METADATA_VERSION_1 = 1; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index d562dec671d14..91587b8bd5606 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -23,7 +23,6 @@ import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -60,7 +59,7 @@ * 3. Initialize file groups for the enabled partition types within a transaction. * */ -public class ScheduleIndexActionExecutor extends BaseActionExecutor> { +public class ScheduleIndexActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(ScheduleIndexActionExecutor.class); private static final Integer INDEX_PLAN_VERSION_1 = 1; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index 8a577021d249f..dfcec915b0094 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -24,7 +24,6 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -49,7 +48,7 @@ import java.util.Map; import java.util.stream.Collectors; -public abstract class BaseRestoreActionExecutor extends BaseActionExecutor { +public abstract class BaseRestoreActionExecutor extends BaseActionExecutor { private static final Logger LOG = LogManager.getLogger(BaseRestoreActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java index f6e104e3dcdc5..9dcb34091353a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -30,7 +29,7 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; -public class CopyOnWriteRestoreActionExecutor +public class CopyOnWriteRestoreActionExecutor extends BaseRestoreActionExecutor { public CopyOnWriteRestoreActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java index 01c3d44fabc93..a7e5774515d49 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -29,7 +28,7 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor; -public class MergeOnReadRestoreActionExecutor +public class MergeOnReadRestoreActionExecutor extends BaseRestoreActionExecutor { public MergeOnReadRestoreActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, String restoreInstantTime) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index eb888bdec7730..b5ae9471e5847 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -50,7 +49,7 @@ import java.util.Objects; import java.util.stream.Collectors; -public abstract class BaseRollbackActionExecutor extends BaseActionExecutor { +public abstract class BaseRollbackActionExecutor extends BaseActionExecutor { private static final Logger LOG = LogManager.getLogger(BaseRollbackActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java index 63b9e8a414b55..c8aa992cf890b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.avro.model.HoodieRollbackRequest; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -44,7 +43,7 @@ * Base rollback plan action executor to assist in scheduling rollback requests. This phase serialized {@link HoodieRollbackPlan} * to rollback.requested instant. */ -public class BaseRollbackPlanActionExecutor extends BaseActionExecutor> { +public class BaseRollbackPlanActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(BaseRollbackPlanActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java index 64b3f483e6486..6579dcfb559f8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; @@ -34,7 +33,7 @@ import java.util.ArrayList; import java.util.List; -public class CopyOnWriteRollbackActionExecutor extends BaseRollbackActionExecutor { +public class CopyOnWriteRollbackActionExecutor extends BaseRollbackActionExecutor { private static final Logger LOG = LogManager.getLogger(CopyOnWriteRollbackActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java index 87ee7d94723d9..000ea21af987e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; @@ -48,7 +47,7 @@ /** * Performs rollback using marker files generated during the write.. */ -public class MarkerBasedRollbackStrategy implements BaseRollbackPlanActionExecutor.RollbackStrategy { +public class MarkerBasedRollbackStrategy implements BaseRollbackPlanActionExecutor.RollbackStrategy { private static final Logger LOG = LogManager.getLogger(MarkerBasedRollbackStrategy.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java index 097897bd1524a..34f456b2d3151 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.config.HoodieWriteConfig; @@ -34,7 +33,7 @@ import java.util.ArrayList; import java.util.List; -public class MergeOnReadRollbackActionExecutor extends BaseRollbackActionExecutor { +public class MergeOnReadRollbackActionExecutor extends BaseRollbackActionExecutor { private static final Logger LOG = LogManager.getLogger(MergeOnReadRollbackActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RestorePlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RestorePlanActionExecutor.java index e33dffcb7b953..34617afc5251a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RestorePlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RestorePlanActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieRestorePlan; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -45,7 +44,7 @@ /** * Plans the restore action and add a restore.requested meta file to timeline. */ -public class RestorePlanActionExecutor extends BaseActionExecutor> { +public class RestorePlanActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(RestorePlanActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java index 7f408c1b8d24a..3bfdd20721eb0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -44,7 +43,7 @@ import java.util.Map; import java.util.stream.Collectors; -public class SavepointActionExecutor extends BaseActionExecutor { +public class SavepointActionExecutor extends BaseActionExecutor { private static final Logger LOG = LogManager.getLogger(SavepointActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroParquetWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroParquetWriter.java index 5b4c4df285c3c..b625d57f7fc21 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroParquetWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroParquetWriter.java @@ -71,8 +71,8 @@ public void testProperWriting() throws IOException { Path filePath = new Path(tmpDir.resolve("test.parquet").toAbsolutePath().toString()); - try (HoodieAvroParquetWriter writer = - new HoodieAvroParquetWriter<>(filePath, parquetConfig, "001", new DummyTaskContextSupplier(), true)) { + try (HoodieAvroParquetWriter writer = + new HoodieAvroParquetWriter(filePath, parquetConfig, "001", new DummyTaskContextSupplier(), true)) { for (GenericRecord record : records) { writer.writeAvro((String) record.get("_row_key"), record); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java index a45b8a9aaa3a5..815b04c9dbc87 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java @@ -69,7 +69,7 @@ import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; import static org.apache.hudi.common.util.CollectionUtils.toStream; import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR; -import static org.apache.hudi.io.storage.HoodieHFileReader.SCHEMA_KEY; +import static org.apache.hudi.io.storage.HoodieAvroHFileReader.SCHEMA_KEY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -89,7 +89,7 @@ protected Path getFilePath() { } @Override - protected HoodieFileWriter createWriter( + protected HoodieAvroHFileWriter createWriter( Schema avroSchema, boolean populateMetaFields) throws Exception { String instantTime = "000"; HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() @@ -104,15 +104,15 @@ protected HoodieFileWriter createWriter( when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier); when(partitionSupplier.get()).thenReturn(10); - return HoodieFileWriterFactory.newHFileFileWriter( - instantTime, getFilePath(), writeConfig, avroSchema, conf, mockTaskContextSupplier); + return (HoodieAvroHFileWriter)HoodieFileWriterFactory.getFileWriter( + instantTime, getFilePath(), conf, writeConfig.getStorageConfig(), avroSchema, mockTaskContextSupplier, writeConfig.getRecordMerger().getRecordType()); } @Override - protected HoodieFileReader createReader( + protected HoodieAvroFileReader createReader( Configuration conf) throws Exception { CacheConfig cacheConfig = new CacheConfig(conf); - return new HoodieHFileReader<>(conf, getFilePath(), cacheConfig, getFilePath().getFileSystem(conf)); + return new HoodieAvroHFileReader(conf, getFilePath(), cacheConfig, getFilePath().getFileSystem(conf)); } @Override @@ -144,7 +144,7 @@ private static Stream populateMetaFieldsAndTestAvroWithMeta() { @MethodSource("populateMetaFieldsAndTestAvroWithMeta") public void testWriteReadHFileWithMetaFields(boolean populateMetaFields, boolean testAvroWithMeta) throws Exception { Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchemaWithMetaFields.avsc"); - HoodieFileWriter writer = createWriter(avroSchema, populateMetaFields); + HoodieAvroHFileWriter writer = createWriter(avroSchema, populateMetaFields); List keys = new ArrayList<>(); Map recordMap = new TreeMap<>(); for (int i = 0; i < 100; i++) { @@ -167,8 +167,8 @@ public void testWriteReadHFileWithMetaFields(boolean populateMetaFields, boolean writer.close(); Configuration conf = new Configuration(); - HoodieHFileReader hoodieHFileReader = (HoodieHFileReader) createReader(conf); - List records = HoodieHFileReader.readAllRecords(hoodieHFileReader); + HoodieAvroHFileReader hoodieHFileReader = (HoodieAvroHFileReader) createReader(conf); + List records = HoodieAvroHFileReader.readAllRecords(hoodieHFileReader); assertEquals(new ArrayList<>(recordMap.values()), records); hoodieHFileReader.close(); @@ -182,8 +182,8 @@ public void testWriteReadHFileWithMetaFields(boolean populateMetaFields, boolean List expectedRecords = rowsList.stream().map(recordMap::get).collect(Collectors.toList()); - hoodieHFileReader = (HoodieHFileReader) createReader(conf); - List result = HoodieHFileReader.readRecords(hoodieHFileReader, rowsList); + hoodieHFileReader = (HoodieAvroHFileReader) createReader(conf); + List result = HoodieAvroHFileReader.readRecords(hoodieHFileReader, rowsList).stream().map(r -> (GenericRecord)r).collect(Collectors.toList()); assertEquals(expectedRecords, result); @@ -212,8 +212,8 @@ public void testReadHFileFormatRecords() throws Exception { byte[] content = FileIOUtils.readAsByteArray( fs.open(getFilePath()), (int) fs.getFileStatus(getFilePath()).getLen()); // Reading byte array in HFile format, without actual file path - HoodieHFileReader hfileReader = - new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content, Option.empty()); + HoodieAvroHFileReader hfileReader = + new HoodieAvroHFileReader(fs, new Path(DUMMY_BASE_PATH), content, Option.empty()); Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc"); assertEquals(NUM_RECORDS, hfileReader.getTotalRecords()); verifySimpleRecords(hfileReader.getRecordIterator(avroSchema)); @@ -222,20 +222,20 @@ public void testReadHFileFormatRecords() throws Exception { @Test public void testReaderGetRecordIterator() throws Exception { writeFileWithSimpleSchema(); - HoodieHFileReader hfileReader = - (HoodieHFileReader) createReader(new Configuration()); + HoodieAvroHFileReader hfileReader = + (HoodieAvroHFileReader) createReader(new Configuration()); List keys = IntStream.concat(IntStream.range(40, NUM_RECORDS * 2), IntStream.range(10, 20)) .mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toList()); Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc"); - Iterator iterator = hfileReader.getRecordsByKeysIterator(keys, avroSchema); + Iterator> iterator = hfileReader.getRecordsByKeysIterator(keys, avroSchema); List expectedIds = IntStream.concat(IntStream.range(40, NUM_RECORDS), IntStream.range(10, 20)) .boxed().collect(Collectors.toList()); int index = 0; while (iterator.hasNext()) { - GenericRecord record = iterator.next(); + GenericRecord record = (GenericRecord) iterator.next().getData(); String key = "key" + String.format("%02d", expectedIds.get(index)); assertEquals(key, record.get("_row_key").toString()); assertEquals(Integer.toString(expectedIds.get(index)), record.get("time").toString()); @@ -247,52 +247,57 @@ public void testReaderGetRecordIterator() throws Exception { @Test public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception { writeFileWithSimpleSchema(); - HoodieHFileReader hfileReader = - (HoodieHFileReader) createReader(new Configuration()); + HoodieAvroHFileReader hfileReader = + (HoodieAvroHFileReader) createReader(new Configuration()); Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc"); List keyPrefixes = Collections.singletonList("key"); - Iterator iterator = - hfileReader.getRecordsByKeyPrefixIterator(keyPrefixes, avroSchema); + Iterator iterator = + hfileReader.getIndexedRecordsByKeyPrefixIterator(keyPrefixes, avroSchema); - List recordsByPrefix = toStream(iterator).collect(Collectors.toList()); + List recordsByPrefix = toStream(iterator).map(r -> (GenericRecord)r).collect(Collectors.toList()); - List allRecords = toStream(hfileReader.getRecordIterator()).collect(Collectors.toList()); + List allRecords = toStream(hfileReader.getRecordIterator()) + .map(r -> (GenericRecord) r.getData()).collect(Collectors.toList()); assertEquals(allRecords, recordsByPrefix); // filter for "key1" : entries from key10 to key19 should be matched List expectedKey1s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key1")).collect(Collectors.toList()); iterator = - hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key1"), avroSchema); + hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key1"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(expectedKey1s, recordsByPrefix); // exact match List expectedKey25 = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key25")).collect(Collectors.toList()); iterator = - hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key25"), avroSchema); + hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key25"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(expectedKey25, recordsByPrefix); // no match. key prefix is beyond entries in file. iterator = - hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key99"), avroSchema); + hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key99"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(Collections.emptyList(), recordsByPrefix); // no match. but keyPrefix is in between the entries found in file. iterator = - hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key1234"), avroSchema); + hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key1234"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(Collections.emptyList(), recordsByPrefix); @@ -300,9 +305,10 @@ public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception { List expectedKey50and1s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key1") || (entry.get("_row_key").toString()).contains("key50")).collect(Collectors.toList()); iterator = - hfileReader.getRecordsByKeyPrefixIterator(Arrays.asList("key50", "key1"), avroSchema); + hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key50", "key1"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(expectedKey50and1s, recordsByPrefix); @@ -310,9 +316,10 @@ public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception { List expectedKey50and0s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key0") || (entry.get("_row_key").toString()).contains("key50")).collect(Collectors.toList()); iterator = - hfileReader.getRecordsByKeyPrefixIterator(Arrays.asList("key50", "key0"), avroSchema); + hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key50", "key0"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(expectedKey50and0s, recordsByPrefix); @@ -321,9 +328,10 @@ public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception { .filter(entry -> (entry.get("_row_key").toString()).contains("key1") || (entry.get("_row_key").toString()).contains("key0")) .collect(Collectors.toList()); iterator = - hfileReader.getRecordsByKeyPrefixIterator(Arrays.asList("key1", "key0"), avroSchema); + hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key1", "key0"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); Collections.sort(recordsByPrefix, new Comparator() { @Override @@ -353,8 +361,8 @@ public void testHoodieHFileCompatibility(String hfilePrefix) throws IOException verifyHFileReader( HoodieHFileUtils.createHFileReader(fs, new Path(DUMMY_BASE_PATH), content), hfilePrefix, true, HFILE_COMPARATOR.getClass(), NUM_RECORDS_FIXTURE); - HoodieHFileReader hfileReader = - new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content, Option.empty()); + HoodieAvroHFileReader hfileReader = + new HoodieAvroHFileReader(fs, new Path(DUMMY_BASE_PATH), content, Option.empty()); Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc"); assertEquals(NUM_RECORDS_FIXTURE, hfileReader.getTotalRecords()); verifySimpleRecords(hfileReader.getRecordIterator(avroSchema)); @@ -362,7 +370,7 @@ public void testHoodieHFileCompatibility(String hfilePrefix) throws IOException content = readHFileFromResources(complexHFile); verifyHFileReader(HoodieHFileUtils.createHFileReader(fs, new Path(DUMMY_BASE_PATH), content), hfilePrefix, true, HFILE_COMPARATOR.getClass(), NUM_RECORDS_FIXTURE); - hfileReader = new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content, Option.empty()); + hfileReader = new HoodieAvroHFileReader(fs, new Path(DUMMY_BASE_PATH), content, Option.empty()); avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchemaWithUDT.avsc"); assertEquals(NUM_RECORDS_FIXTURE, hfileReader.getTotalRecords()); verifySimpleRecords(hfileReader.getRecordIterator(avroSchema)); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java index 373fc31a56272..438024d2f2688 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java @@ -23,10 +23,10 @@ import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.orc.CompressionKind; @@ -51,7 +51,7 @@ protected Path getFilePath() { } @Override - protected HoodieFileWriter createWriter( + protected HoodieAvroOrcWriter createWriter( Schema avroSchema, boolean populateMetaFields) throws Exception { BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.00001, -1, BloomFilterTypeCode.SIMPLE.name()); Configuration conf = new Configuration(); @@ -64,13 +64,13 @@ protected HoodieFileWriter createWriter( when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier); when(partitionSupplier.get()).thenReturn(10); String instantTime = "000"; - return new HoodieOrcWriter<>(instantTime, getFilePath(), config, avroSchema, mockTaskContextSupplier); + return new HoodieAvroOrcWriter(instantTime, getFilePath(), config, avroSchema, mockTaskContextSupplier); } @Override - protected HoodieFileReader createReader( + protected HoodieAvroFileReader createReader( Configuration conf) throws Exception { - return HoodieFileReaderFactory.getFileReader(conf, getFilePath()); + return (HoodieAvroFileReader) HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(conf, getFilePath()); } @Override diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java index 902f42e38f32b..fde1315a34d5c 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java @@ -21,10 +21,13 @@ import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.AfterEach; @@ -60,10 +63,10 @@ public abstract class TestHoodieReaderWriterBase { protected abstract Path getFilePath(); - protected abstract HoodieFileWriter createWriter( + protected abstract HoodieAvroFileWriter createWriter( Schema avroSchema, boolean populateMetaFields) throws Exception; - protected abstract HoodieFileReader createReader( + protected abstract HoodieAvroFileReader createReader( Configuration conf) throws Exception; protected abstract void verifyMetadata(Configuration conf) throws IOException; @@ -87,7 +90,7 @@ public void testWriteReadMetadata() throws Exception { Configuration conf = new Configuration(); verifyMetadata(conf); - HoodieFileReader hoodieReader = createReader(conf); + HoodieAvroFileReader hoodieReader = createReader(conf); BloomFilter filter = hoodieReader.readBloomFilter(); for (int i = 0; i < NUM_RECORDS; i++) { String key = "key" + String.format("%02d", i); @@ -118,7 +121,7 @@ public void testWriteReadComplexRecord() throws Exception { String schemaPath = "/exampleSchemaWithUDT.avsc"; Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, schemaPath); Schema udtSchema = avroSchema.getField("driver").schema().getTypes().get(1); - HoodieFileWriter writer = createWriter(avroSchema, true); + HoodieAvroFileWriter writer = createWriter(avroSchema, true); for (int i = 0; i < NUM_RECORDS; i++) { GenericRecord record = new GenericData.Record(avroSchema); String key = "key" + String.format("%02d", i); @@ -145,7 +148,7 @@ public void testWriteReadWithEvolvedSchema() throws Exception { writeFileWithSimpleSchema(); Configuration conf = new Configuration(); - HoodieFileReader hoodieReader = createReader(conf); + HoodieAvroFileReader hoodieReader = createReader(conf); String[] schemaList = new String[] { "/exampleEvolvedSchema.avsc", "/exampleEvolvedSchemaChangeOrder.avsc", "/exampleEvolvedSchemaColumnRequire.avsc", "/exampleEvolvedSchemaColumnType.avsc", @@ -166,21 +169,22 @@ public void testReaderFilterRowKeys() throws Exception { protected void writeFileWithSimpleSchema() throws Exception { Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc"); - HoodieFileWriter writer = createWriter(avroSchema, true); + HoodieAvroFileWriter writer = createWriter(avroSchema, true); for (int i = 0; i < NUM_RECORDS; i++) { GenericRecord record = new GenericData.Record(avroSchema); String key = "key" + String.format("%02d", i); record.put("_row_key", key); record.put("time", Integer.toString(i)); record.put("number", i); - writer.writeAvro(key, record); + HoodieRecord avroRecord = new HoodieAvroIndexedRecord(record); + writer.write(key, avroRecord, avroSchema); } writer.close(); } protected void writeFileWithSchemaWithMeta() throws Exception { Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchemaWithMetaFields.avsc"); - HoodieFileWriter writer = createWriter(avroSchema, true); + HoodieAvroFileWriter writer = createWriter(avroSchema, true); for (int i = 0; i < NUM_RECORDS; i++) { GenericRecord record = new GenericData.Record(avroSchema); String key = "key" + String.format("%02d", i); @@ -193,10 +197,10 @@ protected void writeFileWithSchemaWithMeta() throws Exception { writer.close(); } - protected void verifySimpleRecords(Iterator iterator) { + protected void verifySimpleRecords(Iterator> iterator) { int index = 0; while (iterator.hasNext()) { - GenericRecord record = iterator.next(); + GenericRecord record = (GenericRecord) iterator.next().getData(); String key = "key" + String.format("%02d", index); assertEquals(key, record.get("_row_key").toString()); assertEquals(Integer.toString(index), record.get("time").toString()); @@ -205,10 +209,10 @@ protected void verifySimpleRecords(Iterator iterator) { } } - protected void verifyComplexRecords(Iterator iterator) { + protected void verifyComplexRecords(Iterator> iterator) { int index = 0; while (iterator.hasNext()) { - GenericRecord record = iterator.next(); + GenericRecord record = (GenericRecord) iterator.next().getData(); String key = "key" + String.format("%02d", index); assertEquals(key, record.get("_row_key").toString()); assertEquals(Integer.toString(index), record.get("time").toString()); @@ -232,7 +236,7 @@ protected void verifyComplexRecords(Iterator iterator) { } } - private void verifyFilterRowKeys(HoodieFileReader hoodieReader) { + private void verifyFilterRowKeys(HoodieAvroFileReader hoodieReader) { Set candidateRowKeys = IntStream.range(40, NUM_RECORDS * 2) .mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toCollection(TreeSet::new)); List expectedKeys = IntStream.range(40, NUM_RECORDS) @@ -241,12 +245,12 @@ private void verifyFilterRowKeys(HoodieFileReader hoodieReader) { .stream().sorted().collect(Collectors.toList())); } - private void verifyReaderWithSchema(String schemaPath, HoodieFileReader hoodieReader) throws IOException { + private void verifyReaderWithSchema(String schemaPath, HoodieAvroFileReader hoodieReader) throws IOException { Schema evolvedSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, schemaPath); - Iterator iter = hoodieReader.getRecordIterator(evolvedSchema); + Iterator> iter = hoodieReader.getRecordIterator(evolvedSchema); int index = 0; while (iter.hasNext()) { - verifyRecord(schemaPath, iter.next(), index); + verifyRecord(schemaPath, (GenericRecord) iter.next().getData(), index); index++; } } 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 fb66957524955..1c66834b3575d 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 @@ -29,6 +29,7 @@ import org.apache.hudi.avro.HoodieAvroWriteSupport; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; @@ -42,10 +43,10 @@ import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.io.storage.HoodieAvroOrcWriter; import org.apache.hudi.io.storage.HoodieAvroParquetWriter; import org.apache.hudi.io.storage.HoodieOrcConfig; -import org.apache.hudi.io.storage.HoodieOrcWriter; import org.apache.hudi.io.storage.HoodieParquetConfig; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.log4j.LogManager; @@ -111,7 +112,7 @@ public Path withInserts(String partition, String fileId, List reco HoodieParquetConfig config = new HoodieParquetConfig<>(writeSupport, CompressionCodecName.GZIP, ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, new Configuration(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue()), true); - try (HoodieAvroParquetWriter writer = new HoodieAvroParquetWriter<>( + try (HoodieAvroParquetWriter writer = new HoodieAvroParquetWriter( new Path(Paths.get(basePath, partition, fileName).toString()), config, currentInstantTime, contextSupplier, populateMetaFields)) { int seqId = 1; @@ -133,7 +134,7 @@ public Path withInserts(String partition, String fileId, List reco int orcBlockSize = Integer.parseInt(HoodieStorageConfig.ORC_BLOCK_SIZE.defaultValue()); int maxFileSize = Integer.parseInt(HoodieStorageConfig.ORC_FILE_MAX_SIZE.defaultValue()); HoodieOrcConfig config = new HoodieOrcConfig(conf, CompressionKind.ZLIB, orcStripSize, orcBlockSize, maxFileSize, filter); - try (HoodieOrcWriter writer = new HoodieOrcWriter( + try (HoodieAvroOrcWriter writer = new HoodieAvroOrcWriter( currentInstantTime, new Path(Paths.get(basePath, partition, fileName).toString()), config, schema, contextSupplier)) { @@ -174,7 +175,7 @@ private Pair appendRecordsToLogFile(String partitionPath, LOG.warn("Failed to convert record " + r.toString(), e); return null; } - }).collect(Collectors.toList()), header, HoodieRecord.RECORD_KEY_METADATA_FIELD)); + }).map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()), header, HoodieRecord.RECORD_KEY_METADATA_FIELD)); return Pair.of(partitionPath, logWriter.getLogFile()); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index 5a7150687ac7f..278e85fbedd70 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.TableServiceType; @@ -86,7 +85,7 @@ * @param type of the payload */ @SuppressWarnings("checkstyle:LineLength") -public class HoodieFlinkWriteClient extends +public class HoodieFlinkWriteClient extends BaseHoodieWriteClient>, List, List> { private static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkWriteClient.class); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java index 891ddf89937e2..3abffe38d8bcc 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java @@ -25,7 +25,6 @@ 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.HoodieRecordPayload; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -48,7 +47,7 @@ * 1) Creates clustering groups based on max size allowed per group. * 2) Excludes files that are greater than 'small.file.limit' from clustering plan. */ -public class FlinkSizeBasedClusteringPlanStrategy> +public class FlinkSizeBasedClusteringPlanStrategy extends PartitionAwareClusteringPlanStrategy>, List, List> { private static final Logger LOG = LogManager.getLogger(FlinkSizeBasedClusteringPlanStrategy.class); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java index 86f38c6f5ff65..187efd8fc814f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java @@ -19,8 +19,8 @@ package org.apache.hudi.execution; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.HoodieConsumer; import org.apache.hudi.io.HoodieWriteHandle; @@ -32,7 +32,7 @@ /** * Consumes stream of hoodie records from in-memory queue and writes to one explicit create handle. */ -public class ExplicitWriteHandler +public class ExplicitWriteHandler implements HoodieConsumer, List> { private final List statuses = new ArrayList<>(); @@ -44,9 +44,9 @@ public ExplicitWriteHandler(HoodieWriteHandle handle) { } @Override - public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { - final HoodieRecord insertPayload = payload.record; - handle.write(insertPayload, payload.insertValue, payload.exception); + public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult genResult) { + final HoodieRecord insertPayload = genResult.getResult(); + handle.write(insertPayload, genResult.schema, new TypedProperties(genResult.props)); } @Override diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java index d99a4e8bf7e16..8a0eb8beca75f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.HoodieExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -41,7 +40,7 @@ * * @param type of the payload */ -public class FlinkLazyInsertIterable extends HoodieLazyInsertIterable { +public class FlinkLazyInsertIterable extends HoodieLazyInsertIterable { public FlinkLazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, @@ -60,8 +59,7 @@ protected List computeNext() { HoodieExecutor, HoodieInsertValueGenResult, List> bufferedIteratorExecutor = null; try { final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); - bufferedIteratorExecutor = QueueBasedExecutorFactory.create(hoodieConfig, inputItr, getExplicitInsertHandler(), - getTransformFunction(schema, hoodieConfig)); + bufferedIteratorExecutor = QueueBasedExecutorFactory.create(hoodieConfig, inputItr, getExplicitInsertHandler(), getCloningTransformer(schema, hoodieConfig)); final List result = bufferedIteratorExecutor.execute(); checkState(result != null && !result.isEmpty()); return result; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java index be2273a8409b8..f43130234f8dd 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.table.HoodieTable; @@ -37,7 +36,7 @@ /** * Base flink implementation of {@link HoodieIndex}. */ -public abstract class FlinkHoodieIndex extends HoodieIndex>, List> { +public abstract class FlinkHoodieIndex extends HoodieIndex>, List> { protected FlinkHoodieIndex(HoodieWriteConfig config) { super(config); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/ExplicitWriteHandleFactory.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/ExplicitWriteHandleFactory.java index e598a033750dd..c040603db4c4a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/ExplicitWriteHandleFactory.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/ExplicitWriteHandleFactory.java @@ -19,14 +19,13 @@ package org.apache.hudi.io; import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; /** * Create handle factory for Flink writer, use the specified write handle directly. */ -public class ExplicitWriteHandleFactory +public class ExplicitWriteHandleFactory extends WriteHandleFactory { private final HoodieWriteHandle writeHandle; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkAppendHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkAppendHandle.java index 2258375fdd164..4b56d6a442c3a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkAppendHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkAppendHandle.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.marker.WriteMarkers; @@ -44,7 +43,7 @@ *

The back-up writer may rollover on condition(for e.g, the filesystem does not support append * or the file size hits the configured threshold). */ -public class FlinkAppendHandle +public class FlinkAppendHandle extends HoodieAppendHandle implements MiniBatchHandle { private static final Logger LOG = LoggerFactory.getLogger(FlinkAppendHandle.class); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkConcatAndReplaceHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkConcatAndReplaceHandle.java index 300e8c512bb34..6ba7ac4d00524 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkConcatAndReplaceHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkConcatAndReplaceHandle.java @@ -18,16 +18,13 @@ package org.apache.hudi.io; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; -import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.HoodieTable; - -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,7 +38,7 @@ *

The records iterator for super constructor is reset as empty thus the initialization for new records * does nothing. This handle keep the iterator for itself to override the write behavior. */ -public class FlinkConcatAndReplaceHandle +public class FlinkConcatAndReplaceHandle extends FlinkMergeAndReplaceHandle { private static final Logger LOG = LoggerFactory.getLogger(FlinkConcatAndReplaceHandle.class); @@ -59,10 +56,11 @@ public FlinkConcatAndReplaceHandle(HoodieWriteConfig config, String instantTime, * Write old record as is w/o merging with incoming record. */ @Override - public void write(GenericRecord oldRecord) { - String key = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt); + public void write(HoodieRecord oldRecord) { + Schema oldSchema = config.populateMetaFields() ? writeSchemaWithMetaFields : writeSchema; + String key = oldRecord.getRecordKey(oldSchema, keyGeneratorOpt); try { - fileWriter.writeAvro(key, oldRecord); + fileWriter.write(key, oldRecord, writeSchema); } catch (IOException | RuntimeException e) { String errMsg = String.format("Failed to write old record into new file for key %s from old file %s to new file %s with writerSchema %s", key, getOldFilePath(), newFilePath, writeSchemaWithMetaFields.toString(true)); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkConcatHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkConcatHandle.java index 812155c3d2fb0..4f5f522df401d 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkConcatHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkConcatHandle.java @@ -18,15 +18,12 @@ package org.apache.hudi.io; +import org.apache.avro.Schema; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; -import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.HoodieTable; - -import org.apache.avro.generic.GenericRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,7 +37,7 @@ *

The records iterator for super constructor is reset as empty thus the initialization for new records * does nothing. This handle keep the iterator for itself to override the write behavior. */ -public class FlinkConcatHandle +public class FlinkConcatHandle extends FlinkMergeHandle { private static final Logger LOG = LoggerFactory.getLogger(FlinkConcatHandle.class); @@ -58,13 +55,14 @@ public FlinkConcatHandle(HoodieWriteConfig config, String instantTime, HoodieTab * Write old record as is w/o merging with incoming record. */ @Override - public void write(GenericRecord oldRecord) { - String key = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt); + public void write(HoodieRecord oldRecord) { + Schema oldSchema = config.populateMetaFields() ? writeSchemaWithMetaFields : writeSchema; + String key = oldRecord.getRecordKey(oldSchema, keyGeneratorOpt); try { - fileWriter.writeAvro(key, oldRecord); + fileWriter.write(key, oldRecord, oldSchema); } catch (IOException | RuntimeException e) { String errMsg = String.format("Failed to write old record into new file for key %s from old file %s to new file %s with writerSchema %s", - key, getOldFilePath(), newFilePath, writeSchemaWithMetaFields.toString(true)); + key, getOldFilePath(), newFilePath, oldSchema.toString(true)); LOG.debug("Old record is " + oldRecord); throw new HoodieUpsertException(errMsg, e); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java index 777e228c9510d..460329f446a26 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java @@ -22,7 +22,6 @@ 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; @@ -48,7 +47,7 @@ * * @see FlinkMergeAndReplaceHandle */ -public class FlinkCreateHandle +public class FlinkCreateHandle extends HoodieCreateHandle implements MiniBatchHandle { private static final Logger LOG = LogManager.getLogger(FlinkCreateHandle.class); 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 9fea0a97185cb..cc987b16bf834 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 @@ -23,7 +23,6 @@ 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; @@ -50,7 +49,7 @@ * then closes the file and rename to the old file name, * behaves like the new data buffer are appended to the old file. */ -public class FlinkMergeAndReplaceHandle +public class FlinkMergeAndReplaceHandle extends HoodieMergeHandle implements MiniBatchHandle { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandleWithChangeLog.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandleWithChangeLog.java index 52ae8abd81b5a..9604428684e08 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandleWithChangeLog.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandleWithChangeLog.java @@ -20,19 +20,23 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.cdc.HoodieCDCUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; -import org.apache.avro.generic.GenericRecord; +import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.Path; +import java.io.IOException; import java.util.Iterator; import java.util.List; @@ -42,8 +46,11 @@ *

The cdc about logic is copied from {@link HoodieMergeHandleWithChangeLog}, * we should refactor it out when there are good abstractions. */ -public class FlinkMergeAndReplaceHandleWithChangeLog +public class FlinkMergeAndReplaceHandleWithChangeLog extends FlinkMergeAndReplaceHandle { + + private static final Logger LOG = LogManager.getLogger(FlinkMergeAndReplaceHandleWithChangeLog.class); + private final HoodieCDCLogger cdcLogger; public FlinkMergeAndReplaceHandleWithChangeLog(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, @@ -61,19 +68,28 @@ public FlinkMergeAndReplaceHandleWithChangeLog(HoodieWriteConfig config, String IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config)); } - protected boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option indexedRecord) { - final boolean result = super.writeUpdateRecord(hoodieRecord, oldRecord, indexedRecord); + protected boolean writeUpdateRecord(HoodieRecord newRecord, HoodieRecord oldRecord, Option combineRecordOpt, Schema writerSchema) + throws IOException { + // TODO [HUDI-5019] Remove these unnecessary newInstance invocations + Option savedCombineRecordOp = combineRecordOpt.map(HoodieRecord::newInstance); + final boolean result = super.writeUpdateRecord(newRecord, oldRecord, combineRecordOpt, writerSchema); if (result) { - boolean isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation()); - cdcLogger.put(hoodieRecord, oldRecord, isDelete ? Option.empty() : indexedRecord); + boolean isDelete = HoodieOperation.isDelete(newRecord.getOperation()); + Option avroRecordOpt = savedCombineRecordOp.flatMap(r -> + toAvroRecord(r, writerSchema, config.getPayloadConfig().getProps())); + cdcLogger.put(newRecord, (GenericRecord) oldRecord.getData(), isDelete ? Option.empty() : avroRecordOpt); } return result; } - protected void writeInsertRecord(HoodieRecord hoodieRecord, Option insertRecord) { - super.writeInsertRecord(hoodieRecord, insertRecord); - if (!HoodieOperation.isDelete(hoodieRecord.getOperation())) { - cdcLogger.put(hoodieRecord, null, insertRecord); + protected void writeInsertRecord(HoodieRecord newRecord) throws IOException { + Schema schema = useWriterSchemaForCompaction ? writeSchemaWithMetaFields : writeSchema; + // TODO Remove these unnecessary newInstance invocations + HoodieRecord savedRecord = newRecord.newInstance(); + super.writeInsertRecord(newRecord); + if (!HoodieOperation.isDelete(newRecord.getOperation())) { + cdcLogger.put(newRecord, null, savedRecord.toIndexedRecord(schema, config.getPayloadConfig().getProps()).map(HoodieAvroIndexedRecord::getData)); + newRecord.deflate(); } } 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 a44783f99e437..9a508e0b1f2a8 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 @@ -22,7 +22,6 @@ 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; @@ -51,7 +50,7 @@ * * @see FlinkMergeAndReplaceHandle */ -public class FlinkMergeHandle +public class FlinkMergeHandle extends HoodieMergeHandle implements MiniBatchHandle { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandleWithChangeLog.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandleWithChangeLog.java index fe845966c1594..4fc389a349a0b 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandleWithChangeLog.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandleWithChangeLog.java @@ -20,20 +20,22 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.cdc.HoodieCDCUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.avro.generic.GenericRecord; +import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.generic.GenericRecord; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.IOException; import java.util.Iterator; import java.util.List; @@ -43,7 +45,7 @@ *

The cdc about logic is copied from {@link HoodieMergeHandleWithChangeLog}, * we should refactor it out when there are good abstractions. */ -public class FlinkMergeHandleWithChangeLog +public class FlinkMergeHandleWithChangeLog extends FlinkMergeHandle { private final HoodieCDCLogger cdcLogger; @@ -64,19 +66,27 @@ public FlinkMergeHandleWithChangeLog(HoodieWriteConfig config, String instantTim IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config)); } - protected boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option indexedRecord) { - final boolean result = super.writeUpdateRecord(hoodieRecord, oldRecord, indexedRecord); + protected boolean writeUpdateRecord(HoodieRecord newRecord, HoodieRecord oldRecord, Option combineRecordOpt, Schema writerSchema) + throws IOException { + // TODO [HUDI-5019] Remove these unnecessary newInstance invocations + Option savedCombineRecordOp = combineRecordOpt.map(HoodieRecord::newInstance); + final boolean result = super.writeUpdateRecord(newRecord, oldRecord, combineRecordOpt, writerSchema); if (result) { - boolean isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation()); - cdcLogger.put(hoodieRecord, oldRecord, isDelete ? Option.empty() : indexedRecord); + boolean isDelete = HoodieOperation.isDelete(newRecord.getOperation()); + Option avroRecordOpt = savedCombineRecordOp.flatMap(r -> + toAvroRecord(r, writerSchema, config.getPayloadConfig().getProps())); + cdcLogger.put(newRecord, (GenericRecord) oldRecord.getData(), isDelete ? Option.empty() : avroRecordOpt); } return result; } - protected void writeInsertRecord(HoodieRecord hoodieRecord, Option insertRecord) { - super.writeInsertRecord(hoodieRecord, insertRecord); - if (!HoodieOperation.isDelete(hoodieRecord.getOperation())) { - cdcLogger.put(hoodieRecord, null, insertRecord); + protected void writeInsertRecord(HoodieRecord newRecord) throws IOException { + Schema schema = useWriterSchemaForCompaction ? writeSchemaWithMetaFields : writeSchema; + // TODO Remove these unnecessary newInstance invocations + HoodieRecord savedRecord = newRecord.newInstance(); + super.writeInsertRecord(newRecord); + if (!HoodieOperation.isDelete(newRecord.getOperation())) { + cdcLogger.put(newRecord, null, savedRecord.toIndexedRecord(schema, config.getPayloadConfig().getProps()).map(HoodieAvroIndexedRecord::getData)); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkWriteHandleFactory.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkWriteHandleFactory.java index fbc1c7ec55fa9..9759d84cae15e 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkWriteHandleFactory.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkWriteHandleFactory.java @@ -20,7 +20,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -39,7 +38,7 @@ public class FlinkWriteHandleFactory { /** * Returns the write handle factory with given write config. */ - public static Factory getFactory( + public static Factory getFactory( HoodieTableConfig tableConfig, HoodieWriteConfig writeConfig) { if (writeConfig.allowDuplicateInserts()) { @@ -58,7 +57,7 @@ public static Factory getFa // Inner Class // ------------------------------------------------------------------------- - public interface Factory { + public interface Factory { /** * Get or create a new write handle in order to reuse the file handles. * @@ -87,7 +86,7 @@ public interface Factory { * Base clazz for commit write handle factory, * it encapsulates the handle switching logic: INSERT OR UPSERT. */ - private abstract static class BaseCommitWriteHandleFactory implements Factory { + private abstract static class BaseCommitWriteHandleFactory implements Factory { @Override public HoodieWriteHandle create( Map> bucketToHandles, @@ -140,12 +139,12 @@ private abstract static class BaseCommitWriteHandleFactory + private static class CommitWriteHandleFactory extends BaseCommitWriteHandleFactory { private static final CommitWriteHandleFactory INSTANCE = new CommitWriteHandleFactory<>(); @SuppressWarnings("unchecked") - public static CommitWriteHandleFactory getInstance() { + public static CommitWriteHandleFactory getInstance() { return (CommitWriteHandleFactory) INSTANCE; } @@ -178,12 +177,12 @@ public static CommitWriteHandleFactory< /** * Write handle factory for inline clustering. */ - private static class ClusterWriteHandleFactory + private static class ClusterWriteHandleFactory extends BaseCommitWriteHandleFactory { private static final ClusterWriteHandleFactory INSTANCE = new ClusterWriteHandleFactory<>(); @SuppressWarnings("unchecked") - public static ClusterWriteHandleFactory getInstance() { + public static ClusterWriteHandleFactory getInstance() { return (ClusterWriteHandleFactory) INSTANCE; } @@ -216,12 +215,12 @@ public static ClusterWriteHandleFactory /** * Write handle factory for commit, the write handle supports logging change logs. */ - private static class CdcWriteHandleFactory + private static class CdcWriteHandleFactory extends BaseCommitWriteHandleFactory { private static final CdcWriteHandleFactory INSTANCE = new CdcWriteHandleFactory<>(); @SuppressWarnings("unchecked") - public static CdcWriteHandleFactory getInstance() { + public static CdcWriteHandleFactory getInstance() { return (CdcWriteHandleFactory) INSTANCE; } @@ -254,11 +253,11 @@ public static CdcWriteHandleFactory implements Factory { + private static class DeltaCommitWriteHandleFactory implements Factory { private static final DeltaCommitWriteHandleFactory INSTANCE = new DeltaCommitWriteHandleFactory<>(); @SuppressWarnings("unchecked") - public static DeltaCommitWriteHandleFactory getInstance() { + public static DeltaCommitWriteHandleFactory getInstance() { return (DeltaCommitWriteHandleFactory) INSTANCE; } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/ExplicitWriteHandleTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/ExplicitWriteHandleTable.java index b95894bed8d54..4145c9630cd7f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/ExplicitWriteHandleTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/ExplicitWriteHandleTable.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.io.HoodieWriteHandle; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -32,7 +31,7 @@ * HoodieTable that need to pass in the * {@link org.apache.hudi.io.HoodieWriteHandle} explicitly. */ -public interface ExplicitWriteHandleTable { +public interface ExplicitWriteHandleTable { /** * Upsert a batch of new records into Hoodie table at the supplied instantTime. * 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 cb38cb876dae7..8e671b2ec5e56 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 @@ -35,7 +35,6 @@ 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.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; @@ -83,7 +82,7 @@ *

* UPDATES - Produce a new version of the file, just replacing the updated records with new values */ -public class HoodieFlinkCopyOnWriteTable +public class HoodieFlinkCopyOnWriteTable extends HoodieFlinkTable implements HoodieCompactionHandler { private static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkCopyOnWriteTable.class); @@ -402,7 +401,7 @@ protected HoodieMergeHandle getUpdateHandle(String instantTime, String partition @Override public Iterator> handleInsert( String instantTime, String partitionPath, String fileId, - Map> recordMap) { + Map> recordMap) { HoodieCreateHandle createHandle = new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier); createHandle.write(); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java index b4595439cc519..79261da2f583e 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java @@ -24,7 +24,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -48,7 +47,7 @@ /** * Flink MERGE_ON_READ table. */ -public class HoodieFlinkMergeOnReadTable +public class HoodieFlinkMergeOnReadTable extends HoodieFlinkCopyOnWriteTable { HoodieFlinkMergeOnReadTable( diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index 40a13c14d5bbc..3302b70214e90 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -46,7 +45,7 @@ /** * Impl of a flink hoodie table. */ -public abstract class HoodieFlinkTable +public abstract class HoodieFlinkTable extends HoodieTable>, List, List> implements ExplicitWriteHandleTable { @@ -54,7 +53,7 @@ protected HoodieFlinkTable(HoodieWriteConfig config, HoodieEngineContext context super(config, context, metaClient); } - public static HoodieFlinkTable create(HoodieWriteConfig config, HoodieFlinkEngineContext context) { + public static HoodieFlinkTable create(HoodieWriteConfig config, HoodieFlinkEngineContext context) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) @@ -66,9 +65,9 @@ public static HoodieFlinkTable create(HoodieW return HoodieFlinkTable.create(config, context, metaClient); } - public static HoodieFlinkTable create(HoodieWriteConfig config, - HoodieFlinkEngineContext context, - HoodieTableMetaClient metaClient) { + public static HoodieFlinkTable create(HoodieWriteConfig config, + HoodieFlinkEngineContext context, + HoodieTableMetaClient metaClient) { final HoodieFlinkTable hoodieFlinkTable; switch (metaClient.getTableType()) { case COPY_ON_WRITE: diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java index f7527b2766071..d6c5cb8e291a8 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -67,7 +66,7 @@ *

Computing the records batch locations all at a time is a pressure to the engine, * we should avoid that in streaming system. */ -public abstract class BaseFlinkCommitActionExecutor extends +public abstract class BaseFlinkCommitActionExecutor extends BaseCommitActionExecutor>, List, List, HoodieWriteMetadata> { private static final Logger LOG = LogManager.getLogger(BaseFlinkCommitActionExecutor.class); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java index 23e3c01eac171..115d9555e2361 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieWriteHandle; @@ -33,7 +32,7 @@ /** * Flink delete commit action executor. */ -public class FlinkDeleteCommitActionExecutor> extends BaseFlinkCommitActionExecutor { +public class FlinkDeleteCommitActionExecutor extends BaseFlinkCommitActionExecutor { private final List keys; public FlinkDeleteCommitActionExecutor(HoodieEngineContext context, diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java index 3ae6802e7ce7a..387aae29769bb 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieWriteHandle; @@ -33,7 +32,7 @@ /** * Flink insert commit action executor. */ -public class FlinkInsertCommitActionExecutor> extends BaseFlinkCommitActionExecutor { +public class FlinkInsertCommitActionExecutor extends BaseFlinkCommitActionExecutor { private List> inputRecords; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java index 5f6d06ea544da..d4fdbffa0ed88 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; @@ -34,7 +33,7 @@ /** * Flink INSERT OVERWRITE commit action executor. */ -public class FlinkInsertOverwriteCommitActionExecutor> +public class FlinkInsertOverwriteCommitActionExecutor extends BaseFlinkCommitActionExecutor { protected List> inputRecords; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java index f52b2d9c98339..d1149614c4825 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieWriteHandle; @@ -33,7 +32,7 @@ /** * Flink INSERT OVERWRITE TABLE commit action executor. */ -public class FlinkInsertOverwriteTableCommitActionExecutor> +public class FlinkInsertOverwriteTableCommitActionExecutor extends FlinkInsertOverwriteCommitActionExecutor { public FlinkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context, diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java index 240b04d7eb6fd..78c63f15a6516 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieWriteHandle; @@ -33,7 +32,7 @@ /** * Flink insert prepped commit action executor. */ -public class FlinkInsertPreppedCommitActionExecutor> extends BaseFlinkCommitActionExecutor { +public class FlinkInsertPreppedCommitActionExecutor extends BaseFlinkCommitActionExecutor { private final List> preppedRecords; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java index 1e7dbb85b1971..7194593e2a6dd 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieWriteHandle; @@ -33,7 +32,7 @@ /** * Flink upsert commit action executor. */ -public class FlinkUpsertCommitActionExecutor> extends BaseFlinkCommitActionExecutor { +public class FlinkUpsertCommitActionExecutor extends BaseFlinkCommitActionExecutor { private List> inputRecords; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java index 8fb7bc6de2c8f..1865cb0c4d519 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieWriteHandle; @@ -33,7 +32,7 @@ /** * Flink upsert prepped commit action executor. */ -public class FlinkUpsertPreppedCommitActionExecutor> extends BaseFlinkCommitActionExecutor { +public class FlinkUpsertPreppedCommitActionExecutor extends BaseFlinkCommitActionExecutor { private final List> preppedRecords; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java index ee57ecb29acc7..8855457684993 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java @@ -18,23 +18,23 @@ package org.apache.hudi.table.action.commit; +import org.apache.avro.Schema; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.avro.Schema; - +import java.io.IOException; import java.time.Duration; import java.time.Instant; import java.util.List; @@ -51,7 +51,7 @@ *

Computing the records batch locations all at a time is a pressure to the engine, * we should avoid that in streaming system. */ -public class FlinkWriteHelper extends BaseWriteHelper>, +public class FlinkWriteHelper extends BaseWriteHelper>, List, List, R> { private FlinkWriteHelper() { @@ -90,8 +90,8 @@ protected List> tag(List> dedupedRecords, Hoodie } @Override - public List> deduplicateRecords( - List> records, HoodieIndex index, int parallelism, String schemaStr) { + protected List> doDeduplicateRecords( + List> records, HoodieIndex index, int parallelism, String schemaStr, TypedProperties props, HoodieRecordMerger merger) { // If index used is global, then records are expected to differ in their partitionPath Map>> keyedRecords = records.stream() .collect(Collectors.groupingBy(record -> record.getKey().getRecordKey())); @@ -99,17 +99,20 @@ public List> deduplicateRecords( // caution that the avro schema is not serializable final Schema schema = new Schema.Parser().parse(schemaStr); return keyedRecords.values().stream().map(x -> x.stream().reduce((rec1, rec2) -> { - final T data1 = rec1.getData(); - final T data2 = rec2.getData(); - - @SuppressWarnings("unchecked") final T reducedData = (T) data2.preCombine(data1, schema, CollectionUtils.emptyProps()); + HoodieRecord reducedRecord; + try { + // Precombine do not need schema and do not return null + reducedRecord = merger.merge(rec1, schema, rec2, schema, props).get().getLeft(); + } catch (IOException e) { + throw new HoodieException(String.format("Error to merge two records, %s, %s", rec1, rec2), e); + } // we cannot allow the user to change the key or partitionPath, since that will affect // everything // so pick it from one of the records. - boolean choosePrev = data1 == reducedData; + boolean choosePrev = rec1 == reducedRecord; HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey(); HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation(); - HoodieRecord hoodieRecord = new HoodieAvroRecord<>(reducedKey, reducedData, operation); + HoodieRecord hoodieRecord = reducedRecord.newInstance(reducedKey, operation); // reuse the location from the first record. hoodieRecord.setCurrentLocation(rec1.getCurrentLocation()); return hoodieRecord; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/BaseFlinkDeltaCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/BaseFlinkDeltaCommitActionExecutor.java index d8ea958266b82..3fd8e23098092 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/BaseFlinkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/BaseFlinkDeltaCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.execution.FlinkLazyInsertIterable; @@ -37,7 +36,7 @@ /** * Base flink delta commit action executor. */ -public abstract class BaseFlinkDeltaCommitActionExecutor> +public abstract class BaseFlinkDeltaCommitActionExecutor extends BaseFlinkCommitActionExecutor { public BaseFlinkDeltaCommitActionExecutor(HoodieEngineContext context, diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java index c95a6c1c7b9f0..7e17e3883209e 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java @@ -20,7 +20,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.FlinkAppendHandle; @@ -33,7 +32,7 @@ /** * Flink upsert delta commit action executor. */ -public class FlinkUpsertDeltaCommitActionExecutor> +public class FlinkUpsertDeltaCommitActionExecutor extends BaseFlinkDeltaCommitActionExecutor { private final List> inputRecords; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertPreppedDeltaCommitActionExecutor.java index 94bcbc586c2e2..683c7726c2453 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertPreppedDeltaCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.FlinkAppendHandle; @@ -33,7 +32,7 @@ /** * Flink upsert prepped delta commit action executor. */ -public class FlinkUpsertPreppedDeltaCommitActionExecutor> +public class FlinkUpsertPreppedDeltaCommitActionExecutor extends BaseFlinkDeltaCommitActionExecutor { private final List> preppedRecords; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java index d47da217169ef..8b99ed815bd5a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -39,7 +38,7 @@ *

Note: the compaction logic is invoked through the flink pipeline. */ @SuppressWarnings("checkstyle:LineLength") -public class HoodieFlinkMergeOnReadTableCompactor +public class HoodieFlinkMergeOnReadTableCompactor extends HoodieCompactor>, List, List> { @Override diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java index cd4096663f332..e0c5e9cdabb48 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java @@ -23,6 +23,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.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -144,7 +145,7 @@ private Pair appendRecordsToLogFile(List gr LOG.warn("Failed to convert record " + r.toString(), e); return null; } - }).collect(Collectors.toList()), header, HoodieRecord.RECORD_KEY_METADATA_FIELD)); + }).map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()), header, HoodieRecord.RECORD_KEY_METADATA_FIELD)); return Pair.of(partitionPath, logWriter.getLogFile()); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index 1f205d0e82397..041d994623c41 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -49,7 +48,7 @@ import java.util.function.BiConsumer; import java.util.stream.Collectors; -public class HoodieJavaWriteClient extends +public class HoodieJavaWriteClient extends BaseHoodieWriteClient>, List, List> { public HoodieJavaWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java index b2291652413c6..739c2ed48bc8b 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java @@ -26,7 +26,6 @@ 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.HoodieRecordPayload; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -48,7 +47,7 @@ * 1) Creates clustering groups based on max size allowed per group. * 2) Excludes files that are greater than 'small.file.limit' from clustering plan. */ -public class JavaSizeBasedClusteringPlanStrategy> +public class JavaSizeBasedClusteringPlanStrategy extends PartitionAwareClusteringPlanStrategy>, List, List> { private static final Logger LOG = LogManager.getLogger(JavaSizeBasedClusteringPlanStrategy.class); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java index c6f885fa9160a..bd7dfd79eddef 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java @@ -19,6 +19,8 @@ package org.apache.hudi.client.clustering.run.strategy; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -28,12 +30,9 @@ import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.ClusteringOperation; -import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.RewriteAvroPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.util.Option; @@ -46,17 +45,10 @@ 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.BaseKeyGenerator; -import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -65,6 +57,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.stream.Collectors; import static org.apache.hudi.common.table.log.HoodieFileSliceReader.getFileSliceReader; @@ -73,7 +66,7 @@ /** * Clustering strategy for Java engine. */ -public abstract class JavaExecutionStrategy> +public abstract class JavaExecutionStrategy extends ClusteringExecutionStrategy>, HoodieData, HoodieData> { private static final Logger LOG = LogManager.getLogger(JavaExecutionStrategy.class); @@ -195,15 +188,15 @@ private List> readRecordsForGroupWithLogs(List> fileSliceReader = getFileSliceReader(baseFileReader, scanner, readerSchema, - tableConfig.getPayloadClass(), - tableConfig.getPreCombineField(), + tableConfig.getProps(), tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp()))); fileSliceReader.forEachRemaining(records::add); @@ -228,10 +221,13 @@ private List> readRecordsForGroupWithLogs(List> readRecordsForGroupBaseFiles(List clusteringOps) { List> records = new ArrayList<>(); clusteringOps.forEach(clusteringOp -> { - try (HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath()))) { + try (HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath()))) { Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); - Iterator recordIterator = baseFileReader.getRecordIterator(readerSchema); - recordIterator.forEachRemaining(record -> records.add(transform(record))); + Iterator recordIterator = baseFileReader.getRecordIterator(readerSchema); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into the records(List). + recordIterator.forEachRemaining(record -> records.add(record.copy().wrapIntoHoodieRecordPayloadWithKeyGen(readerSchema, new Properties(), Option.empty()))); } catch (IOException e) { throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + " and " + clusteringOp.getDeltaFilePaths(), e); @@ -239,19 +235,4 @@ private List> readRecordsForGroupBaseFiles(List transform(IndexedRecord indexedRecord) { - GenericRecord record = (GenericRecord) indexedRecord; - Option keyGeneratorOpt = Option.empty(); - String key = KeyGenUtils.getRecordKeyFromGenericRecord(record, keyGeneratorOpt); - String partition = KeyGenUtils.getPartitionPathFromGenericRecord(record, keyGeneratorOpt); - HoodieKey hoodieKey = new HoodieKey(key, partition); - - HoodieRecordPayload avroPayload = new RewriteAvroPayload(record); - HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, avroPayload); - return hoodieRecord; - } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java index d34673c2d9b9a..9e0a675fceb77 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java @@ -24,8 +24,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.table.HoodieTable; @@ -43,7 +42,7 @@ * 1) Java execution engine. * 2) Uses bulk_insert to write data into new files. */ -public class JavaSortAndSizeExecutionStrategy> +public class JavaSortAndSizeExecutionStrategy extends JavaExecutionStrategy { private static final Logger LOG = LogManager.getLogger(JavaSortAndSizeExecutionStrategy.class); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java index bc07513450936..b5b131e64ed7a 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.HoodieExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -35,7 +34,7 @@ import static org.apache.hudi.common.util.ValidationUtils.checkState; -public class JavaLazyInsertIterable extends HoodieLazyInsertIterable { +public class JavaLazyInsertIterable extends HoodieLazyInsertIterable { public JavaLazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, HoodieWriteConfig config, @@ -65,7 +64,7 @@ protected List computeNext() { try { final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); bufferedIteratorExecutor = - QueueBasedExecutorFactory.create(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema)); + QueueBasedExecutorFactory.create(hoodieConfig, inputItr, getInsertHandler(), getCloningTransformer(schema)); final List result = bufferedIteratorExecutor.execute(); checkState(result != null && !result.isEmpty()); return result; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java index b9e466485f209..540cac2ab777a 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java @@ -20,8 +20,8 @@ package org.apache.hudi.execution.bulkinsert; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.avro.Schema; @@ -34,7 +34,7 @@ * * @param HoodieRecordPayload type */ -public class JavaCustomColumnsSortPartitioner +public class JavaCustomColumnsSortPartitioner implements BulkInsertPartitioner>> { private final String[] sortColumnNames; @@ -51,8 +51,8 @@ public JavaCustomColumnsSortPartitioner(String[] columnNames, Schema schema, boo public List> repartitionRecords( List> records, int outputPartitions) { return records.stream().sorted((o1, o2) -> { - Object values1 = HoodieAvroUtils.getRecordColumnValues(o1, sortColumnNames, schema, consistentLogicalTimestampEnabled); - Object values2 = HoodieAvroUtils.getRecordColumnValues(o2, sortColumnNames, schema, consistentLogicalTimestampEnabled); + Object values1 = HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)o1, sortColumnNames, schema, consistentLogicalTimestampEnabled); + Object values2 = HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)o2, sortColumnNames, schema, consistentLogicalTimestampEnabled); return values1.toString().compareTo(values2.toString()); }).collect(Collectors.toList()); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java index d272849a19f28..5317914a9cded 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java @@ -19,7 +19,6 @@ package org.apache.hudi.execution.bulkinsert; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.table.BulkInsertPartitioner; import java.util.Comparator; @@ -32,7 +31,7 @@ * * @param HoodieRecordPayload type */ -public class JavaGlobalSortPartitioner +public class JavaGlobalSortPartitioner implements BulkInsertPartitioner>> { @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaNonSortPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaNonSortPartitioner.java index b40459d838444..acd9f4621a126 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaNonSortPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaNonSortPartitioner.java @@ -19,7 +19,6 @@ package org.apache.hudi.execution.bulkinsert; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.table.BulkInsertPartitioner; import java.util.List; @@ -30,7 +29,7 @@ * * @param HoodieRecordPayload type */ -public class JavaNonSortPartitioner +public class JavaNonSortPartitioner implements BulkInsertPartitioner>> { @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java index dcc9d050dcbe5..f8efb141fdfe4 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.table.HoodieTable; @@ -34,7 +33,7 @@ import java.util.List; import java.util.stream.Collectors; -public abstract class JavaHoodieIndex extends HoodieIndex>, List> { +public abstract class JavaHoodieIndex extends HoodieIndex>, List> { protected JavaHoodieIndex(HoodieWriteConfig config) { super(config); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index efa96e2eddba9..2cf2895f3f4d7 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -35,7 +35,6 @@ 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.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; @@ -78,7 +77,7 @@ import java.util.List; import java.util.Map; -public class HoodieJavaCopyOnWriteTable +public class HoodieJavaCopyOnWriteTable extends HoodieJavaTable implements HoodieCompactionHandler { private static final Logger LOG = LoggerFactory.getLogger(HoodieJavaCopyOnWriteTable.class); @@ -300,7 +299,7 @@ protected HoodieMergeHandle getUpdateHandle(String instantTime, String partition @Override public Iterator> handleInsert( String instantTime, String partitionPath, String fileId, - Map> recordMap) { + Map> recordMap) { HoodieCreateHandle createHandle = new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier); createHandle.write(); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java index 2f5f291087310..abacfbb3c1317 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java @@ -23,7 +23,6 @@ import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; @@ -38,7 +37,7 @@ import java.util.List; import java.util.Map; -public class HoodieJavaMergeOnReadTable extends HoodieJavaCopyOnWriteTable { +public class HoodieJavaMergeOnReadTable extends HoodieJavaCopyOnWriteTable { protected HoodieJavaMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java index 3c878cbc14cf8..7b25f389cf803 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; @@ -36,13 +35,13 @@ import java.util.List; -public abstract class HoodieJavaTable +public abstract class HoodieJavaTable extends HoodieTable>, List, List> { protected HoodieJavaTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); } - public static HoodieJavaTable create(HoodieWriteConfig config, HoodieEngineContext context) { + public static HoodieJavaTable create(HoodieWriteConfig config, HoodieEngineContext context) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) @@ -50,7 +49,7 @@ public static HoodieJavaTable create(HoodieWr return HoodieJavaTable.create(config, (HoodieJavaEngineContext) context, metaClient); } - public static HoodieJavaTable create(HoodieWriteConfig config, + public static HoodieJavaTable create(HoodieWriteConfig config, HoodieJavaEngineContext context, HoodieTableMetaClient metaClient) { switch (metaClient.getTableType()) { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java index 168d558143bd3..717f8446a2b01 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java @@ -23,7 +23,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.ClusteringUtils; @@ -36,7 +35,7 @@ import java.util.List; -public class JavaExecuteClusteringCommitActionExecutor> +public class JavaExecuteClusteringCommitActionExecutor extends BaseJavaCommitActionExecutor { private final HoodieClusteringPlan clusteringPlan; 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 d9df257f1b2b9..6e5bf5d4d57da 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 @@ -25,7 +25,6 @@ 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.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -63,7 +62,7 @@ import java.util.Map; import java.util.stream.Collectors; -public abstract class BaseJavaCommitActionExecutor extends +public abstract class BaseJavaCommitActionExecutor extends BaseCommitActionExecutor>, List, List, HoodieWriteMetadata> { private static final Logger LOG = LogManager.getLogger(BaseJavaCommitActionExecutor.class); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java index d5c7a0b0b5dc7..0a4b6a65f2843 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -33,7 +32,7 @@ import java.util.List; import java.util.Map; -public class JavaBulkInsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { +public class JavaBulkInsertCommitActionExecutor extends BaseJavaCommitActionExecutor { private final List> inputRecords; private final Option bulkInsertPartitioner; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java index e126372aa9068..0c76ea168a748 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java @@ -22,7 +22,6 @@ 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.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; @@ -45,7 +44,7 @@ * @param */ @SuppressWarnings("checkstyle:LineLength") -public class JavaBulkInsertHelper extends BaseBulkInsertHelper>, +public class JavaBulkInsertHelper extends BaseBulkInsertHelper>, List, List, R> { private JavaBulkInsertHelper() { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java index 14c4c8a93e916..b7a581160535f 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -32,7 +31,7 @@ import java.util.List; -public class JavaBulkInsertPreppedCommitActionExecutor> +public class JavaBulkInsertPreppedCommitActionExecutor extends BaseJavaCommitActionExecutor { private final List> preppedInputRecord; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java index 72c2332645cf3..13cd8e5945a98 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -29,7 +28,7 @@ import java.util.List; -public class JavaDeleteCommitActionExecutor> extends BaseJavaCommitActionExecutor { +public class JavaDeleteCommitActionExecutor extends BaseJavaCommitActionExecutor { private final List keys; public JavaDeleteCommitActionExecutor(HoodieEngineContext context, diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java index c1fae07a9d6c2..ec4f987df66c4 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -29,7 +28,7 @@ import java.util.List; -public class JavaInsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { +public class JavaInsertCommitActionExecutor extends BaseJavaCommitActionExecutor { private List> inputRecords; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java index a99485e67bb81..1b64adf5b7a02 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.collection.Pair; @@ -33,7 +32,7 @@ import java.util.Map; import java.util.stream.Collectors; -public class JavaInsertOverwriteCommitActionExecutor> +public class JavaInsertOverwriteCommitActionExecutor extends BaseJavaCommitActionExecutor { private final List> inputRecords; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java index a52ab6e0f3d0c..3b4131e823505 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; 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.model.WriteOperationType; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -34,7 +33,7 @@ import java.util.Map; import java.util.stream.Collectors; -public class JavaInsertOverwriteTableCommitActionExecutor> +public class JavaInsertOverwriteTableCommitActionExecutor extends JavaInsertOverwriteCommitActionExecutor { public JavaInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context, diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java index 349cf69dcc30b..7f3bcc4d8d70c 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -30,7 +29,7 @@ import java.util.List; -public class JavaInsertPreppedCommitActionExecutor> +public class JavaInsertPreppedCommitActionExecutor extends BaseJavaCommitActionExecutor { private final List> preppedRecords; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java index ed0af4402869d..34ec4e792c71f 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -29,7 +28,7 @@ import java.util.List; -public class JavaUpsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { +public class JavaUpsertCommitActionExecutor extends BaseJavaCommitActionExecutor { private List> inputRecords; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java index fb19259b55591..40cd2eb57728c 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; 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.model.HoodieWriteStat; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -52,7 +51,7 @@ /** * Packs incoming records to be upserted, into buckets. */ -public class JavaUpsertPartitioner> implements Partitioner { +public class JavaUpsertPartitioner implements Partitioner { private static final Logger LOG = LogManager.getLogger(JavaUpsertPartitioner.class); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java index 8eea5b5105826..23fcf1f9ca7ae 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -30,7 +29,7 @@ import java.util.List; -public class JavaUpsertPreppedCommitActionExecutor> +public class JavaUpsertPreppedCommitActionExecutor extends BaseJavaCommitActionExecutor { private final List> preppedRecords; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java index 6294b046d87c6..dc109f8103012 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java @@ -18,26 +18,26 @@ package org.apache.hudi.table.action.commit; +import org.apache.avro.Schema; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; -import org.apache.avro.Schema; - +import java.io.IOException; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; -public class JavaWriteHelper extends BaseWriteHelper>, +public class JavaWriteHelper extends BaseWriteHelper>, List, List, R> { private JavaWriteHelper() { @@ -57,8 +57,8 @@ protected List> tag(List> dedupedRecords, Hoodie } @Override - public List> deduplicateRecords( - List> records, HoodieIndex index, int parallelism, String schemaStr) { + protected List> doDeduplicateRecords( + List> records, HoodieIndex index, int parallelism, String schemaStr, TypedProperties props, HoodieRecordMerger merger) { boolean isIndexingGlobal = index.isGlobal(); Map>>> keyedRecords = records.stream().map(record -> { HoodieKey hoodieKey = record.getKey(); @@ -69,12 +69,16 @@ public List> deduplicateRecords( final Schema schema = new Schema.Parser().parse(schemaStr); return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> { - @SuppressWarnings("unchecked") - T reducedData = (T) rec1.getData().preCombine(rec2.getData(), schema, CollectionUtils.emptyProps()); + HoodieRecord reducedRecord; + try { + reducedRecord = merger.merge(rec1, schema, rec2, schema, props).get().getLeft(); + } catch (IOException e) { + throw new HoodieException(String.format("Error to merge two records, %s, %s", rec1, rec2), e); + } // we cannot allow the user to change the key or partitionPath, since that will affect // everything // so pick it from one of the records. - return new HoodieAvroRecord(rec1.getKey(), reducedData); + return reducedRecord.newInstance(rec1.getKey()); }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList()); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java index 6dc65649a1d40..97e26e7c41da9 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -37,7 +36,7 @@ * compactions, passes it through a CompactionFilter and executes all the compactions and * writes a new version of base files and make a normal commit. */ -public class HoodieJavaMergeOnReadTableCompactor +public class HoodieJavaMergeOnReadTableCompactor extends HoodieCompactor>, List, List> { @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseJavaDeltaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseJavaDeltaCommitActionExecutor.java index 0b4a654074408..dda399712b05f 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseJavaDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseJavaDeltaCommitActionExecutor.java @@ -20,13 +20,12 @@ package org.apache.hudi.table.action.deltacommit; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.BaseJavaCommitActionExecutor; -public abstract class BaseJavaDeltaCommitActionExecutor> extends BaseJavaCommitActionExecutor { +public abstract class BaseJavaDeltaCommitActionExecutor extends BaseJavaCommitActionExecutor { public BaseJavaDeltaCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, WriteOperationType operationType) { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitActionExecutor.java index f6faa28bbb1ef..ae01312f1f6df 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/deltacommit/JavaUpsertPreppedDeltaCommitActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -41,7 +40,7 @@ import java.util.LinkedList; import java.util.List; -public class JavaUpsertPreppedDeltaCommitActionExecutor> extends BaseJavaDeltaCommitActionExecutor { +public class JavaUpsertPreppedDeltaCommitActionExecutor extends BaseJavaDeltaCommitActionExecutor { private static final Logger LOG = LogManager.getLogger(JavaUpsertPreppedDeltaCommitActionExecutor.class); diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java index 16ee0f9953a77..e4a30ba84120a 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java @@ -20,6 +20,7 @@ package org.apache.hudi.execution.bulkinsert; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; @@ -70,7 +71,7 @@ public void testCustomColumnSortPartitioner(String sortColumnString) throws Exce private Comparator getCustomColumnComparator(Schema schema, String[] sortColumns) { return Comparator.comparing( - record -> HoodieAvroUtils.getRecordColumnValues(record, sortColumns, schema, false).toString()); + record -> HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)record, sortColumns, schema, false).toString()); } private void verifyRecordAscendingOrder(List records, diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index c6d83f9e94168..fec3a34b0bc31 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -35,7 +35,7 @@ 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.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java index 7277479f64ec0..2ada056099b08 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java @@ -19,7 +19,6 @@ package org.apache.hudi.client; import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; @@ -31,7 +30,7 @@ * @deprecated This. Use {@link SparkRDDReadClient instead.} */ @Deprecated -public class HoodieReadClient> extends SparkRDDReadClient { +public class HoodieReadClient extends SparkRDDReadClient { public HoodieReadClient(HoodieSparkEngineContext context, String basePath) { super(context, basePath); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java index 0812b366aadac..73037df40cb0a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -37,7 +36,7 @@ /** * Async clustering client for Spark datasource. */ -public class HoodieSparkClusteringClient extends +public class HoodieSparkClusteringClient extends BaseClusterer>, JavaRDD, JavaRDD> { private static final Logger LOG = LogManager.getLogger(HoodieSparkClusteringClient.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java index b3dc27b6fc65b..ae826a6bf266a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java @@ -21,7 +21,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; @@ -34,7 +33,7 @@ import java.util.List; -public class HoodieSparkCompactor extends BaseCompactor extends BaseCompactor>, JavaRDD, JavaRDD> { private static final Logger LOG = LogManager.getLogger(HoodieSparkCompactor.class); private transient HoodieEngineContext context; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDReadClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDReadClient.java index adddabfdc0299..d173d3d1a20dd 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDReadClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDReadClient.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; @@ -59,7 +58,7 @@ /** * Provides an RDD based API for accessing/filtering Hoodie tables, based on keys. */ -public class SparkRDDReadClient> implements Serializable { +public class SparkRDDReadClient implements Serializable { private static final long serialVersionUID = 1L; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 9734db6cd6a19..df2741e806425 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.TableServiceType; @@ -64,7 +63,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -75,7 +73,7 @@ import java.util.stream.Collectors; @SuppressWarnings("checkstyle:LineLength") -public class SparkRDDWriteClient extends +public class SparkRDDWriteClient extends BaseHoodieWriteClient>, JavaRDD, JavaRDD> { private static final Logger LOG = LogManager.getLogger(SparkRDDWriteClient.class); @@ -100,17 +98,6 @@ public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeC super(context, writeConfig, timelineService, SparkUpgradeDowngradeHelper.getInstance()); } - /** - * Register hudi classes for Kryo serialization. - * - * @param conf instance of SparkConf - * @return SparkConf - */ - public static SparkConf registerClasses(SparkConf conf) { - conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); - return conf; - } - @Override protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) { return SparkHoodieIndexFactory.createIndex(config); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java index acb8dadf0a7bb..ba3f917d04c70 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java @@ -22,7 +22,6 @@ import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -36,7 +35,7 @@ * In this strategy, clustering group for each partition is built in the same way as {@link SparkSizeBasedClusteringPlanStrategy}. * The difference is that the output groups is 1 and file group id remains the same. */ -public class SparkSingleFileSortPlanStrategy> +public class SparkSingleFileSortPlanStrategy extends SparkSizeBasedClusteringPlanStrategy { public SparkSingleFileSortPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java index 46d1851d131b8..5b048c2da7206 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java @@ -25,7 +25,6 @@ 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.HoodieRecordPayload; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -49,7 +48,7 @@ * 1) Creates clustering groups based on max size allowed per group. * 2) Excludes files that are greater than 'small.file.limit' from clustering plan. */ -public class SparkSizeBasedClusteringPlanStrategy> +public class SparkSizeBasedClusteringPlanStrategy extends PartitionAwareClusteringPlanStrategy>, JavaRDD, JavaRDD> { private static final Logger LOG = LogManager.getLogger(SparkSizeBasedClusteringPlanStrategy.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 954daaad1e169..3c5c415211249 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -33,11 +33,12 @@ import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.FutureUtils; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; @@ -54,13 +55,14 @@ 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.BaseKeyGenerator; +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -90,7 +92,7 @@ /** * Clustering strategy to submit multiple spark jobs and union the results. */ -public abstract class MultipleSparkJobExecutionStrategy> +public abstract class MultipleSparkJobExecutionStrategy extends ClusteringExecutionStrategy>, HoodieData, HoodieData> { private static final Logger LOG = LogManager.getLogger(MultipleSparkJobExecutionStrategy.class); @@ -196,7 +198,7 @@ private BulkInsertPartitioner getPartitioner(Map strategy return isRowPartitioner ? new RowSpatialCurveSortPartitioner(getWriteConfig()) : new RDDSpatialCurveSortPartitioner((HoodieSparkEngineContext) getEngineContext(), orderByColumns, layoutOptStrategy, - getWriteConfig().getLayoutOptimizationCurveBuildMethod(), HoodieAvroUtils.addMetadataFields(schema)); + getWriteConfig().getLayoutOptimizationCurveBuildMethod(), HoodieAvroUtils.addMetadataFields(schema), recordType); case LINEAR: return isRowPartitioner ? new RowCustomColumnsSortPartitioner(orderByColumns) @@ -219,6 +221,9 @@ private CompletableFuture> runClusteringForGroupAsync(Ho JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(getEngineContext()); HoodieData> inputRecords = readRecordsForGroup(jsc, clusteringGroup, instantTime); Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be shuffled later. List inputFileIds = clusteringGroup.getSlices().stream() .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) .collect(Collectors.toList()); @@ -291,15 +296,15 @@ private HoodieData> readRecordsForGroupWithLogs(JavaSparkContext .withUseScanV2(config.useScanV2ForLogRecordReader()) .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType()) .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) + .withRecordMerger(config.getRecordMerger()) .build(); Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) ? Option.empty() - : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); + : Option.of(HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); recordIterators.add(getFileSliceReader(baseFileReader, scanner, readerSchema, - tableConfig.getPayloadClass(), - tableConfig.getPreCombineField(), + tableConfig.getProps(), tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp())))); } catch (IOException e) { @@ -324,12 +329,20 @@ private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContex // closure, as this might lead to issues attempting to serialize its nested fields return HoodieJavaRDD.of(jsc.parallelize(clusteringOps, clusteringOps.size()) .mapPartitions(clusteringOpsPartition -> { - List> iteratorsForPartition = new ArrayList<>(); + List>> iteratorsForPartition = new ArrayList<>(); clusteringOpsPartition.forEachRemaining(clusteringOp -> { try { Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(writeConfig.getSchema())); - HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath())); - iteratorsForPartition.add(baseFileReader.getRecordIterator(readerSchema)); + HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath())); + Option keyGeneratorOp = + writeConfig.populateMetaFields() ? Option.empty() : Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(writeConfig.getProps())); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be shuffled later. + MappingIterator mappingIterator = new MappingIterator((ClosableIterator) baseFileReader.getRecordIterator(readerSchema), + rec -> ((HoodieRecord) rec).copy().wrapIntoHoodieRecordPayloadWithKeyGen(readerSchema, + writeConfig.getProps(), keyGeneratorOp)); + iteratorsForPartition.add(mappingIterator); } catch (IOException e) { throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + " and " + clusteringOp.getDeltaFilePaths(), e); @@ -337,8 +350,7 @@ private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContex }); return new ConcatenatingIterator<>(iteratorsForPartition); - }) - .map(record -> transform(record, writeConfig))); + })); } /** @@ -413,11 +425,4 @@ private JavaRDD[] convertStreamToArray(Stream> } return writeStatusRDDArray; } - - /** - * Transform IndexedRecord into HoodieRecord. - */ - private static HoodieRecord transform(IndexedRecord indexedRecord, HoodieWriteConfig writeConfig) { - return ExecutionStrategyUtil.transform(indexedRecord, writeConfig); - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java index 8606c89c49b1f..4c27270144706 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java @@ -32,18 +32,21 @@ import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -65,7 +68,7 @@ * Clustering strategy to submit single spark jobs. * MultipleSparkJobExecution strategy is not ideal for use cases that require large number of clustering groups */ -public abstract class SingleSparkJobExecutionStrategy> +public abstract class SingleSparkJobExecutionStrategy extends ClusteringExecutionStrategy>, HoodieData, HoodieData> { private static final Logger LOG = LogManager.getLogger(SingleSparkJobExecutionStrategy.class); @@ -141,25 +144,27 @@ private Iterator> readRecordsForGroupBaseFiles(List>> iteratorsForPartition = clusteringOps.stream().map(clusteringOp -> { Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); - Iterable indexedRecords = () -> { + Iterable> indexedRecords = () -> { try { - return HoodieFileReaderFactory.getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath())).getRecordIterator(readerSchema); + HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath())); + Option keyGeneratorOp = + writeConfig.populateMetaFields() ? Option.empty() : Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(writeConfig.getProps())); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be shuffled later. + MappingIterator mappingIterator = new MappingIterator((ClosableIterator) baseFileReader.getRecordIterator(readerSchema), + rec -> ((HoodieRecord) rec).copy().wrapIntoHoodieRecordPayloadWithKeyGen(readerSchema, + getWriteConfig().getProps(), keyGeneratorOp)); + return mappingIterator; } catch (IOException e) { throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + " and " + clusteringOp.getDeltaFilePaths(), e); } }; - return StreamSupport.stream(indexedRecords.spliterator(), false).map(this::transform).iterator(); + return StreamSupport.stream(indexedRecords.spliterator(), false).iterator(); }).collect(Collectors.toList()); return new ConcatenatingIterator<>(iteratorsForPartition); } - - /** - * Transform IndexedRecord into HoodieRecord. - */ - private HoodieRecord transform(IndexedRecord indexedRecord) { - return ExecutionStrategyUtil.transform(indexedRecord, getWriteConfig()); - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index f2ae9a922d811..6c0c09d193aba 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -25,8 +25,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.io.SingleFileHandleCreateFactory; @@ -46,7 +45,7 @@ * This strategy is similar to {@link SparkSortAndSizeExecutionStrategy} with the difference being that * there should be only one large file group per clustering group. */ -public class SparkSingleFileSortExecutionStrategy> +public class SparkSingleFileSortExecutionStrategy extends MultipleSparkJobExecutionStrategy { private static final Logger LOG = LogManager.getLogger(SparkSingleFileSortExecutionStrategy.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index 35c8f288bc891..dbba0e642e872 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -24,8 +24,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.table.HoodieTable; @@ -45,7 +44,7 @@ * 1) Spark execution engine. * 2) Uses bulk_insert to write data into new files. */ -public class SparkSortAndSizeExecutionStrategy> +public class SparkSortAndSizeExecutionStrategy extends MultipleSparkJobExecutionStrategy { private static final Logger LOG = LogManager.getLogger(SparkSortAndSizeExecutionStrategy.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java index 8263c209218f7..751e2a2858bca 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; @@ -33,7 +32,7 @@ * Spark base update strategy, write records to the file groups which are in clustering * need to check. Spark relate implementations should extend this base class. */ -public abstract class BaseSparkUpdateStrategy> extends UpdateStrategy>> { +public abstract class BaseSparkUpdateStrategy extends UpdateStrategy>> { public BaseSparkUpdateStrategy(HoodieEngineContext engineContext, HoodieTable table, Set fileGroupsInPendingClustering) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java index 6d819df3c2093..7de85ae977871 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.table.HoodieTable; @@ -33,7 +32,7 @@ /** * Allow ingestion commits during clustering job. */ -public class SparkAllowUpdateStrategy> extends BaseSparkUpdateStrategy { +public class SparkAllowUpdateStrategy extends BaseSparkUpdateStrategy { public SparkAllowUpdateStrategy( HoodieEngineContext engineContext, HoodieTable table, Set fileGroupsInPendingClustering) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java index 2b05942f00a74..654ddc7be1350 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieClusteringUpdateException; import org.apache.hudi.table.HoodieTable; @@ -38,7 +37,7 @@ * Update strategy based on following. * if some file groups have update record, throw exception */ -public class SparkRejectUpdateStrategy> extends BaseSparkUpdateStrategy { +public class SparkRejectUpdateStrategy extends BaseSparkUpdateStrategy { private static final Logger LOG = LogManager.getLogger(SparkRejectUpdateStrategy.class); public SparkRejectUpdateStrategy(HoodieEngineContext engineContext, HoodieTable table, Set fileGroupsInPendingClustering) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java index 7425137665746..7370ee0f088a2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java @@ -29,8 +29,6 @@ import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; -import java.util.Arrays; - /** * Hudi internal implementation of the {@link InternalRow} allowing to extend arbitrary * {@link InternalRow} overlaying Hudi-internal meta-fields on top of it. @@ -84,7 +82,7 @@ public HoodieInternalRow(UTF8String commitTime, this.sourceContainsMetaFields = sourceContainsMetaFields; } - private HoodieInternalRow(UTF8String[] metaFields, + public HoodieInternalRow(UTF8String[] metaFields, InternalRow sourceRow, boolean sourceContainsMetaFields) { this.metaFields = metaFields; @@ -231,7 +229,11 @@ public MapData getMap(int ordinal) { @Override public InternalRow copy() { - return new HoodieInternalRow(Arrays.copyOf(metaFields, metaFields.length), sourceRow.copy(), sourceContainsMetaFields); + UTF8String[] copyMetaFields = new UTF8String[metaFields.length]; + for (int i = 0; i < metaFields.length; i++) { + copyMetaFields[i] = metaFields[i] != null ? metaFields[i].copy() : null; + } + return new HoodieInternalRow(copyMetaFields, sourceRow.copy(), sourceContainsMetaFields); } private int rebaseOrdinal(int ordinal) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java index c1f0f1a83b4cb..a87789544a40d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.config.HoodieWriteConfig; @@ -42,7 +41,7 @@ /** * Validator can be configured pre-commit. */ -public abstract class SparkPreCommitValidator> { +public abstract class SparkPreCommitValidator> { private static final Logger LOG = LogManager.getLogger(SparkPreCommitValidator.class); private HoodieSparkTable table; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java index 2506d52b4a416..98b930f7a7206 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieValidationException; @@ -40,7 +39,7 @@ * * Expects both queries to return same result. */ -public class SqlQueryEqualityPreCommitValidator> extends SqlQueryPreCommitValidator { +public class SqlQueryEqualityPreCommitValidator> extends SqlQueryPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryEqualityPreCommitValidator.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java index 8a25150651943..0821774619569 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieValidationException; @@ -40,7 +39,7 @@ *

* Expects query results do not match. */ -public class SqlQueryInequalityPreCommitValidator> extends SqlQueryPreCommitValidator { +public class SqlQueryInequalityPreCommitValidator> extends SqlQueryPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryInequalityPreCommitValidator.class); public SqlQueryInequalityPreCommitValidator(HoodieSparkTable table, HoodieEngineContext engineContext, HoodieWriteConfig config) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java index 3a88d54d36d7b..e0120c29612f3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieValidationException; @@ -42,7 +41,7 @@ /** * Validator framework to run sql queries and compare table state at different locations. */ -public abstract class SqlQueryPreCommitValidator> extends SparkPreCommitValidator { +public abstract class SqlQueryPreCommitValidator> extends SparkPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryPreCommitValidator.class); private static final AtomicInteger TABLE_COUNTER = new AtomicInteger(0); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java index b1942244d3d39..c62187a8f2364 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieValidationException; @@ -40,7 +39,7 @@ *

* Example configuration: "query1#expectedResult1;query2#expectedResult2;" */ -public class SqlQuerySingleResultPreCommitValidator> extends SqlQueryPreCommitValidator { +public class SqlQuerySingleResultPreCommitValidator> extends SqlQueryPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryInequalityPreCommitValidator.class); public SqlQuerySingleResultPreCommitValidator(HoodieSparkTable table, HoodieEngineContext engineContext, HoodieWriteConfig config) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java new file mode 100644 index 0000000000000..b119e75e2177c --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoSerializable; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.avro.Schema; +import org.apache.hudi.SparkAdapterSupport$; +import org.apache.hudi.client.model.HoodieInternalRow; +import org.apache.hudi.common.util.ConfigUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.SparkKeyGeneratorInterface; +import org.apache.hudi.util.HoodieSparkRecordUtils; +import org.apache.spark.sql.HoodieInternalRowUtils; +import org.apache.spark.sql.HoodieUnsafeRowUtils; +import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath; +import org.apache.spark.sql.catalyst.CatalystTypeConverters; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.JoinedRow; +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + +import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS; +import static org.apache.hudi.util.HoodieSparkRecordUtils.getNullableValAsString; +import static org.apache.hudi.util.HoodieSparkRecordUtils.getValue; +import static org.apache.spark.sql.types.DataTypes.BooleanType; +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** + * Spark Engine-specific Implementations of `HoodieRecord` + * + * NOTE: [[HoodieSparkRecord]] is expected to hold either [[UnsafeRow]] or [[HoodieInternalRow]]: + * + *

    + *
  • [[UnsafeRow]] is held to make sure a) we don't deserialize raw bytes payload + * into JVM types unnecessarily, b) we don't incur penalty of ser/de during shuffling, + * c) we don't add strain on GC
  • + *
  • [[HoodieInternalRow]] is held in cases when underlying [[UnsafeRow]]'s metadata fields + * need to be updated (ie serving as an overlay layer on top of [[UnsafeRow]])
  • + *
+ * + */ +public class HoodieSparkRecord extends HoodieRecord implements KryoSerializable { + + /** + * Record copy operation to avoid double copying. InternalRow do not need to copy twice. + */ + private boolean copy; + + /** + * NOTE: {@code HoodieSparkRecord} is holding the schema only in cases when it would have + * to execute {@link UnsafeProjection} so that the {@link InternalRow} it's holding to + * could be projected into {@link UnsafeRow} and be efficiently serialized subsequently + * (by Kryo) + */ + private final transient StructType schema; + + public HoodieSparkRecord(UnsafeRow data) { + this(data, null); + } + + public HoodieSparkRecord(InternalRow data, StructType schema) { + super(null, data); + + validateRow(data, schema); + this.copy = false; + this.schema = schema; + } + + public HoodieSparkRecord(HoodieKey key, UnsafeRow data, boolean copy) { + this(key, data, null, copy); + } + + public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, boolean copy) { + super(key, data); + + validateRow(data, schema); + this.copy = copy; + this.schema = schema; + } + + private HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, boolean copy) { + super(key, data, operation, Option.empty()); + + validateRow(data, schema); + this.copy = copy; + this.schema = schema; + } + + public HoodieSparkRecord( + HoodieKey key, + InternalRow data, + StructType schema, + HoodieOperation operation, + HoodieRecordLocation currentLocation, + HoodieRecordLocation newLocation, + boolean copy) { + super(key, data, operation, currentLocation, newLocation); + this.copy = copy; + this.schema = schema; + } + + @Override + public HoodieSparkRecord newInstance() { + return new HoodieSparkRecord(this.key, this.data, this.schema, this.operation, this.copy); + } + + @Override + public HoodieSparkRecord newInstance(HoodieKey key, HoodieOperation op) { + return new HoodieSparkRecord(key, this.data, this.schema, op, this.copy); + } + + @Override + public HoodieSparkRecord newInstance(HoodieKey key) { + return new HoodieSparkRecord(key, this.data, this.schema, this.operation, this.copy); + } + + @Override + public String getRecordKey(Schema recordSchema, Option keyGeneratorOpt) { + if (key != null) { + return getRecordKey(); + } + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()) + .getRecordKey(data, structType).toString() : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal()); + } + + @Override + public String getRecordKey(Schema recordSchema, String keyFieldName) { + if (key != null) { + return getRecordKey(); + } + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + DataType dataType = structType.apply(keyFieldName).dataType(); + int pos = structType.fieldIndex(keyFieldName); + return data.get(pos, dataType).toString(); + } + + @Override + public HoodieRecordType getRecordType() { + return HoodieRecordType.SPARK; + } + + @Override + public Object[] getColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) { + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled); + } + + @Override + public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { + StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema); + InternalRow mergeRow = new JoinedRow(data, (InternalRow) other.getData()); + UnsafeProjection projection = + HoodieInternalRowUtils.getCachedUnsafeProjection(targetStructType, targetStructType); + return new HoodieSparkRecord(getKey(), projection.apply(mergeRow), targetStructType, getOperation(), this.currentLocation, this.newLocation, copy); + } + + @Override + public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException { + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema); + + // TODO HUDI-5281 Rewrite HoodieSparkRecord with UnsafeRowWriter + InternalRow rewriteRecord = HoodieInternalRowUtils.rewriteRecord(this.data, structType, targetStructType); + UnsafeRow unsafeRow = HoodieInternalRowUtils.getCachedUnsafeProjection(targetStructType, targetStructType).apply(rewriteRecord); + + boolean containMetaFields = hasMetaFields(targetStructType); + UTF8String[] metaFields = tryExtractMetaFields(unsafeRow, targetStructType); + HoodieInternalRow internalRow = new HoodieInternalRow(metaFields, unsafeRow, containMetaFields); + + return new HoodieSparkRecord(getKey(), internalRow, targetStructType, getOperation(), this.currentLocation, this.newLocation, false); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException { + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema); + + // TODO HUDI-5281 Rewrite HoodieSparkRecord with UnsafeRowWriter + InternalRow rewriteRecord = HoodieInternalRowUtils.rewriteRecordWithNewSchema(this.data, structType, newStructType, renameCols); + UnsafeRow unsafeRow = HoodieInternalRowUtils.getCachedUnsafeProjection(newStructType, newStructType).apply(rewriteRecord); + + boolean containMetaFields = hasMetaFields(newStructType); + UTF8String[] metaFields = tryExtractMetaFields(unsafeRow, newStructType); + HoodieInternalRow internalRow = new HoodieInternalRow(metaFields, unsafeRow, containMetaFields); + + return new HoodieSparkRecord(getKey(), internalRow, newStructType, getOperation(), this.currentLocation, this.newLocation, false); + } + + @Override + public HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException { + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + HoodieInternalRow updatableRow = wrapIntoUpdatableOverlay(data, structType); + + metadataValues.getKv().forEach((key, value) -> { + int pos = structType.fieldIndex(key); + if (value != null) { + updatableRow.update(pos, CatalystTypeConverters.convertToCatalyst(value)); + } + }); + + return new HoodieSparkRecord(getKey(), updatableRow, structType, getOperation(), this.currentLocation, this.newLocation, copy); + } + + @Override + public HoodieRecord truncateRecordKey(Schema recordSchema, Properties props, String keyFieldName) { + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + int pos = structType.fieldIndex(keyFieldName); + data.update(pos, CatalystTypeConverters.convertToCatalyst(StringUtils.EMPTY_STRING)); + return this; + } + + @Override + public boolean isDelete(Schema recordSchema, Properties props) throws IOException { + if (null == data) { + return true; + } + if (recordSchema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) { + return false; + } + Object deleteMarker = data.get(recordSchema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType); + return deleteMarker instanceof Boolean && (boolean) deleteMarker; + } + + @Override + public boolean shouldIgnore(Schema recordSchema, Properties props) throws IOException { + return false; + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithParams( + Schema recordSchema, Properties props, + Option> simpleKeyGenFieldsOpt, + Boolean withOperation, + Option partitionNameOp, + Boolean populateMetaFields) { + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + if (populateMetaFields) { + return convertToHoodieSparkRecord(structType, this, withOperation); + } else if (simpleKeyGenFieldsOpt.isPresent()) { + return convertToHoodieSparkRecord(structType, this, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty()); + } else { + return convertToHoodieSparkRecord(structType, this, withOperation, partitionNameOp); + } + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Schema recordSchema, Properties props, Option keyGen) { + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + String key; + String partition; + boolean populateMetaFields = Boolean.parseBoolean(props.getOrDefault(POPULATE_META_FIELDS.key(), + POPULATE_META_FIELDS.defaultValue().toString()).toString()); + if (!populateMetaFields && keyGen.isPresent()) { + SparkKeyGeneratorInterface keyGenerator = (SparkKeyGeneratorInterface) keyGen.get(); + key = keyGenerator.getRecordKey(data, structType).toString(); + partition = keyGenerator.getPartitionPath(data, structType).toString(); + } else { + key = data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString(); + partition = data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString(); + } + HoodieKey hoodieKey = new HoodieKey(key, partition); + return new HoodieSparkRecord(hoodieKey, data, structType, getOperation(), this.currentLocation, this.newLocation, copy); + } + + @Override + public Option> getMetadata() { + // TODO HUDI-5282 support metaData + return Option.empty(); + } + + @Override + public Option toIndexedRecord(Schema recordSchema, Properties prop) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieSparkRecord copy() { + if (!copy) { + this.data = this.data.copy(); + this.copy = true; + } + return this; + } + + @Override + public Comparable getOrderingValue(Schema recordSchema, Properties props) { + StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); + String orderingField = ConfigUtils.getOrderingField(props); + if (!HoodieInternalRowUtils.existField(structType, orderingField)) { + return 0; + } else { + NestedFieldPath nestedFieldPath = HoodieInternalRowUtils.getCachedPosList(structType, orderingField); + Comparable value = (Comparable) HoodieUnsafeRowUtils.getNestedInternalRowValue(data, nestedFieldPath); + return value; + } + } + + /** + * NOTE: This method is declared final to make sure there's no polymorphism and therefore + * JIT compiler could perform more aggressive optimizations + */ + @Override + protected final void writeRecordPayload(InternalRow payload, Kryo kryo, Output output) { + // NOTE: [[payload]] could be null if record has already been deflated + UnsafeRow unsafeRow = convertToUnsafeRow(payload, schema); + + kryo.writeObjectOrNull(output, unsafeRow, UnsafeRow.class); + } + + /** + * NOTE: This method is declared final to make sure there's no polymorphism and therefore + * JIT compiler could perform more aggressive optimizations + */ + @Override + protected final InternalRow readRecordPayload(Kryo kryo, Input input) { + // NOTE: After deserialization every object is allocated on the heap, therefore + // we annotate this object as being copied + this.copy = true; + + return kryo.readObjectOrNull(input, UnsafeRow.class); + } + + private static UnsafeRow convertToUnsafeRow(InternalRow payload, StructType schema) { + if (payload == null) { + return null; + } else if (payload instanceof UnsafeRow) { + return (UnsafeRow) payload; + } + + UnsafeProjection unsafeProjection = HoodieInternalRowUtils.getCachedUnsafeProjection(schema, schema); + return unsafeProjection.apply(payload); + } + + private static HoodieInternalRow wrapIntoUpdatableOverlay(InternalRow data, StructType structType) { + if (data instanceof HoodieInternalRow) { + return (HoodieInternalRow) data; + } + + boolean containsMetaFields = hasMetaFields(structType); + UTF8String[] metaFields = tryExtractMetaFields(data, structType); + return new HoodieInternalRow(metaFields, data, containsMetaFields); + } + + private static UTF8String[] tryExtractMetaFields(InternalRow row, StructType structType) { + boolean containsMetaFields = hasMetaFields(structType); + if (containsMetaFields && structType.size() == 1) { + // Support bootstrap with RECORD_KEY_SCHEMA + return new UTF8String[] {row.getUTF8String(0)}; + } else if (containsMetaFields) { + return HoodieRecord.HOODIE_META_COLUMNS.stream() + .map(col -> row.getUTF8String(HOODIE_META_COLUMNS_NAME_TO_POS.get(col))) + .toArray(UTF8String[]::new); + } else { + return new UTF8String[HoodieRecord.HOODIE_META_COLUMNS.size()]; + } + } + + private static boolean hasMetaFields(StructType structType) { + return structType.getFieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD).isDefined(); + } + + /** + * Utility method to convert InternalRow to HoodieRecord using schema and payload class. + */ + private static HoodieRecord convertToHoodieSparkRecord(StructType structType, HoodieSparkRecord record, boolean withOperationField) { + return convertToHoodieSparkRecord(structType, record, + Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), + withOperationField, Option.empty()); + } + + private static HoodieRecord convertToHoodieSparkRecord(StructType structType, HoodieSparkRecord record, boolean withOperationField, + Option partitionName) { + return convertToHoodieSparkRecord(structType, record, + Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), + withOperationField, partitionName); + } + + /** + * Utility method to convert bytes to HoodieRecord using schema and payload class. + */ + private static HoodieRecord convertToHoodieSparkRecord(StructType structType, HoodieSparkRecord record, Pair recordKeyPartitionPathFieldPair, + boolean withOperationField, Option partitionName) { + final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), record.data).toString(); + final String partitionPath = (partitionName.isPresent() ? partitionName.get() : + getValue(structType, recordKeyPartitionPathFieldPair.getRight(), record.data).toString()); + + HoodieOperation operation = withOperationField + ? HoodieOperation.fromName(getNullableValAsString(structType, record.data, HoodieRecord.OPERATION_METADATA_FIELD)) : null; + return new HoodieSparkRecord(new HoodieKey(recKey, partitionPath), record.data, structType, operation, record.copy); + } + + private static void validateRow(InternalRow data, StructType schema) { + // NOTE: [[HoodieSparkRecord]] is expected to hold either + // - Instance of [[UnsafeRow]] or + // - Instance of [[HoodieInternalRow]] or + // - Instance of [[ColumnarBatchRow]] + // + // In case provided row is anything but [[UnsafeRow]], it's expected that the + // corresponding schema has to be provided as well so that it could be properly + // serialized (in case it would need to be) + boolean isValid = data instanceof UnsafeRow + || schema != null && (data instanceof HoodieInternalRow || SparkAdapterSupport$.MODULE$.sparkAdapter().isColumnarBatchRow(data)); + + ValidationUtils.checkState(isValid); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index acb9a3169c642..fec382a6a611d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.HoodieExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -37,7 +36,7 @@ import static org.apache.hudi.common.util.ValidationUtils.checkState; -public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { +public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { private boolean useWriterSchema; @@ -88,7 +87,7 @@ protected List computeNext() { } bufferedIteratorExecutor = QueueBasedExecutorFactory.create(hoodieConfig, inputItr, getInsertHandler(), - getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); + getCloningTransformer(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); final List result = bufferedIteratorExecutor.execute(); checkState(result != null && !result.isEmpty()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java index 66c3bdddcb1ef..c54d579224a92 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java @@ -20,7 +20,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.execution.SparkLazyInsertIterable; import org.apache.hudi.io.WriteHandleFactory; @@ -35,7 +34,7 @@ /** * Map function that handles a stream of HoodieRecords. */ -public class BulkInsertMapFunction +public class BulkInsertMapFunction implements Function2>, Iterator>> { private String instantTime; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java index e10d23743dac1..308ee1992e304 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java @@ -19,7 +19,6 @@ package org.apache.hudi.execution.bulkinsert; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.BulkInsertPartitioner; @@ -35,7 +34,7 @@ * * @param HoodieRecordPayload type */ -public class GlobalSortPartitioner +public class GlobalSortPartitioner implements BulkInsertPartitioner>> { private final boolean shouldPopulateMetaFields; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java index 67cd599731c13..b2262948f19d8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java @@ -19,7 +19,6 @@ package org.apache.hudi.execution.bulkinsert; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.spark.api.java.JavaRDD; @@ -36,7 +35,7 @@ * * @param HoodieRecordPayload type */ -public class NonSortPartitioner +public class NonSortPartitioner implements BulkInsertPartitioner>> { private final boolean enforceNumOutputPartitions; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java index dc5317f7d8a33..6c490f5bebcec 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java @@ -19,7 +19,6 @@ package org.apache.hudi.execution.bulkinsert; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.spark.api.java.JavaRDD; @@ -28,6 +27,6 @@ * Abstract of bucket index bulk_insert partitioner * TODO implement partitioner for SIMPLE BUCKET INDEX */ -public abstract class RDDBucketIndexPartitioner +public abstract class RDDBucketIndexPartitioner implements BulkInsertPartitioner>> { } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java index 7b644938bb9f2..37273ce464040 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java @@ -25,10 +25,11 @@ import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.FlatLists; +import org.apache.hudi.common.util.collection.FlatLists.ComparableList; import org.apache.hudi.index.bucket.ConsistentBucketIdentifier; import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex; import org.apache.hudi.io.AppendHandleFactory; @@ -59,7 +60,7 @@ /** * A partitioner for (consistent hashing) bucket index used in bulk_insert */ -public class RDDConsistentBucketPartitioner extends RDDBucketIndexPartitioner { +public class RDDConsistentBucketPartitioner extends RDDBucketIndexPartitioner { private static final Logger LOG = LogManager.getLogger(RDDConsistentBucketPartitioner.class); @@ -235,9 +236,9 @@ private JavaRDD> doPartitionAndCustomColumnSort(JavaRDD> comparator = (Comparator> & Serializable) (t1, t2) -> { - Object obj1 = HoodieAvroUtils.getRecordColumnValues(t1, sortColumns, schema, consistentLogicalTimestampEnabled); - Object obj2 = HoodieAvroUtils.getRecordColumnValues(t2, sortColumns, schema, consistentLogicalTimestampEnabled); - return ((Comparable) obj1).compareTo(obj2); + ComparableList obj1 = FlatLists.ofComparableArray(t1.getColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled)); + ComparableList obj2 = FlatLists.ofComparableArray(t2.getColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled)); + return obj1.compareTo(obj2); }; return records.mapToPair(record -> new Tuple2<>(record, record)) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java index dc80498c7a964..e723d724b6405 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java @@ -18,10 +18,8 @@ package org.apache.hudi.execution.bulkinsert; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.BulkInsertPartitioner; @@ -36,7 +34,7 @@ * * @param HoodieRecordPayload type */ -public class RDDCustomColumnsSortPartitioner +public class RDDCustomColumnsSortPartitioner implements BulkInsertPartitioner>> { private final String[] sortColumnNames; @@ -63,7 +61,7 @@ public JavaRDD> repartitionRecords(JavaRDD> reco final boolean consistentLogicalTimestampEnabled = this.consistentLogicalTimestampEnabled; return records.sortBy( record -> { - Object recordValue = HoodieAvroUtils.getRecordColumnValues(record, sortColumns, schema, consistentLogicalTimestampEnabled); + Object recordValue = record.getColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled); // null values are replaced with empty string for null_first order if (recordValue == null) { return StringUtils.EMPTY_STRING; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java index b5dc83cc2fae1..0f81322070486 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java @@ -19,7 +19,6 @@ package org.apache.hudi.execution.bulkinsert; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.BulkInsertPartitioner; @@ -41,7 +40,7 @@ * * @param HoodieRecordPayload type */ -public class RDDPartitionSortPartitioner +public class RDDPartitionSortPartitioner implements BulkInsertPartitioner>> { private final boolean shouldPopulateMetaFields; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java index 2ab9107fa54b5..0512042d93f0a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java @@ -18,67 +18,93 @@ package org.apache.hudi.execution.bulkinsert; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.HoodieSparkUtils; +import org.apache.hudi.SparkConversionUtils; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieSparkRecord; import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.RewriteAvroPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; + +import java.util.Properties; /** * A partitioner that does spatial curve optimization sorting based on specified column values for each RDD partition. * support z-curve optimization, hilbert will come soon. * @param HoodieRecordPayload type */ -public class RDDSpatialCurveSortPartitioner +public class RDDSpatialCurveSortPartitioner extends SpatialCurveSortPartitionerBase>> { private final transient HoodieSparkEngineContext sparkEngineContext; private final SerializableSchema schema; + private final HoodieRecordType recordType; public RDDSpatialCurveSortPartitioner(HoodieSparkEngineContext sparkEngineContext, String[] orderByColumns, HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType, - Schema schema) { + Schema schema, + HoodieRecordType recordType) { super(orderByColumns, layoutOptStrategy, curveCompositionStrategyType); this.sparkEngineContext = sparkEngineContext; this.schema = new SerializableSchema(schema); + this.recordType = recordType; } @Override public JavaRDD> repartitionRecords(JavaRDD> records, int outputSparkPartitions) { - JavaRDD genericRecordsRDD = - records.map(f -> (GenericRecord) f.getData().getInsertValue(schema.get()).get()); + if (recordType == HoodieRecordType.AVRO) { + JavaRDD genericRecordsRDD = + records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get().getData()); - Dataset sourceDataset = - AvroConversionUtils.createDataFrame( - genericRecordsRDD.rdd(), - schema.toString(), - sparkEngineContext.getSqlContext().sparkSession() - ); + Dataset sourceDataset = + AvroConversionUtils.createDataFrame( + genericRecordsRDD.rdd(), + schema.toString(), + sparkEngineContext.getSqlContext().sparkSession() + ); + Dataset sortedDataset = reorder(sourceDataset, outputSparkPartitions); - Dataset sortedDataset = reorder(sourceDataset, outputSparkPartitions); + return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty()) + .toJavaRDD() + .map(record -> { + String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + HoodieKey hoodieKey = new HoodieKey(key, partition); + HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record)); + return hoodieRecord; + }); + } else if (recordType == HoodieRecordType.SPARK) { + StructType structType = HoodieInternalRowUtils.getCachedSchema(schema.get()); + Dataset sourceDataset = SparkConversionUtils.createDataFrame(records.rdd(), + sparkEngineContext.getSqlContext().sparkSession(), structType); + Dataset sortedDataset = reorder(sourceDataset, outputSparkPartitions); - return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty()) - .toJavaRDD() - .map(record -> { - String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); - String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); - HoodieKey hoodieKey = new HoodieKey(key, partition); - HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record)); - return hoodieRecord; - }); + return sortedDataset.queryExecution().toRdd() + .toJavaRDD() + .map(internalRow -> { + String key = internalRow.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal()); + String partition = internalRow.getString(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal()); + HoodieKey hoodieKey = new HoodieKey(key, partition); + return (HoodieRecord) new HoodieSparkRecord(hoodieKey, internalRow, structType, false); + }); + } else { + throw new UnsupportedOperationException(recordType.name()); + } } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java index aece86a3878ee..76606e316c16f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieIndexException; @@ -34,7 +33,7 @@ import org.apache.spark.api.java.JavaRDD; @SuppressWarnings("checkstyle:LineLength") -public abstract class SparkHoodieIndex> +public abstract class SparkHoodieIndex extends HoodieIndex>, JavaRDD> { protected SparkHoodieIndex(HoodieWriteConfig config) { super(config); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReader.java new file mode 100644 index 0000000000000..553b084e29c58 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReader.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.spark.sql.catalyst.InternalRow; + +/** + * Marker interface for every {@link HoodieFileReader} reading in Catalyst (Spark native tyeps, ie + * producing {@link InternalRow}s) + */ +public interface HoodieSparkFileReader extends HoodieFileReader {} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java new file mode 100644 index 0000000000000..112981f9022dd --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.exception.HoodieIOException; + +import java.io.IOException; +import org.apache.spark.sql.internal.SQLConf; + +public class HoodieSparkFileReaderFactory extends HoodieFileReaderFactory { + + protected HoodieFileReader newParquetFileReader(Configuration conf, Path path) { + conf.setIfUnset(SQLConf.PARQUET_BINARY_AS_STRING().key(), + SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()); + conf.setIfUnset(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), + SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()); + conf.setIfUnset(SQLConf.CASE_SENSITIVE().key(), SQLConf.CASE_SENSITIVE().defaultValueString()); + return new HoodieSparkParquetReader(conf, path); + } + + protected HoodieFileReader newHFileFileReader(Configuration conf, Path path) throws IOException { + throw new HoodieIOException("Not support read HFile"); + } + + protected HoodieFileReader newOrcFileReader(Configuration conf, Path path) { + throw new HoodieIOException("Not support read orc file"); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java new file mode 100644 index 0000000000000..58ee01182c373 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.avro.Schema; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.spark.sql.catalyst.CatalystTypeConverters; +import org.apache.spark.sql.catalyst.InternalRow; + +import java.io.IOException; +import java.util.Properties; + +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD; + +public interface HoodieSparkFileWriter extends HoodieFileWriter { + boolean canWrite(); + + void close() throws IOException; + + void writeRowWithMetadata(HoodieKey recordKey, InternalRow row) throws IOException; + + void writeRow(String recordKey, InternalRow row) throws IOException; + + @Override + default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException { + writeRow(recordKey, (InternalRow) record.getData()); + } + + @Override + default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException { + writeRowWithMetadata(key, (InternalRow) record.getData()); + } + + default InternalRow prepRecordWithMetadata(HoodieKey key, InternalRow row, String instantTime, Integer partitionId, long recordIndex, String fileName) { + String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex); + row.update(COMMIT_TIME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(instantTime)); + row.update(COMMIT_SEQNO_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(seqId)); + row.update(RECORD_KEY_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getRecordKey())); + row.update(PARTITION_PATH_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getPartitionPath())); + row.update(FILENAME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(fileName)); + return row; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java new file mode 100644 index 0000000000000..d2ab83f1481e8 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.row.HoodieRowParquetConfig; +import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.spark.sql.HoodieInternalRowUtils; + +import java.io.IOException; + +public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory { + + @Override + protected HoodieFileWriter newParquetFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS); + boolean enableBloomFilter = populateMetaFields; + Option filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); + String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); + // Support PARQUET_COMPRESSION_CODEC_NAME is "" + if (compressionCodecName.isEmpty()) { + compressionCodecName = null; + } + HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf, + HoodieInternalRowUtils.getCachedSchema(schema), filter, + HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build()); + HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport, + CompressionCodecName.fromConf(compressionCodecName), + config.getIntOrDefault(HoodieStorageConfig.PARQUET_BLOCK_SIZE), + config.getIntOrDefault(HoodieStorageConfig.PARQUET_PAGE_SIZE), + config.getLongOrDefault(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE), + conf, + config.getDoubleOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION), + config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED)); + parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf()); + + return new HoodieSparkParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, populateMetaFields); + } + + protected HoodieFileWriter newParquetFileWriter( + FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException { + boolean enableBloomFilter = false; + Option filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); + HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf, + HoodieInternalRowUtils.getCachedSchema(schema), filter, + HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build()); + String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); + // Support PARQUET_COMPRESSION_CODEC_NAME is "" + if (compressionCodecName.isEmpty()) { + compressionCodecName = null; + } + HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport, + CompressionCodecName.fromConf(compressionCodecName), + config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE), + config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE), + config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE), + writeSupport.getHadoopConf(), config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION), + config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED)); + parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf()); + return new HoodieSparkParquetStreamWriter(outputStream, parquetConfig); + } + + @Override + protected HoodieFileWriter newHFileFileWriter(String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + throw new HoodieIOException("Not support write to HFile"); + } + + @Override + protected HoodieFileWriter newOrcFileWriter(String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + throw new HoodieIOException("Not support write to Orc file"); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java new file mode 100644 index 0000000000000..320217aff1d4a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.SparkAdapterSupport$; +import org.apache.hudi.common.model.HoodieSparkRecord; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.MappingIterator; +import org.apache.hudi.common.util.ParquetReaderIterator; +import org.apache.hudi.common.util.ParquetUtils; +import org.apache.hudi.common.util.StringUtils; + +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.api.ReadSupport; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.sql.HoodieInternalRowUtils; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport; +import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +import static org.apache.hudi.common.util.TypeUtils.unsafeCast; +import static org.apache.parquet.avro.AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS; + +public class HoodieSparkParquetReader implements HoodieSparkFileReader { + + private final Path path; + private final Configuration conf; + private final BaseFileUtils parquetUtils; + private List readerIterators = new ArrayList<>(); + + public HoodieSparkParquetReader(Configuration conf, Path path) { + this.path = path; + this.conf = new Configuration(conf); + // Avoid adding record in list element when convert parquet schema to avro schema + conf.set(ADD_LIST_ELEMENT_RECORDS, "false"); + this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); + } + + @Override + public String[] readMinMaxRecordKeys() { + return parquetUtils.readMinMaxRecordKeys(conf, path); + } + + @Override + public BloomFilter readBloomFilter() { + return parquetUtils.readBloomFilterFromMetadata(conf, path); + } + + @Override + public Set filterRowKeys(Set candidateRowKeys) { + return parquetUtils.filterRowKeys(conf, path, candidateRowKeys); + } + + @Override + public ClosableIterator> getRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { + ClosableIterator iterator = getInternalRowIterator(readerSchema, requestedSchema); + StructType structType = HoodieInternalRowUtils.getCachedSchema(requestedSchema); + UnsafeProjection projection = HoodieInternalRowUtils.getCachedUnsafeProjection(structType, structType); + + return new MappingIterator<>(iterator, data -> { + // NOTE: We have to do [[UnsafeProjection]] of incoming [[InternalRow]] to convert + // it to [[UnsafeRow]] holding just raw bytes + UnsafeRow unsafeRow = projection.apply(data); + return unsafeCast(new HoodieSparkRecord(unsafeRow)); + }); + } + + private ClosableIterator getInternalRowIterator(Schema readerSchema, Schema requestedSchema) throws IOException { + if (requestedSchema == null) { + requestedSchema = readerSchema; + } + StructType readerStructType = HoodieInternalRowUtils.getCachedSchema(readerSchema); + StructType requestedStructType = HoodieInternalRowUtils.getCachedSchema(requestedSchema); + conf.set(ParquetReadSupport.PARQUET_READ_SCHEMA, readerStructType.json()); + conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), requestedStructType.json()); + conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(), (Boolean) SQLConf.get().getConf(SQLConf.PARQUET_BINARY_AS_STRING())); + conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), (Boolean) SQLConf.get().getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP())); + ParquetReader reader = ParquetReader.builder((ReadSupport) new ParquetReadSupport(), path) + .withConf(conf) + .build(); + ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader); + readerIterators.add(parquetReaderIterator); + return parquetReaderIterator; + } + + @Override + public Schema getSchema() { + // Some types in avro are not compatible with parquet. + // Avro only supports representing Decimals as fixed byte array + // and therefore if we convert to Avro directly we'll lose logical type-info. + MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(conf, path); + StructType structType = new ParquetToSparkSchemaConverter(conf).convert(messageType); + return SparkAdapterSupport$.MODULE$.sparkAdapter().getAvroSchemaConverters() + .toAvroType(structType, true, messageType.getName(), StringUtils.EMPTY_STRING); + } + + @Override + public void close() { + readerIterators.forEach(ParquetReaderIterator::close); + } + + @Override + public long getTotalRecords() { + return parquetUtils.getRowCount(conf, path); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java new file mode 100644 index 0000000000000..5c6f85af290d0 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.io.storage.row.HoodieRowParquetConfig; +import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport; +import org.apache.hudi.parquet.io.OutputStreamBackedOutputFile; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.api.WriteSupport; +import org.apache.parquet.io.OutputFile; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.unsafe.types.UTF8String; + +public class HoodieSparkParquetStreamWriter implements HoodieSparkFileWriter, AutoCloseable { + private final ParquetWriter writer; + private final HoodieRowParquetWriteSupport writeSupport; + + public HoodieSparkParquetStreamWriter(FSDataOutputStream outputStream, + HoodieRowParquetConfig parquetConfig) throws IOException { + this.writeSupport = parquetConfig.getWriteSupport(); + this.writer = new Builder<>(new OutputStreamBackedOutputFile(outputStream), writeSupport) + .withWriteMode(ParquetFileWriter.Mode.CREATE) + .withCompressionCodec(parquetConfig.getCompressionCodecName()) + .withRowGroupSize(parquetConfig.getBlockSize()) + .withPageSize(parquetConfig.getPageSize()) + .withDictionaryPageSize(parquetConfig.getPageSize()) + .withDictionaryEncoding(parquetConfig.dictionaryEnabled()) + .withWriterVersion(ParquetWriter.DEFAULT_WRITER_VERSION) + .withConf(parquetConfig.getHadoopConf()) + .build(); + } + + @Override + public boolean canWrite() { + return true; + } + + @Override + public void writeRow(String key, InternalRow record) throws IOException { + writer.write(record); + writeSupport.add(UTF8String.fromString(key)); + } + + @Override + public void writeRowWithMetadata(HoodieKey key, InternalRow record) throws IOException { + // TODO support populating the metadata + this.writeRow(key.getRecordKey(), record); + } + + @Override + public void close() throws IOException { + writer.close(); + } + + private static class Builder extends ParquetWriter.Builder> { + private final WriteSupport writeSupport; + + private Builder(Path file, WriteSupport writeSupport) { + super(file); + this.writeSupport = writeSupport; + } + + private Builder(OutputFile file, WriteSupport writeSupport) { + super(file); + this.writeSupport = writeSupport; + } + + @Override + protected Builder self() { + return this; + } + + @Override + protected WriteSupport getWriteSupport(Configuration conf) { + return writeSupport; + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java new file mode 100644 index 0000000000000..3b4a86502d280 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.io.storage.row.HoodieRowParquetConfig; +import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.unsafe.types.UTF8String; + +import java.io.IOException; + +public class HoodieSparkParquetWriter extends HoodieBaseParquetWriter implements HoodieSparkFileWriter { + + // TODO: better code reuse + private final String fileName; + private final String instantTime; + private final TaskContextSupplier taskContextSupplier; + private final boolean populateMetaFields; + private final HoodieRowParquetWriteSupport writeSupport; + + public HoodieSparkParquetWriter(Path file, + HoodieRowParquetConfig parquetConfig, + String instantTime, + TaskContextSupplier taskContextSupplier, + boolean populateMetaFields) throws IOException { + super(file, parquetConfig); + this.writeSupport = parquetConfig.getWriteSupport(); + this.fileName = file.getName(); + this.instantTime = instantTime; + this.taskContextSupplier = taskContextSupplier; + this.populateMetaFields = populateMetaFields; + } + + @Override + public void writeRowWithMetadata(HoodieKey key, InternalRow row) throws IOException { + if (populateMetaFields) { + prepRecordWithMetadata(key, row, instantTime, + taskContextSupplier.getPartitionIdSupplier().get(), getWrittenRecordCount(), fileName); + super.write(row); + writeSupport.add(UTF8String.fromString(key.getRecordKey())); + } else { + super.write(row); + } + } + + @Override + public void writeRow(String recordKey, InternalRow row) throws IOException { + super.write(row); + if (populateMetaFields) { + writeSupport.add(UTF8String.fromString(recordKey)); + } + } + + @Override + public void close() throws IOException { + super.close(); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java index e68873f92efdb..8a61c7c44d900 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java @@ -67,7 +67,7 @@ private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter(Path ) throws IOException { HoodieRowParquetWriteSupport writeSupport = - new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, bloomFilterOpt, writeConfig); + new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, bloomFilterOpt, writeConfig.getStorageConfig()); return new HoodieInternalRowParquetWriter( path, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java new file mode 100644 index 0000000000000..f5f6d7b0a5bb1 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage.row; + +import org.apache.hudi.io.storage.HoodieParquetConfig; + +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +/** + * ParquetConfig for datasource implementation with {@link org.apache.hudi.client.model.HoodieInternalRow}. + */ +public class HoodieRowParquetConfig extends HoodieParquetConfig { + + public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName, + int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf, + double compressionRatio, boolean enableDictionary) { + super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio, enableDictionary); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index bb4dd9c619425..3a1b6d000becc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -21,8 +21,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.HoodieBloomFilterWriteSupport; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; import org.apache.parquet.hadoop.api.WriteSupport; import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport; import org.apache.spark.sql.types.StructType; @@ -31,6 +31,8 @@ import java.util.Collections; import java.util.Map; +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_FIELD_ID_WRITE_ENABLED; + /** * Hoodie Write Support for directly writing Row to Parquet. */ @@ -39,11 +41,11 @@ public class HoodieRowParquetWriteSupport extends ParquetWriteSupport { private final Configuration hadoopConf; private final Option> bloomFilterWriteSupportOpt; - public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option bloomFilterOpt, HoodieWriteConfig writeConfig) { + public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option bloomFilterOpt, HoodieStorageConfig config) { Configuration hadoopConf = new Configuration(conf); - hadoopConf.set("spark.sql.parquet.writeLegacyFormat", writeConfig.parquetWriteLegacyFormatEnabled()); - hadoopConf.set("spark.sql.parquet.outputTimestampType", writeConfig.parquetOutputTimestampType()); - hadoopConf.set("spark.sql.parquet.fieldId.write.enabled", writeConfig.parquetFieldIdWriteEnabled()); + hadoopConf.set("spark.sql.parquet.writeLegacyFormat", config.getString(HoodieStorageConfig.PARQUET_WRITE_LEGACY_FORMAT_ENABLED)); + hadoopConf.set("spark.sql.parquet.outputTimestampType", config.getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE)); + hadoopConf.set("spark.sql.parquet.fieldId.write.enabled", config.getString(PARQUET_FIELD_ID_WRITE_ENABLED)); setSchema(structType, hadoopConf); this.hadoopConf = hadoopConf; 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 9a07ab09ed409..40154d8675cff 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 @@ -48,6 +48,7 @@ import java.util.function.Supplier; import static org.apache.hudi.common.util.CollectionUtils.tail; +import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_COMPOSITE_KEY_FILED_VALUE; import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_RECORD_KEY_PARTS_SEPARATOR; import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER; import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER; @@ -151,11 +152,12 @@ protected final UTF8String combinePartitionPathUnsafe(Object... partitionPathPar * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain * optimizations, like inlining) */ - protected final String combineRecordKey(Object... recordKeyParts) { + protected final String combineRecordKey(List fieldNames, List recordKeyParts) { return combineRecordKeyInternal( StringPartitionPathFormatter.JavaStringBuilder::new, BuiltinKeyGenerator::toString, BuiltinKeyGenerator::handleNullRecordKey, + fieldNames, recordKeyParts ); } @@ -164,11 +166,12 @@ protected final String combineRecordKey(Object... recordKeyParts) { * NOTE: This method has to stay final (so that it's easier for JIT compiler to apply certain * optimizations, like inlining) */ - protected final UTF8String combineRecordKeyUnsafe(Object... recordKeyParts) { + protected final UTF8String combineRecordKeyUnsafe(List fieldNames, List recordKeyParts) { return combineRecordKeyInternal( UTF8StringPartitionPathFormatter.UTF8StringBuilder::new, BuiltinKeyGenerator::toUTF8String, BuiltinKeyGenerator::handleNullRecordKey, + fieldNames, recordKeyParts ); } @@ -205,18 +208,20 @@ private S combineRecordKeyInternal( Supplier> builderFactory, Function converter, Function emptyKeyPartHandler, - Object... recordKeyParts + List fieldNames, + List recordKeyParts ) { - if (recordKeyParts.length == 1) { - return emptyKeyPartHandler.apply(converter.apply(recordKeyParts[0])); + if (recordKeyParts.size() == 1) { + return emptyKeyPartHandler.apply(converter.apply(recordKeyParts.get(0))); } PartitionPathFormatterBase.StringBuilder sb = builderFactory.get(); - for (int i = 0; i < recordKeyParts.length; ++i) { + for (int i = 0; i < recordKeyParts.size(); ++i) { + sb.appendJava(fieldNames.get(i)).appendJava(DEFAULT_COMPOSITE_KEY_FILED_VALUE); // NOTE: If record-key part has already been a string [[toString]] will be a no-op - sb.append(emptyKeyPartHandler.apply(converter.apply(recordKeyParts[i]))); + sb.append(emptyKeyPartHandler.apply(converter.apply(recordKeyParts.get(i)))); - if (i < recordKeyParts.length - 1) { + if (i < recordKeyParts.size() - 1) { sb.appendJava(DEFAULT_RECORD_KEY_PARTS_SEPARATOR); } } @@ -423,8 +428,11 @@ protected class SparkRowAccessor { private final HoodieUnsafeRowUtils.NestedFieldPath[] partitionPathFieldsPaths; SparkRowAccessor(StructType schema) { - this.recordKeyFieldsPaths = resolveNestedFieldPaths(getRecordKeyFieldNames(), schema); - this.partitionPathFieldsPaths = resolveNestedFieldPaths(getPartitionPathFields(), schema); + this.recordKeyFieldsPaths = resolveNestedFieldPaths(getRecordKeyFieldNames(), schema, false); + // Sometimes, we need to extract the recordKey from the partition-dropped data + // To be consistent with avro key generator + // ParquetBootstrapMetadataHandler + this.partitionPathFieldsPaths = resolveNestedFieldPaths(getPartitionPathFields(), schema, true); } public Object[] getRecordKeyParts(Row row) { @@ -432,6 +440,9 @@ public Object[] getRecordKeyParts(Row row) { } public Object[] getRecordPartitionPathValues(Row row) { + if (partitionPathFieldsPaths == null) { + throw new HoodieException("Failed to resolve nested partition field"); + } return getNestedFieldValues(row, partitionPathFieldsPaths); } @@ -440,6 +451,9 @@ public Object[] getRecordKeyParts(InternalRow row) { } public Object[] getRecordPartitionPathValues(InternalRow row) { + if (partitionPathFieldsPaths == null) { + throw new HoodieException("Failed to resolve nested partition field"); + } return getNestedFieldValues(row, partitionPathFieldsPaths); } @@ -463,12 +477,15 @@ private Object[] getNestedFieldValues(InternalRow row, HoodieUnsafeRowUtils.Nest return nestedFieldValues; } - private HoodieUnsafeRowUtils.NestedFieldPath[] resolveNestedFieldPaths(List fieldPaths, StructType schema) { + private HoodieUnsafeRowUtils.NestedFieldPath[] resolveNestedFieldPaths(List fieldPaths, StructType schema, boolean returnNull) { try { return fieldPaths.stream() .map(fieldPath -> HoodieUnsafeRowUtils$.MODULE$.composeNestedFieldPath(schema, fieldPath)) .toArray(HoodieUnsafeRowUtils.NestedFieldPath[]::new); } catch (Exception e) { + if (returnNull) { + return null; + } LOG.error(String.format("Failed to resolve nested field-paths (%s) in schema (%s)", fieldPaths, schema), e); throw new HoodieException("Failed to resolve nested field-paths", e); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java index ccd37dc9ce324..100bcc2cd7f2f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java @@ -61,13 +61,13 @@ public String getRecordKey(GenericRecord record) { @Override public String getRecordKey(Row row) { tryInitRowAccessor(row.schema()); - return combineRecordKey(rowAccessor.getRecordKeyParts(row)); + return combineRecordKey(getRecordKeyFieldNames(), Arrays.asList(rowAccessor.getRecordKeyParts(row))); } @Override public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { tryInitRowAccessor(schema); - return combineRecordKeyUnsafe(rowAccessor.getRecordKeyParts(internalRow)); + return combineRecordKeyUnsafe(getRecordKeyFieldNames(), Arrays.asList(rowAccessor.getRecordKeyParts(internalRow))); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java index f090320ccbcc3..fa36f2152cbbf 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java @@ -28,6 +28,7 @@ import org.apache.spark.unsafe.types.UTF8String; import java.io.IOException; +import java.util.Arrays; import java.util.Objects; import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH; @@ -61,13 +62,13 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { tryInitRowAccessor(row.schema()); - return combineRecordKey(rowAccessor.getRecordKeyParts(row)); + return combineRecordKey(getRecordKeyFieldNames(), Arrays.asList(rowAccessor.getRecordKeyParts(row))); } @Override public UTF8String getRecordKey(InternalRow internalRow, StructType schema) { tryInitRowAccessor(schema); - return combineRecordKeyUnsafe(rowAccessor.getRecordKeyParts(internalRow)); + return combineRecordKeyUnsafe(getRecordKeyFieldNames(), Arrays.asList(rowAccessor.getRecordKeyParts(internalRow))); } @Override 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 df33c4458f522..81b7cd00242d7 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 @@ -37,7 +37,6 @@ 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.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -94,7 +93,7 @@ *

* UPDATES - Produce a new version of the file, just replacing the updated records with new values */ -public class HoodieSparkCopyOnWriteTable +public class HoodieSparkCopyOnWriteTable extends HoodieSparkTable implements HoodieCompactionHandler { private static final Logger LOG = LogManager.getLogger(HoodieSparkCopyOnWriteTable.class); @@ -252,7 +251,7 @@ protected HoodieMergeHandle getUpdateHandle(String instantTime, String partition @Override public Iterator> handleInsert( String instantTime, String partitionPath, String fileId, - Map> recordMap) { + Map> recordMap) { HoodieCreateHandle createHandle = new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier); createHandle.write(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java index 67194aa06e0f4..25511cd19ab99 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java @@ -28,7 +28,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -78,7 +77,7 @@ * action *

*/ -public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable implements HoodieCompactionHandler { +public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable implements HoodieCompactionHandler { HoodieSparkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); @@ -177,7 +176,7 @@ public Option scheduleRollback(HoodieEngineContext context, @Override public Iterator> handleInsertsForLogCompaction(String instantTime, String partitionPath, String fileId, - Map> recordMap, + Map> recordMap, Map header) { HoodieAppendHandle appendHandle = new HoodieAppendHandle(config, instantTime, this, partitionPath, fileId, recordMap.values().iterator(), taskContextSupplier, header); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index 66d51c91283f3..789f044cf52f3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; @@ -44,7 +43,7 @@ import java.io.IOException; -public abstract class HoodieSparkTable +public abstract class HoodieSparkTable extends HoodieTable>, HoodieData, HoodieData> { private volatile boolean isMetadataTableExists = false; @@ -53,7 +52,7 @@ protected HoodieSparkTable(HoodieWriteConfig config, HoodieEngineContext context super(config, context, metaClient); } - public static HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context) { + public static HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) @@ -63,9 +62,9 @@ public static HoodieSparkTable create(HoodieW return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient); } - public static HoodieSparkTable create(HoodieWriteConfig config, - HoodieSparkEngineContext context, - HoodieTableMetaClient metaClient) { + public static HoodieSparkTable create(HoodieWriteConfig config, + HoodieSparkEngineContext context, + HoodieTableMetaClient metaClient) { HoodieSparkTable hoodieSparkTable; switch (metaClient.getTableType()) { case COPY_ON_WRITE: diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java index 96ac794dcbc82..d7a4a2b52abf7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.util.AvroOrcUtils; import org.apache.hudi.common.util.OrcReaderIterator; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; @@ -63,6 +64,10 @@ Schema getAvroSchema(Path sourceFilePath) throws IOException { @Override void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, Path sourceFilePath, KeyGeneratorInterface keyGenerator, String partitionPath, Schema avroSchema) throws Exception { + // TODO support spark orc reader + if (config.getRecordMerger().getRecordType() == HoodieRecordType.SPARK) { + throw new UnsupportedOperationException(); + } BoundedInMemoryExecutor wrapper = null; Reader orcReader = OrcFile.createReader(sourceFilePath, OrcFile.readerOptions(table.getHadoopConf())); TypeDescription orcSchema = orcReader.getSchema(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java index 5f45629ba8023..d008d7cf9fd96 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java @@ -20,34 +20,31 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; -import org.apache.hudi.client.bootstrap.BootstrapRecordPayload; -import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.util.ParquetReaderIterator; +import org.apache.hudi.common.model.MetadataValues; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.io.HoodieBootstrapHandle; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.keygen.KeyGeneratorInterface; import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; import java.io.IOException; +import java.util.Properties; class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler { private static final Logger LOG = LogManager.getLogger(ParquetBootstrapMetadataHandler.class); @@ -67,18 +64,28 @@ Schema getAvroSchema(Path sourceFilePath) throws IOException { @Override void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, Path sourceFilePath, KeyGeneratorInterface keyGenerator, String partitionPath, Schema avroSchema) throws Exception { - BoundedInMemoryExecutor wrapper = null; - ParquetReader reader = - AvroParquetReader.builder(sourceFilePath).withConf(table.getHadoopConf()).build(); + BoundedInMemoryExecutor wrapper = null; + HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType()) + .getFileReader(table.getHadoopConf(), sourceFilePath); try { - wrapper = new BoundedInMemoryExecutor(config.getWriteBufferLimitBytes(), - new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> { - String recKey = keyGenerator.getKey(inp).getRecordKey(); - GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA); - gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey); - BootstrapRecordPayload payload = new BootstrapRecordPayload(gr); - HoodieRecord rec = new HoodieAvroRecord(new HoodieKey(recKey, partitionPath), payload); - return rec; + wrapper = new BoundedInMemoryExecutor(config.getWriteBufferLimitBytes(), + reader.getRecordIterator(), new BootstrapRecordConsumer(bootstrapHandle), record -> { + try { + String recKey = record.getRecordKey(reader.getSchema(), Option.of(keyGenerator)); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be inserted into the queue later. + HoodieRecord hoodieRecord = record + .rewriteRecord(reader.getSchema(), config.getProps(), HoodieAvroUtils.RECORD_KEY_SCHEMA) + .copy(); + MetadataValues metadataValues = new MetadataValues().setRecordKey(recKey); + return hoodieRecord + .updateMetadataValues(HoodieAvroUtils.RECORD_KEY_SCHEMA, new Properties(), metadataValues) + .newInstance(new HoodieKey(recKey, partitionPath)); + } catch (IOException e) { + LOG.error("Unable to overrideMetadataFieldValue", e); + return null; + } }, table.getPreExecuteRunnable()); wrapper.execute(); } catch (Exception e) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 88f6a54e0da6c..e3524f4d70959 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -38,7 +38,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -86,7 +85,7 @@ import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE; import static org.apache.hudi.table.action.bootstrap.MetadataBootstrapHandlerFactory.getMetadataHandler; -public class SparkBootstrapCommitActionExecutor> +public class SparkBootstrapCommitActionExecutor extends BaseCommitActionExecutor>, HoodieData, HoodieData, HoodieBootstrapWriteMetadata>> { private static final Logger LOG = LogManager.getLogger(SparkBootstrapCommitActionExecutor.class); @@ -280,7 +279,7 @@ protected Option>> fullBootstrap(Lis properties, context); JavaRDD inputRecordsRDD = (JavaRDD) inputProvider.generateInputRecords("bootstrap_source", config.getBootstrapSourceBasePath(), - partitionFilesList); + partitionFilesList, config); // Start Full Bootstrap String bootstrapInstantTime = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; final HoodieInstant requested = new HoodieInstant( diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java index 0d2ac6ceef896..ea30cbf67851d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -31,7 +30,7 @@ import java.util.Map; -public class SparkBootstrapDeltaCommitActionExecutor> +public class SparkBootstrapDeltaCommitActionExecutor extends SparkBootstrapCommitActionExecutor { public SparkBootstrapDeltaCommitActionExecutor(HoodieSparkEngineContext context, 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 7d2a4c0baabe3..ee61b57a10979 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 @@ -22,7 +22,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.ClusteringUtils; @@ -33,7 +32,7 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; -public class SparkExecuteClusteringCommitActionExecutor> +public class SparkExecuteClusteringCommitActionExecutor extends BaseSparkCommitActionExecutor { private final HoodieClusteringPlan clusteringPlan; 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 8c7d9e41ea5f5..ac5b8555b053f 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 @@ -27,7 +27,6 @@ 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.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -81,7 +80,7 @@ import static org.apache.hudi.common.util.ClusteringUtils.getAllFileGroupsInPendingClusteringPlans; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE; -public abstract class BaseSparkCommitActionExecutor extends +public abstract class BaseSparkCommitActionExecutor extends BaseCommitActionExecutor>, HoodieData, HoodieData, HoodieWriteMetadata>> { private static final Logger LOG = LogManager.getLogger(BaseSparkCommitActionExecutor.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java index 65a45e1c6a047..231825875972b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java @@ -33,7 +33,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; 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.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -46,7 +45,7 @@ /** * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition). */ -public class SparkBucketIndexPartitioner> extends +public class SparkBucketIndexPartitioner extends SparkHoodiePartitioner { private final int numBuckets; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java index f4b01c887b068..9a1d5bf1cc16d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -33,7 +32,7 @@ import java.util.Map; -public class SparkBulkInsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { +public class SparkBulkInsertCommitActionExecutor extends BaseSparkCommitActionExecutor { private final HoodieData> inputRecordsRDD; private final Option bulkInsertPartitioner; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java index 5768520a05fb7..51407eb026d16 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -45,7 +44,7 @@ * @param */ @SuppressWarnings("checkstyle:LineLength") -public class SparkBulkInsertHelper extends BaseBulkInsertHelper>, +public class SparkBulkInsertHelper extends BaseBulkInsertHelper>, HoodieData, HoodieData, R> { private SparkBulkInsertHelper() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java index 8862981c2a2b7..d84279ad2bea2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -31,7 +30,7 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public class SparkBulkInsertPreppedCommitActionExecutor> +public class SparkBulkInsertPreppedCommitActionExecutor extends BaseSparkCommitActionExecutor { private final HoodieData> preppedInputRecordRdd; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java index a6fc996b71c31..9551ecae83f4b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java @@ -22,13 +22,12 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public class SparkDeleteCommitActionExecutor> +public class SparkDeleteCommitActionExecutor extends BaseSparkCommitActionExecutor { private final HoodieData keys; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java index 7cc0efb34efb9..49134d604d2d0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -47,7 +46,7 @@ import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; -public class SparkDeletePartitionCommitActionExecutor> +public class SparkDeletePartitionCommitActionExecutor extends SparkInsertOverwriteCommitActionExecutor { private List partitions; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkHoodiePartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkHoodiePartitioner.java index 4a5bff42153fb..3a74578edf0cc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkHoodiePartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkHoodiePartitioner.java @@ -18,7 +18,6 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; import org.apache.spark.Partitioner; @@ -26,7 +25,7 @@ /** * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition). */ -public abstract class SparkHoodiePartitioner> extends Partitioner +public abstract class SparkHoodiePartitioner extends Partitioner implements org.apache.hudi.table.action.commit.Partitioner { /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java index 479b51322ff32..93ed6f88792db 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java @@ -22,13 +22,12 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public class SparkInsertCommitActionExecutor> +public class SparkInsertCommitActionExecutor extends BaseSparkCommitActionExecutor { private final HoodieData> inputRecordsRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java index 518063ed34186..ee3b31cc577ba 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.collection.Pair; @@ -39,7 +38,7 @@ import java.util.Map; import java.util.stream.Collectors; -public class SparkInsertOverwriteCommitActionExecutor> +public class SparkInsertOverwriteCommitActionExecutor extends BaseSparkCommitActionExecutor { private final HoodieData> inputRecordsRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java index 93d0a8124c4ee..868c8e2b42a6f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; 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.model.WriteOperationType; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -35,7 +34,7 @@ import java.util.List; import java.util.Map; -public class SparkInsertOverwriteTableCommitActionExecutor> +public class SparkInsertOverwriteTableCommitActionExecutor extends SparkInsertOverwriteCommitActionExecutor { public SparkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java index ff1a7e2b9beeb..55e05a3205bf2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java @@ -22,13 +22,12 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public class SparkInsertPreppedCommitActionExecutor> +public class SparkInsertPreppedCommitActionExecutor extends BaseSparkCommitActionExecutor { private final HoodieData> preppedRecords; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java index ccee9cf5a7164..6b699f25c3dca 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java @@ -22,13 +22,12 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public class SparkUpsertCommitActionExecutor> +public class SparkUpsertCommitActionExecutor extends BaseSparkCommitActionExecutor { private final HoodieData> inputRecordsRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java index 73d408593bd37..08b758e602eee 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java @@ -22,13 +22,12 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public class SparkUpsertPreppedCommitActionExecutor> +public class SparkUpsertPreppedCommitActionExecutor extends BaseSparkCommitActionExecutor { private final HoodieData> preppedRecords; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java index 134cfd8d2c0b5..802c9b3b4c048 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; 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.model.HoodieWriteStat; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -61,7 +60,7 @@ /** * Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition). */ -public class UpsertPartitioner> extends SparkHoodiePartitioner { +public class UpsertPartitioner extends SparkHoodiePartitioner { private static final Logger LOG = LogManager.getLogger(UpsertPartitioner.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java index af3e4960ad071..8a3daaf4aac34 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -37,7 +36,7 @@ * a normal commit */ @SuppressWarnings("checkstyle:LineLength") -public class HoodieSparkMergeOnReadTableCompactor +public class HoodieSparkMergeOnReadTableCompactor extends HoodieCompactor>, HoodieData, HoodieData> { @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java index fe467ddbffbeb..3e55ffeef8290 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -43,7 +42,7 @@ import java.util.List; import java.util.Map; -public abstract class BaseSparkDeltaCommitActionExecutor> +public abstract class BaseSparkDeltaCommitActionExecutor extends BaseSparkCommitActionExecutor { private static final Logger LOG = LogManager.getLogger(BaseSparkDeltaCommitActionExecutor.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java index 190a714e44612..1855ed47a5329 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -34,7 +33,7 @@ import java.util.Map; -public class SparkBulkInsertDeltaCommitActionExecutor> +public class SparkBulkInsertDeltaCommitActionExecutor extends BaseSparkDeltaCommitActionExecutor { private final HoodieData> inputRecordsRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java index c01bce2b9cf35..fb4f43aa87fa8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -32,7 +31,7 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; -public class SparkBulkInsertPreppedDeltaCommitActionExecutor> +public class SparkBulkInsertPreppedDeltaCommitActionExecutor extends BaseSparkDeltaCommitActionExecutor { private final HoodieData> preppedInputRecordRdd; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java index 9a5b08df288d6..2aeb7bd3b1c8d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java @@ -22,14 +22,13 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.HoodieDeleteHelper; -public class SparkDeleteDeltaCommitActionExecutor> +public class SparkDeleteDeltaCommitActionExecutor extends BaseSparkDeltaCommitActionExecutor { private final HoodieData keys; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java index 4889460c467fa..1d6febc666eb0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java @@ -22,14 +22,13 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.HoodieWriteHelper; -public class SparkInsertDeltaCommitActionExecutor> +public class SparkInsertDeltaCommitActionExecutor extends BaseSparkDeltaCommitActionExecutor { private final HoodieData> inputRecordsRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java index dbf0cbc676118..d6d155be184ef 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java @@ -22,13 +22,12 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public class SparkInsertPreppedDeltaCommitActionExecutor> +public class SparkInsertPreppedDeltaCommitActionExecutor extends BaseSparkDeltaCommitActionExecutor { private final HoodieData> preppedRecords; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java index 67ecb9a8cbc06..270ac8640121a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java @@ -22,14 +22,13 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.HoodieWriteHelper; -public class SparkUpsertDeltaCommitActionExecutor> +public class SparkUpsertDeltaCommitActionExecutor extends BaseSparkDeltaCommitActionExecutor { private final HoodieData> inputRecordsRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java index e498019c415d8..a5dd9978939f7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; @@ -43,7 +42,7 @@ * UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet files to larger ones * without the need for an index in the logFile. */ -public class SparkUpsertDeltaCommitPartitioner> extends UpsertPartitioner { +public class SparkUpsertDeltaCommitPartitioner extends UpsertPartitioner { public SparkUpsertDeltaCommitPartitioner(WorkloadProfile profile, HoodieSparkEngineContext context, HoodieTable table, HoodieWriteConfig config) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java index 9540030eba157..7f8b21c57fa42 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java @@ -22,13 +22,12 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -public class SparkUpsertPreppedDeltaCommitActionExecutor> +public class SparkUpsertPreppedDeltaCommitActionExecutor extends BaseSparkDeltaCommitActionExecutor { private final HoodieData> preppedRecords; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java new file mode 100644 index 0000000000000..9a4aaa1dbc585 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.util; + +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; + +import org.apache.spark.sql.HoodieInternalRowUtils; +import org.apache.spark.sql.HoodieUnsafeRowUtils; +import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; + +public class HoodieSparkRecordUtils { + + public static Object getValue(StructType structType, String fieldName, InternalRow row) { + NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, fieldName); + return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList); + } + + /** + * Returns the string value of the given record {@code rec} and field {@code fieldName}. The field and value both could be missing. + * + * @param row The record + * @param fieldName The field name + * @return the string form of the field or empty if the schema does not contain the field name or the value is null + */ + public static Option getNullableValAsString(StructType structType, InternalRow row, String fieldName) { + String fieldVal = !HoodieInternalRowUtils.existField(structType, fieldName) + ? null : StringUtils.objToString(getValue(structType, fieldName, row)); + return Option.ofNullable(fieldVal); + } + + /** + * Gets record column values into one object. + * + * @param row InternalRow. + * @param columns Names of the columns to get values. + * @param structType {@link StructType} instance. + * @return Column value if a single column, or concatenated String values by comma. + */ + public static Object[] getRecordColumnValues(InternalRow row, + String[] columns, + StructType structType, boolean consistentLogicalTimestampEnabled) { + Object[] objects = new Object[columns.length]; + for (int i = 0; i < objects.length; i++) { + NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, columns[i]); + Object value = HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList); + objects[i] = value; + } + return objects; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala index 79fa67acdb9d9..ffd2d6071d16e 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala @@ -124,7 +124,7 @@ object HoodieDatasetBulkInsertHelper extends Logging { */ def bulkInsert(dataset: Dataset[Row], instantTime: String, - table: HoodieTable[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]], _, _, _], + table: HoodieTable[_, _, _, _], writeConfig: HoodieWriteConfig, partitioner: BulkInsertPartitioner[Dataset[Row]], parallelism: Int, diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala new file mode 100644 index 0000000000000..799bda2f8e5ba --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi + +import org.apache.hudi.common.model.HoodieRecord +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrameUtil, Dataset, Row, SparkSession} + +object SparkConversionUtils { + + def createDataFrame[T](rdd: RDD[HoodieRecord[T]], ss: SparkSession, structType: StructType): Dataset[Row] = { + if (rdd.isEmpty()) { + ss.emptyDataFrame + } else { + DataFrameUtil.createFromInternalRows(ss, structType, rdd.map(_.getData.asInstanceOf[InternalRow])) + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/unsafe/UTF8StringBuilder.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/unsafe/UTF8StringBuilder.java index 3d9f060515a6d..0e166360b0273 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/unsafe/UTF8StringBuilder.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/unsafe/UTF8StringBuilder.java @@ -52,8 +52,8 @@ public UTF8StringBuilder(int initialSize) { private void grow(int neededSize) { if (neededSize > ARRAY_MAX - totalSize()) { throw new UnsupportedOperationException( - "Cannot grow internal buffer by size " + neededSize + " because the size after growing " + - "exceeds size limitation " + ARRAY_MAX); + "Cannot grow internal buffer by size " + neededSize + " because the size after growing " + + "exceeds size limitation " + ARRAY_MAX); } final int length = totalSize() + neededSize; if (buffer.length < length) { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoProvider.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoProvider.scala new file mode 100644 index 0000000000000..9dcb911971c8b --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoProvider.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import com.esotericsoftware.kryo.Kryo +import org.apache.hudi.client.model.HoodieInternalRow +import org.apache.hudi.common.model.{HoodieKey, HoodieSparkRecord} +import org.apache.hudi.common.util.HoodieCommonKryoProvider +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.serializer.KryoRegistrator + +/** + * NOTE: PLEASE READ CAREFULLY BEFORE CHANGING + * + * This class is responsible for registering Hudi specific components that are often + * serialized by Kryo (for ex, during Spark's Shuffling operations) to make sure Kryo + * doesn't need to serialize their full class-names (for every object) which will quickly + * add up to considerable amount of overhead. + * + * Please note of the following: + *
    + *
  1. Ordering of the registration COULD NOT change as it's directly impacting + * associated class ids (on the Kryo side)
  2. + *
  3. This class might be loaded up using reflection and as such should not be relocated + * or renamed (w/o correspondingly updating such usages)
  4. + *
+ */ +class HoodieSparkKryoProvider extends HoodieCommonKryoProvider { + override def registerClasses(): Array[Class[_]] = { + /////////////////////////////////////////////////////////////////////////// + // NOTE: DO NOT REORDER REGISTRATIONS + /////////////////////////////////////////////////////////////////////////// + val classes = super[HoodieCommonKryoProvider].registerClasses() + classes ++ Array( + classOf[HoodieWriteConfig], + classOf[HoodieSparkRecord], + classOf[HoodieInternalRow] + ) + } +} + +object HoodieSparkKryoProvider { + def register(conf: SparkConf): SparkConf = { + conf.registerKryoClasses(new HoodieSparkKryoProvider().registerClasses()) + } +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/DataFrameUtil.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/DataFrameUtil.scala new file mode 100644 index 0000000000000..f05da8a0ce85a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/DataFrameUtil.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.LogicalRDD +import org.apache.spark.sql.types.StructType + +object DataFrameUtil { + + /** + * Creates a DataFrame out of RDD[InternalRow] that you can get + * using `df.queryExection.toRdd` + */ + def createFromInternalRows(sparkSession: SparkSession, schema: + StructType, rdd: RDD[InternalRow]): DataFrame = { + val logicalPlan = LogicalRDD(schema.toAttributes, rdd)(sparkSession) + Dataset.ofRows(sparkSession, logicalPlan) + } +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala index f2d6f0381a471..24e50613f50ab 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql import org.apache.hudi.SparkAdapterSupport.sparkAdapter import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction} -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Like, Literal, SubqueryExpression, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateMutableProjection, GenerateUnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Like, Literal, MutableProjection, SubqueryExpression, UnsafeProjection} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} +import scala.annotation.tailrec trait HoodieCatalystExpressionUtils { @@ -108,6 +109,23 @@ object HoodieCatalystExpressionUtils { }) } + /** + * Generates instance of [[MutableProjection]] projecting row of one [[StructType]] into another [[StructType]] + * + * NOTE: No safety checks are executed to validate that this projection is actually feasible, + * it's up to the caller to make sure that such projection is possible. + * + * NOTE: Projection of the row from [[StructType]] A to [[StructType]] B is only possible, if + * B is a subset of A + */ + def generateMutableProjection(from: StructType, to: StructType): MutableProjection = { + val attrs = from.toAttributes + val attrsMap = attrs.map(attr => (attr.name, attr)).toMap + val targetExprs = to.fields.map(f => attrsMap(f.name)) + + GenerateMutableProjection.generate(targetExprs, attrs) + } + /** * Parses and resolves expression against the attributes of the given table schema. * diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala new file mode 100644 index 0000000000000..dfe3295cf003c --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala @@ -0,0 +1,321 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.nio.charset.StandardCharsets +import java.util.HashMap +import java.util.concurrent.ConcurrentHashMap +import org.apache.avro.Schema +import org.apache.hbase.thirdparty.com.google.common.base.Supplier +import org.apache.hudi.AvroConversionUtils +import org.apache.hudi.avro.HoodieAvroUtils.{createFullName, toJavaDate} +import org.apache.hudi.exception.HoodieException +import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String +import scala.collection.mutable + +object HoodieInternalRowUtils { + + // Projection are all thread local. Projection is not thread-safe + val unsafeProjectionThreadLocal: ThreadLocal[HashMap[(StructType, StructType), UnsafeProjection]] = + ThreadLocal.withInitial(new Supplier[HashMap[(StructType, StructType), UnsafeProjection]] { + override def get(): HashMap[(StructType, StructType), UnsafeProjection] = new HashMap[(StructType, StructType), UnsafeProjection] + }) + val schemaMap = new ConcurrentHashMap[Schema, StructType] + val orderPosListMap = new ConcurrentHashMap[(StructType, String), NestedFieldPath] + + /** + * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecord(org.apache.avro.generic.GenericRecord, org.apache.avro.Schema) + */ + def rewriteRecord(oldRecord: InternalRow, oldSchema: StructType, newSchema: StructType): InternalRow = { + val newRow = new GenericInternalRow(Array.fill(newSchema.fields.length)(null).asInstanceOf[Array[Any]]) + + for ((field, pos) <- newSchema.fields.zipWithIndex) { + var oldValue: AnyRef = null + var oldType: DataType = null + if (existField(oldSchema, field.name)) { + val oldField = oldSchema(field.name) + val oldPos = oldSchema.fieldIndex(field.name) + oldType = oldField.dataType + oldValue = oldRecord.get(oldPos, oldType) + } + if (oldValue != null) { + field.dataType match { + case structType: StructType => + val oldType = oldSchema(field.name).dataType.asInstanceOf[StructType] + val newValue = rewriteRecord(oldValue.asInstanceOf[InternalRow], oldType, structType) + newRow.update(pos, newValue) + case decimalType: DecimalType => + val oldFieldSchema = oldSchema(field.name).dataType.asInstanceOf[DecimalType] + if (decimalType.scale != oldFieldSchema.scale || decimalType.precision != oldFieldSchema.precision) { + newRow.update(pos, Decimal.fromDecimal(oldValue.asInstanceOf[Decimal].toBigDecimal.setScale(newSchema.asInstanceOf[DecimalType].scale)) + ) + } else { + newRow.update(pos, oldValue) + } + case t if t == oldType => newRow.update(pos, oldValue) + // Type promotion + case _: ShortType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toShort) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: IntegerType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toInt) + case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toInt) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: LongType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toLong) + case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toLong) + case _: IntegerType => newRow.update(pos, oldValue.asInstanceOf[Int].toLong) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: FloatType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toFloat) + case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toFloat) + case _: IntegerType => newRow.update(pos, oldValue.asInstanceOf[Int].toFloat) + case _: LongType => newRow.update(pos, oldValue.asInstanceOf[Long].toFloat) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: DoubleType => + oldType match { + case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toDouble) + case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toDouble) + case _: IntegerType => newRow.update(pos, oldValue.asInstanceOf[Int].toDouble) + case _: LongType => newRow.update(pos, oldValue.asInstanceOf[Long].toDouble) + case _: FloatType => newRow.update(pos, oldValue.asInstanceOf[Float].toDouble) + case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") + } + case _: BinaryType if oldType.isInstanceOf[StringType] => newRow.update(pos, oldValue.asInstanceOf[String].getBytes) + case _ => newRow.update(pos, oldValue) + } + } else { + // TODO default value in newSchema + } + } + + newRow + } + + /** + * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecordWithNewSchema(org.apache.avro.generic.IndexedRecord, org.apache.avro.Schema, java.util.Map) + */ + def rewriteRecordWithNewSchema(oldRecord: InternalRow, oldSchema: StructType, newSchema: StructType, renameCols: java.util.Map[String, String]): InternalRow = { + rewriteRecordWithNewSchema(oldRecord, oldSchema, newSchema, renameCols, new java.util.LinkedList[String]).asInstanceOf[InternalRow] + } + + /** + * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecordWithNewSchema(java.lang.Object, org.apache.avro.Schema, org.apache.avro.Schema, java.util.Map, java.util.Deque) + */ + private def rewriteRecordWithNewSchema(oldRecord: Any, oldSchema: DataType, newSchema: DataType, renameCols: java.util.Map[String, String], fieldNames: java.util.Deque[String]): Any = { + if (oldRecord == null) { + null + } else { + newSchema match { + case targetSchema: StructType => + if (!oldRecord.isInstanceOf[InternalRow]) { + throw new IllegalArgumentException("cannot rewrite record with different type") + } + val oldRow = oldRecord.asInstanceOf[InternalRow] + val helper = mutable.Map[Integer, Any]() + + val oldStrucType = oldSchema.asInstanceOf[StructType] + targetSchema.fields.zipWithIndex.foreach { case (field, i) => + fieldNames.push(field.name) + if (existField(oldStrucType, field.name)) { + val oldField = oldStrucType(field.name) + val oldPos = oldStrucType.fieldIndex(field.name) + helper(i) = rewriteRecordWithNewSchema(oldRow.get(oldPos, oldField.dataType), oldField.dataType, field.dataType, renameCols, fieldNames) + } else { + val fieldFullName = createFullName(fieldNames) + val colNamePartsFromOldSchema = renameCols.getOrDefault(fieldFullName, "").split("\\.") + val lastColNameFromOldSchema = colNamePartsFromOldSchema(colNamePartsFromOldSchema.length - 1) + // deal with rename + if (!existField(oldStrucType, field.name) && existField(oldStrucType, lastColNameFromOldSchema)) { + // find rename + val oldField = oldStrucType(lastColNameFromOldSchema) + val oldPos = oldStrucType.fieldIndex(lastColNameFromOldSchema) + helper(i) = rewriteRecordWithNewSchema(oldRow.get(oldPos, oldField.dataType), oldField.dataType, field.dataType, renameCols, fieldNames) + } + } + fieldNames.pop() + } + val newRow = new GenericInternalRow(Array.fill(targetSchema.length)(null).asInstanceOf[Array[Any]]) + targetSchema.fields.zipWithIndex.foreach { case (_, i) => + if (helper.contains(i)) { + newRow.update(i, helper(i)) + } else { + // TODO add default val + newRow.update(i, null) + } + } + + newRow + case targetSchema: ArrayType => + if (!oldRecord.isInstanceOf[ArrayData]) { + throw new IllegalArgumentException("cannot rewrite record with different type") + } + val oldElementType = oldSchema.asInstanceOf[ArrayType].elementType + val oldArray = oldRecord.asInstanceOf[ArrayData] + val newElementType = targetSchema.elementType + val newArray = new GenericArrayData(Array.fill(oldArray.numElements())(null).asInstanceOf[Array[Any]]) + fieldNames.push("element") + oldArray.toSeq[Any](oldElementType).zipWithIndex.foreach { case (value, i) => newArray.update(i, rewriteRecordWithNewSchema(value.asInstanceOf[AnyRef], oldElementType, newElementType, renameCols, fieldNames)) } + fieldNames.pop() + + newArray + case targetSchema: MapType => + if (!oldRecord.isInstanceOf[MapData]) { + throw new IllegalArgumentException("cannot rewrite record with different type") + } + val oldValueType = oldSchema.asInstanceOf[MapType].valueType + val oldKeyType = oldSchema.asInstanceOf[MapType].keyType + val oldMap = oldRecord.asInstanceOf[MapData] + val newValueType = targetSchema.valueType + val newKeyArray = new GenericArrayData(Array.fill(oldMap.keyArray().numElements())(null).asInstanceOf[Array[Any]]) + val newValueArray = new GenericArrayData(Array.fill(oldMap.valueArray().numElements())(null).asInstanceOf[Array[Any]]) + val newMap = new ArrayBasedMapData(newKeyArray, newValueArray) + fieldNames.push("value") + oldMap.keyArray().toSeq[Any](oldKeyType).zipWithIndex.foreach { case (value, i) => newKeyArray.update(i, rewritePrimaryType(value, oldKeyType, oldKeyType)) } + oldMap.valueArray().toSeq[Any](oldValueType).zipWithIndex.foreach { case (value, i) => newValueArray.update(i, rewriteRecordWithNewSchema(value.asInstanceOf[AnyRef], oldValueType, newValueType, renameCols, fieldNames)) } + fieldNames.pop() + + newMap + case _ => rewritePrimaryType(oldRecord, oldSchema, newSchema) + } + } + } + + def getCachedPosList(structType: StructType, field: String): NestedFieldPath = { + val schemaPair = (structType, field) + if (!orderPosListMap.containsKey(schemaPair)) { + val posList = HoodieUnsafeRowUtils.composeNestedFieldPath(structType, field) + orderPosListMap.put(schemaPair, posList) + } + orderPosListMap.get(schemaPair) + } + + def getCachedUnsafeProjection(from: StructType, to: StructType): UnsafeProjection = { + val schemaPair = (from, to) + val map = unsafeProjectionThreadLocal.get() + if (!map.containsKey(schemaPair)) { + val projection = HoodieCatalystExpressionUtils.generateUnsafeProjection(from, to) + map.put(schemaPair, projection) + } + map.get(schemaPair) + } + + def getCachedSchema(schema: Schema): StructType = { + if (!schemaMap.containsKey(schema)) { + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + schemaMap.put(schema, structType) + } + schemaMap.get(schema) + } + + def existField(structType: StructType, name: String): Boolean = { + try { + HoodieUnsafeRowUtils.composeNestedFieldPath(structType, name) + true + } catch { + case _: IllegalArgumentException => false + } + } + + private def rewritePrimaryType(oldValue: Any, oldSchema: DataType, newSchema: DataType) = { + if (oldSchema.equals(newSchema) || (oldSchema.isInstanceOf[DecimalType] && newSchema.isInstanceOf[DecimalType])) { + oldSchema match { + case NullType | BooleanType | IntegerType | LongType | FloatType | DoubleType | DateType | TimestampType | BinaryType => + oldValue + // Copy UTF8String before putting into GenericInternalRow + case StringType => UTF8String.fromString(oldValue.toString) + case DecimalType() => + Decimal.fromDecimal(oldValue.asInstanceOf[Decimal].toBigDecimal.setScale(newSchema.asInstanceOf[DecimalType].scale)) + case _ => + throw new HoodieException("Unknown schema type: " + newSchema) + } + } else { + rewritePrimaryTypeWithDiffSchemaType(oldValue, oldSchema, newSchema) + } + } + + private def rewritePrimaryTypeWithDiffSchemaType(oldValue: Any, oldSchema: DataType, newSchema: DataType): Any = { + val value = newSchema match { + case NullType | BooleanType => + case DateType if oldSchema.equals(StringType) => + CatalystTypeConverters.convertToCatalyst(java.sql.Date.valueOf(oldValue.toString)) + case LongType => + oldSchema match { + case IntegerType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Int].longValue()) + case _ => + } + case FloatType => + oldSchema match { + case IntegerType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Int].floatValue()) + case LongType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Long].floatValue()) + case _ => + } + case DoubleType => + oldSchema match { + case IntegerType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Int].doubleValue()) + case LongType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Long].doubleValue()) + case FloatType => CatalystTypeConverters.convertToCatalyst(java.lang.Double.valueOf(oldValue.asInstanceOf[Float] + "")) + case _ => + } + case BinaryType => + oldSchema match { + case StringType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[String].getBytes(StandardCharsets.UTF_8)) + case _ => + } + case StringType => + oldSchema match { + case BinaryType => CatalystTypeConverters.convertToCatalyst(new String(oldValue.asInstanceOf[Array[Byte]])) + case DateType => CatalystTypeConverters.convertToCatalyst(toJavaDate(oldValue.asInstanceOf[Integer]).toString) + case IntegerType | LongType | FloatType | DoubleType | DecimalType() => CatalystTypeConverters.convertToCatalyst(oldValue.toString) + case _ => + } + case DecimalType() => + oldSchema match { + case IntegerType | LongType | FloatType | DoubleType | StringType => + val scale = newSchema.asInstanceOf[DecimalType].scale + + Decimal.fromDecimal(BigDecimal(oldValue.toString).setScale(scale)) + case _ => + } + case _ => + } + if (value == None) { + throw new HoodieException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema)) + } else { + CatalystTypeConverters.convertToCatalyst(value) + } + } + + def removeFields(schema: StructType, fieldsToRemove: java.util.List[String]): StructType = { + StructType(schema.fields.filter(field => !fieldsToRemove.contains(field.name))) + } +} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 31fe21f941903..ca50d82eae915 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -46,6 +46,11 @@ import java.util.Locale */ trait SparkAdapter extends Serializable { + /** + * Checks whether provided instance of [[InternalRow]] is actually an instance of [[ColumnarBatchRow]] + */ + def isColumnarBatchRow(r: InternalRow): Boolean + /** * Inject table-valued functions to SparkSessionExtensions */ 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 d49014c69e757..8e3252a8462a0 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 @@ -20,6 +20,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -134,7 +135,8 @@ private void assertSchemaEvolutionOnUpdateResult(WriteStatus insertResult, Hoodi new Path(updateTable.getConfig().getBasePath() + "/" + insertResult.getStat().getPath()), mergeHandle.getWriterSchemaWithMetaFields()); for (GenericRecord rec : oldRecords) { - mergeHandle.write(rec); + // TODO create hoodie record with rec can getRecordKey + mergeHandle.write(new HoodieAvroIndexedRecord(rec)); } mergeHandle.close(); }; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java index eec79756bd767..7c03a02f5fcc2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java @@ -29,7 +29,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.collection.Triple; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.bucket.ConsistentBucketIdentifier; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java index d970d593d0fe8..a58c2ce8b5761 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java @@ -33,7 +33,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hadoop.HoodieParquetInputFormat; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java index ad58de17d8661..ca55e10703f1d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -40,7 +41,6 @@ import org.apache.hudi.config.HoodieCleanConfig; 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.HoodieIndex; import org.apache.hudi.testutils.GenericRecordValidationTestUtils; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index a82904a8f4d43..26c8ef3273437 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -44,6 +44,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; @@ -79,11 +80,11 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieLockConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.io.storage.HoodieHFileReader; +import org.apache.hudi.io.storage.HoodieAvroHFileReader; import org.apache.hudi.metadata.FileSystemBackedTableMetadata; import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader; @@ -357,7 +358,7 @@ public void testTurnOffMetadataTableAfterEnable() throws Exception { assertTrue(metadataWriter.isPresent()); HoodieTableConfig hoodieTableConfig = - new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig.getPayloadClass()); + new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig.getPayloadClass(), writeConfig.getStringOrDefault(HoodieWriteConfig.MERGER_IMPLS)); assertFalse(hoodieTableConfig.getMetadataPartitions().isEmpty()); // Turn off metadata table @@ -374,7 +375,7 @@ public void testTurnOffMetadataTableAfterEnable() throws Exception { assertFalse(metadataWriter2.isPresent()); HoodieTableConfig hoodieTableConfig2 = - new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig2.getPayloadClass()); + new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig2.getPayloadClass(), writeConfig.getStringOrDefault(HoodieWriteConfig.MERGER_IMPLS)); assertEquals(Collections.emptySet(), hoodieTableConfig2.getMetadataPartitions()); // Assert metadata table folder is deleted assertFalse(metaClient.getFs().exists( @@ -396,7 +397,7 @@ public void testTurnOffMetadataTableAfterEnable() throws Exception { validateMetadata(testTable, true); assertTrue(metadataWriter3.isPresent()); HoodieTableConfig hoodieTableConfig3 = - new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig.getPayloadClass()); + new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig.getPayloadClass(), writeConfig.getStringOrDefault(HoodieWriteConfig.MERGER_IMPLS)); assertFalse(hoodieTableConfig3.getMetadataPartitions().isEmpty()); } @@ -764,9 +765,9 @@ public void testVirtualKeysInBaseFiles(boolean populateMetaFields) throws Except table.getHoodieView().sync(); List fileSlices = table.getSliceView().getLatestFileSlices("files").collect(Collectors.toList()); HoodieBaseFile baseFile = fileSlices.get(0).getBaseFile().get(); - HoodieHFileReader hoodieHFileReader = new HoodieHFileReader(context.getHadoopConf().get(), new Path(baseFile.getPath()), + HoodieAvroHFileReader hoodieHFileReader = new HoodieAvroHFileReader(context.getHadoopConf().get(), new Path(baseFile.getPath()), new CacheConfig(context.getHadoopConf().get())); - List records = HoodieHFileReader.readAllRecords(hoodieHFileReader); + List records = HoodieAvroHFileReader.readAllRecords(hoodieHFileReader); records.forEach(entry -> { if (populateMetaFields) { assertNotNull(((GenericRecord) entry).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); @@ -1011,9 +1012,9 @@ private void verifyMetadataRawRecords(HoodieTable table, List log while (logFileReader.hasNext()) { HoodieLogBlock logBlock = logFileReader.next(); if (logBlock instanceof HoodieDataBlock) { - try (ClosableIterator recordItr = ((HoodieDataBlock) logBlock).getRecordIterator()) { + try (ClosableIterator> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator(HoodieRecordType.AVRO)) { recordItr.forEachRemaining(indexRecord -> { - final GenericRecord record = (GenericRecord) indexRecord; + final GenericRecord record = (GenericRecord) indexRecord.getData(); if (enableMetaFields) { // Metadata table records should have meta fields! assertNotNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); @@ -1070,7 +1071,7 @@ private void verifyMetadataMergedRecords(HoodieTableMetaClient metadataMetaClien logRecordReader.scan(); }, "Metadata log records materialization failed"); - for (Map.Entry> entry : logRecordReader.getRecords().entrySet()) { + for (Map.Entry entry : logRecordReader.getRecords().entrySet()) { assertFalse(entry.getKey().isEmpty()); assertFalse(entry.getValue().getRecordKey().isEmpty()); assertEquals(entry.getKey(), entry.getValue().getRecordKey()); @@ -1094,10 +1095,10 @@ private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable tabl } final HoodieBaseFile baseFile = fileSlices.get(0).getBaseFile().get(); - HoodieHFileReader hoodieHFileReader = new HoodieHFileReader(context.getHadoopConf().get(), + HoodieAvroHFileReader hoodieHFileReader = new HoodieAvroHFileReader(context.getHadoopConf().get(), new Path(baseFile.getPath()), new CacheConfig(context.getHadoopConf().get())); - List records = HoodieHFileReader.readAllRecords(hoodieHFileReader); + List records = HoodieAvroHFileReader.readAllRecords(hoodieHFileReader); records.forEach(entry -> { if (enableMetaFields) { assertNotNull(((GenericRecord) entry).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); @@ -2562,9 +2563,9 @@ private void verifyMetadataColumnStatsRecords(List logFiles) thro while (logFileReader.hasNext()) { HoodieLogBlock logBlock = logFileReader.next(); if (logBlock instanceof HoodieDataBlock) { - try (ClosableIterator recordItr = ((HoodieDataBlock) logBlock).getRecordIterator()) { + try (ClosableIterator> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator(HoodieRecordType.AVRO)) { recordItr.forEachRemaining(indexRecord -> { - final GenericRecord record = (GenericRecord) indexRecord; + final GenericRecord record = (GenericRecord) indexRecord.getData(); final GenericRecord colStatsRecord = (GenericRecord) record.get(HoodieMetadataPayload.SCHEMA_FIELD_ID_COLUMN_STATS); assertNotNull(colStatsRecord); assertNotNull(colStatsRecord.get(HoodieMetadataPayload.COLUMN_STATS_FIELD_COLUMN_NAME)); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java index 719f914816740..3675f7e8f7175 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java @@ -25,7 +25,7 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; @@ -37,7 +37,7 @@ import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.storage.HoodieHFileReader; +import org.apache.hudi.io.storage.HoodieAvroHFileReader; import org.apache.hudi.metadata.HoodieBackedTableMetadata; import org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader; import org.apache.hudi.metadata.HoodieMetadataPayload; @@ -353,9 +353,9 @@ private void verifyMetadataRawRecords(HoodieTable table, List log while (logFileReader.hasNext()) { HoodieLogBlock logBlock = logFileReader.next(); if (logBlock instanceof HoodieDataBlock) { - try (ClosableIterator recordItr = ((HoodieDataBlock) logBlock).getRecordIterator()) { + try (ClosableIterator> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator(HoodieRecordType.AVRO)) { recordItr.forEachRemaining(indexRecord -> { - final GenericRecord record = (GenericRecord) indexRecord; + final GenericRecord record = (GenericRecord) indexRecord.getData(); assertNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); assertNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)); final String key = String.valueOf(record.get(HoodieMetadataPayload.KEY_FIELD_NAME)); @@ -396,7 +396,7 @@ private void verifyMetadataMergedRecords(HoodieTableMetaClient metadataMetaClien logRecordReader.scan(); }, "Metadata log records materialization failed"); - for (Map.Entry> entry : logRecordReader.getRecords().entrySet()) { + for (Map.Entry entry : logRecordReader.getRecords().entrySet()) { assertFalse(entry.getKey().isEmpty()); assertFalse(entry.getValue().getRecordKey().isEmpty()); assertEquals(entry.getKey(), entry.getValue().getRecordKey()); @@ -419,10 +419,10 @@ private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable tabl } final HoodieBaseFile baseFile = fileSlices.get(0).getBaseFile().get(); - HoodieHFileReader hoodieHFileReader = new HoodieHFileReader(context.getHadoopConf().get(), + HoodieAvroHFileReader hoodieHFileReader = new HoodieAvroHFileReader(context.getHadoopConf().get(), new Path(baseFile.getPath()), new CacheConfig(context.getHadoopConf().get())); - List records = HoodieHFileReader.readAllRecords(hoodieHFileReader); + List records = HoodieAvroHFileReader.readAllRecords(hoodieHFileReader); records.forEach(entry -> { assertNull(((GenericRecord) entry).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); final String keyInPayload = (String) ((GenericRecord) entry) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index 9002866a01874..ba32aea0b3463 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -31,6 +31,7 @@ import org.apache.hudi.client.validator.SparkPreCommitValidator; import org.apache.hudi.client.validator.SqlQueryEqualityPreCommitValidator; import org.apache.hudi.client.validator.SqlQuerySingleResultPreCommitValidator; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -38,12 +39,14 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; @@ -68,6 +71,7 @@ import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -78,7 +82,6 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; -import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieCommitException; @@ -469,8 +472,11 @@ private void testDeduplication( // Global dedup should be done based on recordKey only HoodieIndex index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(true); + HoodieRecordMerger recordMerger = HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()); int dedupParallelism = records.getNumPartitions() + 100; - HoodieData> dedupedRecsRdd = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema()); + HoodieData> dedupedRecsRdd = + (HoodieData>) HoodieWriteHelper.newInstance() + .deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema(), writeConfig.getProps(), recordMerger); List> dedupedRecs = dedupedRecsRdd.collectAsList(); assertEquals(records.getNumPartitions(), dedupedRecsRdd.getNumPartitions()); assertEquals(1, dedupedRecs.size()); @@ -480,7 +486,9 @@ private void testDeduplication( // non-Global dedup should be done based on both recordKey and partitionPath index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(false); - dedupedRecsRdd = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema()); + dedupedRecsRdd = + (HoodieData>) HoodieWriteHelper.newInstance() + .deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema(), writeConfig.getProps(), recordMerger); dedupedRecs = dedupedRecsRdd.collectAsList(); assertEquals(records.getNumPartitions(), dedupedRecsRdd.getNumPartitions()); assertEquals(2, dedupedRecs.size()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java index 1def851949ec0..026a607f069d2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java @@ -22,6 +22,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieLogFile; @@ -35,6 +36,7 @@ import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCleanConfig; @@ -443,6 +445,7 @@ private void validateBlockInstantsBeforeAndAfterRollback(HoodieWriteConfig confi .withLatestInstantTime(instant) .withBufferSize(config.getMaxDFSStreamBufferSize()) .withUseScanV2(true) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); scanner.scanInternal(Option.empty(), true); List prevInstants = scanner.getValidBlockInstants(); @@ -456,6 +459,7 @@ private void validateBlockInstantsBeforeAndAfterRollback(HoodieWriteConfig confi .withLatestInstantTime(currentInstant) .withBufferSize(config.getMaxDFSStreamBufferSize()) .withUseScanV2(true) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); scanner2.scanInternal(Option.empty(), true); List currentInstants = scanner2.getValidBlockInstants(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index 8c92f8189f762..d6d5b2495a32b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -38,7 +38,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieLayoutConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index 29c653daee61a..6bb95028523b1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -21,6 +21,7 @@ import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.client.HoodieTimelineArchiver; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -39,7 +40,6 @@ import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java index 6763cb154f614..9dcd1d876a0e1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java @@ -36,7 +36,7 @@ import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java index 26b8fc36465a5..3ccdf1ec10690 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java @@ -41,7 +41,7 @@ import scala.Tuple2; -import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getCloningTransformer; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -95,7 +95,7 @@ public Integer finish() { BoundedInMemoryExecutor>, Integer> executor = null; try { executor = new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); int result = executor.execute(); assertEquals(100, result); @@ -137,7 +137,7 @@ public Integer finish() { BoundedInMemoryExecutor>, Integer> executor = new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); // Interrupt the current thread (therefore triggering executor to throw as soon as it // invokes [[get]] on the [[CompletableFuture]]) @@ -182,7 +182,7 @@ public Integer finish() { BoundedInMemoryExecutor>, Integer> executor = new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), unboundedRecordIter, - consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); executor.shutdownNow(); boolean terminatedGracefully = executor.awaitTermination(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java index 0826d192ee73d..c9be18c9da3da 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java @@ -44,6 +44,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.Semaphore; @@ -53,7 +54,7 @@ import scala.Tuple2; -import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getCloningTransformer; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -84,7 +85,7 @@ public void testRecordReading() throws Exception { final int numRecords = 128; final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); final BoundedInMemoryQueue queue = - new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(FileIOUtils.KB, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce Future resFuture = executorService.submit(() -> { new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue); @@ -97,12 +98,12 @@ public void testRecordReading() throws Exception { final HoodieAvroRecord originalRecord = (HoodieAvroRecord) originalRecordIterator.next(); final Option originalInsertValue = originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA); - final HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = queue.iterator().next(); + final HoodieLazyInsertIterable.HoodieInsertValueGenResult genResult = queue.iterator().next(); // Ensure that record ordering is guaranteed. - assertEquals(originalRecord, payload.record); + assertEquals(originalRecord, genResult.getResult()); // cached insert value matches the expected insert value. assertEquals(originalInsertValue, - ((HoodieAvroRecord) payload.record).getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA)); + ((HoodieAvroRecord) genResult.getResult()).getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA)); recordsRead++; } assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext()); @@ -124,7 +125,7 @@ public void testCompositeProducerRecordReading() throws Exception { final List> recs = new ArrayList<>(); final BoundedInMemoryQueue queue = - new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(FileIOUtils.KB, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); // Record Key to Map> keyToProducerAndIndexMap = new HashMap<>(); @@ -189,8 +190,8 @@ public void testCompositeProducerRecordReading() throws Exception { // Read recs and ensure we have covered all producer recs. while (queue.iterator().hasNext()) { - final HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = queue.iterator().next(); - final HoodieRecord rec = payload.record; + final HoodieLazyInsertIterable.HoodieInsertValueGenResult genResult = queue.iterator().next(); + final HoodieRecord rec = genResult.getResult(); Tuple2 producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); Integer lastSeenPos = lastSeenMap.get(producerPos._1()); countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1); @@ -218,12 +219,12 @@ public void testMemoryLimitForBuffering() throws Exception { // maximum number of records to keep in memory. final int recordLimit = 5; final SizeEstimator sizeEstimator = new DefaultSizeEstimator<>(); - HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply((HoodieAvroRecord) hoodieRecords.get(0)); - final long objSize = sizeEstimator.sizeEstimate(payload); + HoodieLazyInsertIterable.HoodieInsertValueGenResult genResult = + getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA).apply((HoodieAvroRecord) hoodieRecords.get(0)); + final long objSize = sizeEstimator.sizeEstimate(genResult); final long memoryLimitInBytes = recordLimit * objSize; final BoundedInMemoryQueue queue = - new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(memoryLimitInBytes, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce executorService.submit(() -> { @@ -240,8 +241,8 @@ public void testMemoryLimitForBuffering() throws Exception { assertEquals(recordLimit - 1, queue.samplingRecordCounter.get()); // try to read 2 records. - assertEquals(hoodieRecords.get(0), queue.iterator().next().record); - assertEquals(hoodieRecords.get(1), queue.iterator().next().record); + assertEquals(hoodieRecords.get(0), queue.iterator().next().getResult()); + assertEquals(hoodieRecords.get(1), queue.iterator().next().getResult()); // waiting for permits to expire. while (!isQueueFull(queue.rateLimiter)) { @@ -267,16 +268,16 @@ public void testException() throws Exception { final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); final SizeEstimator>> sizeEstimator = new DefaultSizeEstimator<>(); // queue memory limit - HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply((HoodieAvroRecord) hoodieRecords.get(0)); - final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue)); + HoodieLazyInsertIterable.HoodieInsertValueGenResult genResult = + getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0)); + final long objSize = sizeEstimator.sizeEstimate(new Tuple2(genResult.getResult(), genResult.getResult().toIndexedRecord(HoodieTestDataGenerator.AVRO_SCHEMA, new Properties()))); final long memoryLimitInBytes = 4 * objSize; // first let us throw exception from queueIterator reader and test that queueing thread // stops and throws // correct exception back. BoundedInMemoryQueue>> queue1 = - new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(memoryLimitInBytes, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce Future resFuture = executorService.submit(() -> { @@ -304,7 +305,7 @@ public void testException() throws Exception { when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); BoundedInMemoryQueue>> queue2 = - new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(memoryLimitInBytes, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce Future res = executorService.submit(() -> { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index b3ba5413c25dd..674aca7415395 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -18,8 +18,6 @@ package org.apache.hudi.execution; -import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; - import org.apache.avro.generic.IndexedRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -43,6 +41,7 @@ import scala.Tuple2; +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getCloningTransformer; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -85,7 +84,7 @@ public void testExecutor() { @Override public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { - consumedRecords.add(record.record); + consumedRecords.add(record.getResult()); count++; } @@ -98,7 +97,7 @@ public Integer finish() { try { exec = new DisruptorExecutor(hoodieWriteConfig.getDisruptorWriteBufferSize(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); + getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records assertEquals(128, result); @@ -146,7 +145,7 @@ public Integer finish() { DisruptorExecutor>, Integer> executor = new DisruptorExecutor(hoodieWriteConfig.getDisruptorWriteBufferSize(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); + getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); try { Thread.currentThread().interrupt(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index de19296202846..76c22f96e726f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -56,7 +56,7 @@ import java.util.stream.IntStream; import static org.apache.hudi.exception.ExceptionUtil.getRootCause; -import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getCloningTransformer; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -118,9 +118,9 @@ public void testRecordReading() throws Exception { @Override public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { count++; - afterRecord.add((HoodieAvroRecord) record.record); + afterRecord.add((HoodieAvroRecord) record.getResult()); try { - IndexedRecord indexedRecord = (IndexedRecord)((HoodieAvroRecord) record.record) + IndexedRecord indexedRecord = (IndexedRecord)((HoodieAvroRecord) record.getResult()) .getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get(); afterIndexedRecord.add(indexedRecord); } catch (IOException e) { @@ -138,7 +138,7 @@ public Integer finish() { try { exec = new DisruptorExecutor(hoodieWriteConfig.getDisruptorWriteBufferSize(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); + getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records assertEquals(100, result); @@ -167,7 +167,7 @@ public void testCompositeProducerRecordReading() throws Exception { final List> recs = new ArrayList<>(); final DisruptorMessageQueue queue = - new DisruptorMessageQueue(Option.of(1024), getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + new DisruptorMessageQueue(Option.of(1024), getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of("BLOCKING_WAIT"), numProducers, new Runnable() { @Override public void run() { @@ -225,7 +225,7 @@ public void run() { @Override public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { // Read recs and ensure we have covered all producer recs. - final HoodieRecord rec = payload.record; + final HoodieRecord rec = payload.getResult(); Pair producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); Integer lastSeenPos = lastSeenMap.get(producerPos.getLeft()); countMap.put(producerPos.getLeft(), countMap.get(producerPos.getLeft()) + 1); @@ -282,7 +282,7 @@ public void testException() throws Exception { final int numProducers = 40; final DisruptorMessageQueue queue = - new DisruptorMessageQueue(Option.of(1024), getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + new DisruptorMessageQueue(Option.of(1024), getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of("BLOCKING_WAIT"), numProducers, new Runnable() { @Override public void run() { @@ -314,7 +314,7 @@ public void run() { @Override public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { // Read recs and ensure we have covered all producer recs. - final HoodieRecord rec = payload.record; + final HoodieRecord rec = payload.getResult(); count++; } @@ -325,7 +325,7 @@ public Integer finish() { }; DisruptorExecutor>, Integer> exec = new DisruptorExecutor(Option.of(1024), - producers, Option.of(consumer), getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + producers, Option.of(consumer), getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); final Throwable thrown = assertThrows(HoodieException.class, exec::execute, diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java index be80e9d378331..a60ecee1a96ae 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java @@ -23,7 +23,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieHBaseIndexConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java index f22a067ad81e8..5fa5950909185 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java @@ -37,7 +37,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieHBaseIndexConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieSparkTable; 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 e02e613642c1a..2151b13284d26 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 @@ -33,7 +33,7 @@ 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.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.keygen.BaseKeyGenerator; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java index 72749160e6bd0..05a4de483c11f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java @@ -33,7 +33,7 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.testutils.HoodieClientTestHarness; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileWriterFactory.java similarity index 68% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileWriterFactory.java index 66016305d7ad3..7789254bc79eb 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileWriterFactory.java @@ -18,15 +18,14 @@ package org.apache.hudi.io.storage; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestBase; - -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.Test; import java.io.IOException; @@ -37,7 +36,7 @@ /** * Tests for {@link HoodieFileWriterFactory}. */ -public class TestHoodieFileWriterFactory extends HoodieClientTestBase { +public class TestHoodieAvroFileWriterFactory extends HoodieClientTestBase { @Test public void testGetFileWriter() throws IOException { @@ -47,27 +46,27 @@ public void testGetFileWriter() throws IOException { final HoodieWriteConfig cfg = getConfig(); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); SparkTaskContextSupplier supplier = new SparkTaskContextSupplier(); - HoodieFileWriter parquetWriter = HoodieFileWriterFactory.getFileWriter(instantTime, - parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); + HoodieFileWriter parquetWriter = HoodieFileWriterFactory.getFileWriter(instantTime, + parquetPath, table.getHadoopConf(), cfg.getStorageConfig(), HoodieTestDataGenerator.AVRO_SCHEMA, supplier, HoodieRecordType.AVRO); assertTrue(parquetWriter instanceof HoodieAvroParquetWriter); // hfile format. final Path hfilePath = new Path(basePath + "/partition/path/f1_1-0-1_000.hfile"); - HoodieFileWriter hfileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, - hfilePath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); - assertTrue(hfileWriter instanceof HoodieHFileWriter); + HoodieFileWriter hfileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, + hfilePath, table.getHadoopConf(), cfg.getStorageConfig(), HoodieTestDataGenerator.AVRO_SCHEMA, supplier, HoodieRecordType.AVRO); + assertTrue(hfileWriter instanceof HoodieAvroHFileWriter); // orc file format. final Path orcPath = new Path(basePath + "/partition/path/f1_1-0-1_000.orc"); - HoodieFileWriter orcFileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, - orcPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); - assertTrue(orcFileWriter instanceof HoodieOrcWriter); + HoodieFileWriter orcFileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, + orcPath, table.getHadoopConf(), cfg.getStorageConfig(), HoodieTestDataGenerator.AVRO_SCHEMA, supplier, HoodieRecordType.AVRO); + assertTrue(orcFileWriter instanceof HoodieAvroOrcWriter); // other file format exception. final Path logPath = new Path(basePath + "/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1"); final Throwable thrown = assertThrows(UnsupportedOperationException.class, () -> { - HoodieFileWriter logWriter = HoodieFileWriterFactory.getFileWriter(instantTime, logPath, - table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); + HoodieFileWriter logWriter = HoodieFileWriterFactory.getFileWriter(instantTime, logPath, + table.getHadoopConf(), cfg.getStorageConfig(), HoodieTestDataGenerator.AVRO_SCHEMA, supplier, HoodieRecordType.AVRO); }, "should fail since log storage writer is not supported yet."); assertTrue(thrown.getMessage().contains("format not supported yet.")); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java index dce0e2fad5910..2ff120dad32e6 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java @@ -24,10 +24,10 @@ 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.HoodieStorageConfig; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetUtils; -import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.storage.HoodieParquetConfig; import org.apache.hudi.testutils.HoodieClientTestHarness; @@ -131,6 +131,6 @@ private HoodieRowParquetWriteSupport getWriteSupport(HoodieWriteConfig.Builder w writeConfig.getBloomFilterFPP(), writeConfig.getDynamicBloomFilterMaxNumEntries(), writeConfig.getBloomFilterType()); - return new HoodieRowParquetWriteSupport(hadoopConf, SparkDatasetTestUtils.STRUCT_TYPE, Option.of(filter), writeConfig); + return new HoodieRowParquetWriteSupport(hadoopConf, SparkDatasetTestUtils.STRUCT_TYPE, Option.of(filter), writeConfig.getStorageConfig()); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index ffa93b638f5dc..91f89df795fe7 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -40,7 +40,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieLayoutConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.hadoop.HoodieParquetInputFormat; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java index 3039eb3bd9b5f..9d3fcce2d68e1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java @@ -38,7 +38,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieHBaseIndexConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java index 034f239318ee6..551533bb894cd 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java @@ -42,7 +42,7 @@ 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.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java index 59174a9371a58..c6cd554e28963 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java @@ -37,7 +37,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieMemoryConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.HoodieIndex; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java index c952c77fc4e62..cf3a635361687 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java @@ -39,7 +39,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java index ffdd9cb67b6a4..b50b0f90cca13 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java @@ -35,7 +35,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieLayoutConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java index 35d7b6329e262..d2927f2be5a94 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java @@ -46,7 +46,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.index.HoodieIndex; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java index 9d28577059404..f4ddd82a5e914 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.spark.HoodieSparkKryoProvider$; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; @@ -138,7 +139,7 @@ public synchronized void runBeforeEach() throws Exception { initialized = spark != null && hdfsTestService != null; if (!initialized) { SparkConf sparkConf = conf(); - SparkRDDWriteClient.registerClasses(sparkConf); + HoodieSparkKryoProvider$.MODULE$.register(sparkConf); SparkRDDReadClient.addHoodieSupport(sparkConf); spark = SparkSession.builder().config(sparkConf).getOrCreate(); sqlContext = spark.sqlContext(); 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 d2286decfe6b6..94e15206c39d4 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 @@ -21,6 +21,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; @@ -42,7 +43,6 @@ import org.apache.hudi.config.HoodieCleanConfig; 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.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index e2e764e15b853..f2e630eaa36d1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -67,7 +67,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.io.storage.HoodieHFileReader.SCHEMA_KEY; +import static org.apache.hudi.io.storage.HoodieAvroHFileReader.SCHEMA_KEY; /** * Utility methods to aid testing inside the HoodieClient module. diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java index 4684747161ecb..0b4fc38dfed9e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java @@ -41,7 +41,7 @@ import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieIOException; @@ -60,6 +60,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.spark.HoodieSparkKryoProvider$; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -185,7 +186,7 @@ public synchronized void runBeforeEach() { initialized = spark != null; if (!initialized) { SparkConf sparkConf = conf(); - SparkRDDWriteClient.registerClasses(sparkConf); + HoodieSparkKryoProvider$.MODULE$.register(sparkConf); SparkRDDReadClient.addHoodieSupport(sparkConf); spark = SparkSession.builder().config(sparkConf).getOrCreate(); sqlContext = spark.sqlContext(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java index 918462ac0a08f..09e6bd699bce1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java @@ -23,7 +23,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCompatibility.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCompatibility.java index 20bc3ce2fb606..de59cdb3a6e63 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCompatibility.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCompatibility.java @@ -220,7 +220,6 @@ public ReaderWriterCompatibilityChecker(boolean checkNaming) { this.checkNaming = checkNaming; } - /** * Reports the compatibility of a reader/writer schema pair. * 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 0eb0bcb4e3e65..a4157debdbf2a 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 @@ -19,11 +19,14 @@ package org.apache.hudi.avro; import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieOperation; 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.common.util.SpillableMapUtils; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -76,6 +79,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.Properties; import java.util.TimeZone; import java.util.stream.Collectors; @@ -110,6 +114,13 @@ public class HoodieAvroUtils { public static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); + /** + * TODO serialize other type of record. + */ + public static Option recordToBytes(HoodieRecord record, Schema schema) throws IOException { + return Option.of(HoodieAvroUtils.indexedRecordToBytes(record.toIndexedRecord(schema, new Properties()).get().getData())); + } + /** * Convert a given avro record to bytes. */ @@ -432,14 +443,18 @@ public static GenericRecord removeFields(GenericRecord record, Set field private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) { Schema oldSchema = oldRecord.getSchema(); - Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name()); + Field oldSchemaField = oldSchema.getField(field.name()); + Object fieldValue = oldSchemaField == null ? null : oldRecord.get(field.name()); if (fieldValue != null) { // In case field's value is a nested record, we have to rewrite it as well Object newFieldValue; if (fieldValue instanceof GenericRecord) { GenericRecord record = (GenericRecord) fieldValue; - newFieldValue = rewriteRecord(record, resolveUnionSchema(field.schema(), record.getSchema().getFullName())); + // May return null when use rewrite + String recordFullName = record.getSchema().getFullName(); + String fullName = recordFullName != null ? recordFullName : oldSchemaField.name(); + newFieldValue = rewriteRecord(record, resolveUnionSchema(field.schema(), fullName)); } else { newFieldValue = fieldValue; } @@ -599,7 +614,6 @@ public static Schema getNestedFieldSchemaFromRecord(GenericRecord record, String throw new HoodieException("Failed to get schema. Not a valid field name: " + fieldName); } - /** * Get schema for the given field and write schema. Field can be nested, denoted by dot notation. e.g: a.b.c * Use this method when record is not available. Otherwise, prefer to use {@link #getNestedFieldSchemaFromRecord(GenericRecord, String)} @@ -720,11 +734,11 @@ public static String sanitizeName(String name) { * @param schema {@link Schema} instance. * @return Column value if a single column, or concatenated String values by comma. */ - public static Object getRecordColumnValues(HoodieRecord record, + public static Object getRecordColumnValues(HoodieAvroRecord record, String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) { try { - GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); + GenericRecord genericRecord = (GenericRecord) ((HoodieAvroIndexedRecord) record.toIndexedRecord(schema, new Properties()).get()).getData(); if (columns.length == 1) { return HoodieAvroUtils.getNestedFieldVal(genericRecord, columns[0], true, consistentLogicalTimestampEnabled); } else { @@ -749,7 +763,7 @@ public static Object getRecordColumnValues(HoodieRecord record, + public static Object getRecordColumnValues(HoodieAvroRecord record, String[] columns, SerializableSchema schema, boolean consistentLogicalTimestampEnabled) { return getRecordColumnValues(record, columns, schema.get(), consistentLogicalTimestampEnabled); @@ -799,9 +813,7 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldAvr Schema oldSchema = getActualSchemaFromUnion(oldAvroSchema, oldRecord); switch (newSchema.getType()) { case RECORD: - if (!(oldRecord instanceof IndexedRecord)) { - throw new IllegalArgumentException("cannot rewrite record with different type"); - } + ValidationUtils.checkArgument(oldRecord instanceof IndexedRecord, "cannot rewrite record with different type"); IndexedRecord indexedRecord = (IndexedRecord) oldRecord; List fields = newSchema.getFields(); GenericData.Record newRecord = new GenericData.Record(newSchema); @@ -833,9 +845,7 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldAvr } return newRecord; case ARRAY: - if (!(oldRecord instanceof Collection)) { - throw new IllegalArgumentException("cannot rewrite record with different type"); - } + ValidationUtils.checkArgument(oldRecord instanceof Collection, "cannot rewrite record with different type"); Collection array = (Collection)oldRecord; List newArray = new ArrayList(); fieldNames.push("element"); @@ -845,9 +855,7 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldAvr fieldNames.pop(); return newArray; case MAP: - if (!(oldRecord instanceof Map)) { - throw new IllegalArgumentException("cannot rewrite record with different type"); - } + ValidationUtils.checkArgument(oldRecord instanceof Map, "cannot rewrite record with different type"); Map map = (Map) oldRecord; Map newMap = new HashMap<>(); fieldNames.push("value"); @@ -863,7 +871,7 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldAvr } } - private static String createFullName(Deque fieldNames) { + public static String createFullName(Deque fieldNames) { String result = ""; if (!fieldNames.isEmpty()) { List parentNames = new ArrayList<>(); @@ -1053,6 +1061,28 @@ private static Schema getActualSchemaFromUnion(Schema schema, Object data) { return actualSchema; } + public static HoodieRecord createHoodieRecordFromAvro( + IndexedRecord data, + String payloadClass, + String preCombineField, + Option> simpleKeyGenFieldsOpt, + Boolean withOperation, + Option partitionNameOp, + Boolean populateMetaFields) { + if (populateMetaFields) { + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) data, + payloadClass, preCombineField, withOperation); + // Support HoodieFileSliceReader + } else if (simpleKeyGenFieldsOpt.isPresent()) { + // TODO in HoodieFileSliceReader may partitionName=option#empty + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) data, + payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, partitionNameOp); + } else { + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) data, + payloadClass, preCombineField, withOperation, partitionNameOp); + } + } + public static GenericRecord rewriteRecordDeep(GenericRecord oldRecord, Schema newSchema) { return rewriteRecordWithNewSchema(oldRecord, newSchema, Collections.EMPTY_MAP); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java index 91f0671cf9b89..7345f94229869 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java @@ -136,6 +136,14 @@ public List getSplitStrings(ConfigProperty configProperty, String return StringUtils.split(getString(configProperty), delimiter); } + public List getSplitStringsOrDefault(ConfigProperty configProperty) { + return getSplitStringsOrDefault(configProperty, ","); + } + + public List getSplitStringsOrDefault(ConfigProperty configProperty, String delimiter) { + return StringUtils.split(getStringOrDefault(configProperty), delimiter); + } + public String getString(String key) { return props.getProperty(key); } @@ -148,7 +156,7 @@ public Integer getInt(ConfigProperty configProperty) { public Integer getIntOrDefault(ConfigProperty configProperty) { Option rawValue = getRawValue(configProperty); return rawValue.map(v -> Integer.parseInt(v.toString())) - .orElse((Integer) configProperty.defaultValue()); + .orElse(Integer.parseInt(configProperty.defaultValue().toString())); } public Boolean getBoolean(ConfigProperty configProperty) { @@ -179,16 +187,34 @@ public Long getLong(ConfigProperty configProperty) { return rawValue.map(v -> Long.parseLong(v.toString())).orElse(null); } + public Long getLongOrDefault(ConfigProperty configProperty) { + Option rawValue = getRawValue(configProperty); + return rawValue.map(v -> Long.parseLong(v.toString())) + .orElseGet(() -> Long.parseLong(configProperty.defaultValue().toString())); + } + public Float getFloat(ConfigProperty configProperty) { Option rawValue = getRawValue(configProperty); return rawValue.map(v -> Float.parseFloat(v.toString())).orElse(null); } + public Float getFloatOrDefault(ConfigProperty configProperty) { + Option rawValue = getRawValue(configProperty); + return rawValue.map(v -> Float.parseFloat(v.toString())) + .orElseGet(() -> Float.parseFloat(configProperty.defaultValue().toString())); + } + public Double getDouble(ConfigProperty configProperty) { Option rawValue = getRawValue(configProperty); return rawValue.map(v -> Double.parseDouble(v.toString())).orElse(null); } + public Double getDoubleOrDefault(ConfigProperty configProperty) { + Option rawValue = getRawValue(configProperty); + return rawValue.map(v -> Double.parseDouble(v.toString())) + .orElseGet(() -> Double.parseDouble(configProperty.defaultValue().toString())); + } + public String getStringOrDefault(ConfigProperty configProperty) { return getStringOrDefault(configProperty, configProperty.defaultValue().toString()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java similarity index 98% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java rename to hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java index 40c53fae9686b..89841ed2fafa1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java @@ -16,12 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.config; - -import org.apache.hudi.common.config.ConfigClassProperty; -import org.apache.hudi.common.config.ConfigGroups; -import org.apache.hudi.common.config.ConfigProperty; -import org.apache.hudi.common.config.HoodieConfig; +package org.apache.hudi.common.config; import javax.annotation.concurrent.Immutable; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/BaseAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/BaseAvroPayload.java index cd3a95e6bf786..aaafe61abff9a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/BaseAvroPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/BaseAvroPayload.java @@ -21,9 +21,11 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import java.io.Serializable; +import java.util.Properties; /** * Base class for all AVRO record based payloads, that can be ordered based on a field. @@ -32,12 +34,14 @@ public abstract class BaseAvroPayload implements Serializable { /** * Avro data extracted from the source converted to bytes. */ - public final byte[] recordBytes; + protected final byte[] recordBytes; /** * For purposes of preCombining. */ - public final Comparable orderingVal; + protected final Comparable orderingVal; + + protected final boolean isDeletedRecord; /** * Instantiate {@link BaseAvroPayload}. @@ -48,8 +52,46 @@ public abstract class BaseAvroPayload implements Serializable { public BaseAvroPayload(GenericRecord record, Comparable orderingVal) { this.recordBytes = record != null ? HoodieAvroUtils.avroToBytes(record) : new byte[0]; this.orderingVal = orderingVal; + this.isDeletedRecord = record == null || isDeleteRecord(record); + if (orderingVal == null) { throw new HoodieException("Ordering value is null for record: " + record); } } + + public Comparable getOrderingVal() { + return orderingVal; + } + + /** + * Defines whether this implementation of {@link HoodieRecordPayload} is deleted. + * We will not do deserialization in this method. + */ + public boolean isDeleted(Schema schema, Properties props) { + return isDeletedRecord; + } + + /** + * Defines whether this implementation of {@link HoodieRecordPayload} could produce + * {@link HoodieRecord#SENTINEL} + */ + public boolean canProduceSentinel() { + return false; + } + + /** + * @param genericRecord instance of {@link GenericRecord} of interest. + * @returns {@code true} if record represents a delete record. {@code false} otherwise. + */ + protected static boolean isDeleteRecord(GenericRecord genericRecord) { + final String isDeleteKey = HoodieRecord.HOODIE_IS_DELETED_FIELD; + // Modify to be compatible with new version Avro. + // The new version Avro throws for GenericRecord.get if the field name + // does not exist in the schema. + if (genericRecord.getSchema().getField(isDeleteKey) == null) { + return false; + } + Object deleteMarker = genericRecord.get(isDeleteKey); + return (deleteMarker instanceof Boolean && (boolean) deleteMarker); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java index 5a588eafa5f3f..a218e9dc33dd7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java @@ -51,7 +51,7 @@ public DefaultHoodieRecordPayload(Option record) { @Override public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException { - if (recordBytes.length == 0) { + if (recordBytes.length == 0 || isDeletedRecord) { return Option.empty(); } @@ -71,18 +71,18 @@ public Option combineAndGetUpdateValue(IndexedRecord currentValue /* * Now check if the incoming record is a delete record. */ - return isDeleteRecord(incomingRecord) ? Option.empty() : Option.of(incomingRecord); + return Option.of(incomingRecord); } @Override public Option getInsertValue(Schema schema, Properties properties) throws IOException { - if (recordBytes.length == 0) { + if (recordBytes.length == 0 || isDeletedRecord) { return Option.empty(); } GenericRecord incomingRecord = HoodieAvroUtils.bytesToAvro(recordBytes, schema); eventTime = updateEventTime(incomingRecord, properties); - return isDeleteRecord(incomingRecord) ? Option.empty() : Option.of(incomingRecord); + return Option.of(incomingRecord); } private static Option updateEventTime(GenericRecord record, Properties properties) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/EventTimeAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/EventTimeAvroPayload.java index 7c8efb66e5cb6..b750cffb6aee0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/EventTimeAvroPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/EventTimeAvroPayload.java @@ -46,7 +46,7 @@ public EventTimeAvroPayload(Option record) { @Override public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException { - if (recordBytes.length == 0) { + if (recordBytes.length == 0 || isDeletedRecord) { return Option.empty(); } @@ -61,17 +61,16 @@ public Option combineAndGetUpdateValue(IndexedRecord currentValue /* * Now check if the incoming record is a delete record. */ - return isDeleteRecord(incomingRecord) ? Option.empty() : Option.of(incomingRecord); + return Option.of(incomingRecord); } @Override public Option getInsertValue(Schema schema, Properties properties) throws IOException { - if (recordBytes.length == 0) { + if (recordBytes.length == 0 || isDeletedRecord) { return Option.empty(); } - GenericRecord incomingRecord = bytesToAvro(recordBytes, schema); - return isDeleteRecord(incomingRecord) ? Option.empty() : Option.of(incomingRecord); + return Option.of(bytesToAvro(recordBytes, schema)); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java new file mode 100644 index 0000000000000..7d497408e6467 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.util.ConfigUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + +import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS; + +/** + * This only use by reader returning. + */ +public class HoodieAvroIndexedRecord extends HoodieRecord { + + public HoodieAvroIndexedRecord(IndexedRecord data) { + super(null, data); + } + + public HoodieAvroIndexedRecord(HoodieKey key, IndexedRecord data) { + super(key, data); + } + + public HoodieAvroIndexedRecord( + HoodieKey key, + IndexedRecord data, + HoodieOperation operation, + Option> metaData) { + super(key, data, operation, metaData); + } + + public HoodieAvroIndexedRecord(HoodieRecord record) { + super(record); + } + + public HoodieAvroIndexedRecord() { + } + + @Override + public HoodieRecord newInstance() { + return new HoodieAvroIndexedRecord(this); + } + + @Override + public HoodieRecord newInstance(HoodieKey key, HoodieOperation op) { + return new HoodieAvroIndexedRecord(key, data, op, metaData); + } + + @Override + public HoodieRecord newInstance(HoodieKey key) { + return new HoodieAvroIndexedRecord(key, data, operation, metaData); + } + + @Override + public String getRecordKey(Schema recordSchema, Option keyGeneratorOpt) { + return keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey((GenericRecord) data) : ((GenericRecord) data).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + } + + @Override + public HoodieRecordType getRecordType() { + return HoodieRecordType.AVRO; + } + + @Override + public String getRecordKey(Schema recordSchema, String keyFieldName) { + return Option.ofNullable(data.getSchema().getField(keyFieldName)) + .map(keyField -> data.get(keyField.pos())) + .map(Object::toString).orElse(null); + } + + @Override + public Object[] getColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { + GenericRecord record = HoodieAvroUtils.stitchRecords((GenericRecord) data, (GenericRecord) other.getData(), targetSchema); + return new HoodieAvroIndexedRecord(key, record, operation, metaData); + } + + @Override + public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException { + GenericRecord record = HoodieAvroUtils.rewriteRecord((GenericRecord) data, targetSchema); + return new HoodieAvroIndexedRecord(key, record, operation, metaData); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException { + GenericRecord record = HoodieAvroUtils.rewriteRecordWithNewSchema(data, newSchema, renameCols); + return new HoodieAvroIndexedRecord(key, record, operation, metaData); + } + + @Override + public HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException { + metadataValues.getKv().forEach((key, value) -> { + if (value != null) { + ((GenericRecord) data).put(key, value); + } + }); + + return new HoodieAvroIndexedRecord(key, data, operation, metaData); + } + + @Override + public HoodieRecord truncateRecordKey(Schema recordSchema, Properties props, String keyFieldName) { + ((GenericRecord) data).put(keyFieldName, StringUtils.EMPTY_STRING); + return this; + } + + @Override + public boolean isDelete(Schema recordSchema, Properties props) { + return false; + } + + @Override + public boolean shouldIgnore(Schema recordSchema, Properties props) throws IOException { + return getData().equals(SENTINEL); + } + + @Override + public HoodieRecord copy() { + return this; + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithParams( + Schema recordSchema, + Properties props, + Option> simpleKeyGenFieldsOpt, + Boolean withOperation, + Option partitionNameOp, + Boolean populateMetaFields) { + String payloadClass = ConfigUtils.getPayloadClass(props); + String preCombineField = ConfigUtils.getOrderingField(props); + return HoodieAvroUtils.createHoodieRecordFromAvro(data, payloadClass, preCombineField, simpleKeyGenFieldsOpt, withOperation, partitionNameOp, populateMetaFields); + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Schema recordSchema, + Properties props, Option keyGen) { + GenericRecord record = (GenericRecord) data; + String key; + String partition; + if (keyGen.isPresent() && !Boolean.parseBoolean(props.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) { + BaseKeyGenerator keyGeneratorOpt = keyGen.get(); + key = keyGeneratorOpt.getRecordKey(record); + partition = keyGeneratorOpt.getPartitionPath(record); + } else { + key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + } + HoodieKey hoodieKey = new HoodieKey(key, partition); + + HoodieRecordPayload avroPayload = new RewriteAvroPayload(record); + HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, avroPayload); + return hoodieRecord; + } + + @Override + public Option> getMetadata() { + return Option.empty(); + } + + @Override + public Comparable getOrderingValue(Schema recordSchema, Properties props) { + boolean consistentLogicalTimestampEnabled = Boolean.parseBoolean(props.getProperty( + KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), + KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())); + return (Comparable) HoodieAvroUtils.getNestedFieldVal((GenericRecord) data, + ConfigUtils.getOrderingField(props), + true, consistentLogicalTimestampEnabled); + } + + @Override + public Option toIndexedRecord(Schema recordSchema, Properties props) { + return Option.of(this); + } + + /** + * NOTE: This method is declared final to make sure there's no polymorphism and therefore + * JIT compiler could perform more aggressive optimizations + */ + @SuppressWarnings("unchecked") + @Override + protected final void writeRecordPayload(IndexedRecord payload, Kryo kryo, Output output) { + // NOTE: We're leveraging Spark's default [[GenericAvroSerializer]] to serialize Avro + Serializer avroSerializer = kryo.getSerializer(GenericRecord.class); + + kryo.writeObjectOrNull(output, payload, avroSerializer); + } + + /** + * NOTE: This method is declared final to make sure there's no polymorphism and therefore + * JIT compiler could perform more aggressive optimizations + */ + @SuppressWarnings("unchecked") + @Override + protected final IndexedRecord readRecordPayload(Kryo kryo, Input input) { + // NOTE: We're leveraging Spark's default [[GenericAvroSerializer]] to serialize Avro + Serializer avroSerializer = kryo.getSerializer(GenericRecord.class); + + return kryo.readObjectOrNull(input, GenericRecord.class, avroSerializer); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java index 210c42def8d81..62001356b4fad 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java @@ -19,24 +19,52 @@ package org.apache.hudi.common.model; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.util.ConfigUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.BaseKeyGenerator; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + /** * Implementation of {@link HoodieRecord} using Avro payload. * * @param payload implementation. */ public class HoodieAvroRecord extends HoodieRecord { + public HoodieAvroRecord(HoodieKey key, T data) { super(key, data); } public HoodieAvroRecord(HoodieKey key, T data, HoodieOperation operation) { - super(key, data, operation); + super(key, data, operation, Option.empty()); } public HoodieAvroRecord(HoodieRecord record) { super(record); } + public HoodieAvroRecord( + HoodieKey key, + T data, + HoodieOperation operation, + HoodieRecordLocation currentLocation, + HoodieRecordLocation newLocation) { + super(key, data, operation, currentLocation, newLocation); + } + public HoodieAvroRecord() { } @@ -45,6 +73,16 @@ public HoodieRecord newInstance() { return new HoodieAvroRecord<>(this); } + @Override + public HoodieRecord newInstance(HoodieKey key, HoodieOperation op) { + return new HoodieAvroRecord<>(key, data, op); + } + + @Override + public HoodieRecord newInstance(HoodieKey key) { + return new HoodieAvroRecord<>(key, data); + } + @Override public T getData() { if (data == null) { @@ -52,4 +90,145 @@ public T getData() { } return data; } + + @Override + public Comparable getOrderingValue(Schema recordSchema, Properties props) { + return this.getData().getOrderingValue(); + } + + @Override + public String getRecordKey(Schema recordSchema, + Option keyGeneratorOpt) { + return getRecordKey(); + } + + @Override + public String getRecordKey(Schema recordSchema, String keyFieldName) { + return getRecordKey(); + } + + @Override + public HoodieRecordType getRecordType() { + return HoodieRecordType.AVRO; + } + + @Override + public Object[] getColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) { + return new Object[]{HoodieAvroUtils.getRecordColumnValues(this, columns, recordSchema, consistentLogicalTimestampEnabled)}; + } + + @Override + public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException { + Option avroRecordPayloadOpt = getData().getInsertValue(recordSchema, props); + GenericRecord avroPayloadInNewSchema = + HoodieAvroUtils.rewriteRecord((GenericRecord) avroRecordPayloadOpt.get(), targetSchema); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(avroPayloadInNewSchema), getOperation(), this.currentLocation, this.newLocation); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException { + GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, props).get(); + GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation(), this.currentLocation, this.newLocation); + } + + @Override + public HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException { + GenericRecord avroRecordPayload = (GenericRecord) getData().getInsertValue(recordSchema, props).get(); + + metadataValues.getKv().forEach((key, value) -> { + if (value != null) { + avroRecordPayload.put(key, value); + } + }); + + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(avroRecordPayload), getOperation(), this.currentLocation, this.newLocation); + } + + @Override + public HoodieRecord truncateRecordKey(Schema recordSchema, Properties props, String keyFieldName) throws IOException { + GenericRecord avroRecordPayload = (GenericRecord) getData().getInsertValue(recordSchema, props).get(); + avroRecordPayload.put(keyFieldName, StringUtils.EMPTY_STRING); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(avroRecordPayload), getOperation(), this.currentLocation, this.newLocation); + } + + @Override + public boolean isDelete(Schema recordSchema, Properties props) throws IOException { + if (this.data instanceof BaseAvroPayload) { + return ((BaseAvroPayload) this.data).isDeleted(recordSchema, props); + } else { + return !this.data.getInsertValue(recordSchema, props).isPresent(); + } + } + + @Override + public boolean shouldIgnore(Schema recordSchema, Properties props) throws IOException { + HoodieRecordPayload recordPayload = getData(); + // NOTE: Currently only records borne by [[ExpressionPayload]] can currently be ignored, + // as such, we limit exposure of this method only to such payloads + if (recordPayload instanceof BaseAvroPayload && ((BaseAvroPayload) recordPayload).canProduceSentinel()) { + Option insertRecord = recordPayload.getInsertValue(recordSchema, props); + return insertRecord.isPresent() && insertRecord.get().equals(SENTINEL); + } + + return false; + } + + @Override + public HoodieRecord copy() { + return this; + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithParams( + Schema recordSchema, Properties props, + Option> simpleKeyGenFieldsOpt, + Boolean withOperation, + Option partitionNameOp, + Boolean populateMetaFields) throws IOException { + IndexedRecord indexedRecord = (IndexedRecord) data.getInsertValue(recordSchema, props).get(); + String payloadClass = ConfigUtils.getPayloadClass(props); + String preCombineField = ConfigUtils.getOrderingField(props); + return HoodieAvroUtils.createHoodieRecordFromAvro(indexedRecord, payloadClass, preCombineField, simpleKeyGenFieldsOpt, withOperation, partitionNameOp, populateMetaFields); + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Schema recordSchema, + Properties props, Option keyGen) { + throw new UnsupportedOperationException(); + } + + public Option> getMetadata() { + return getData().getMetadata(); + } + + @Override + public Option toIndexedRecord(Schema recordSchema, Properties props) throws IOException { + Option avroData = getData().getInsertValue(recordSchema, props); + if (avroData.isPresent()) { + HoodieAvroIndexedRecord record = + new HoodieAvroIndexedRecord(key, avroData.get(), operation, getData().getMetadata()); + return Option.of(record); + } else { + return Option.empty(); + } + } + + @Override + protected final void writeRecordPayload(T payload, Kryo kryo, Output output) { + // NOTE: Since [[orderingVal]] is polymorphic we have to write out its class + // to be able to properly deserialize it + kryo.writeClassAndObject(output, payload); + } + + @SuppressWarnings("unchecked") + @Override + protected final T readRecordPayload(Kryo kryo, Input input) { + return (T) kryo.readClassAndObject(input); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java new file mode 100644 index 0000000000000..b9e29787f8a98 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.metadata.HoodieMetadataPayload; + +import java.io.IOException; +import java.util.Properties; + +import static org.apache.hudi.common.util.TypeUtils.unsafeCast; + +public class HoodieAvroRecordMerger implements HoodieRecordMerger { + + @Override + public String getMergingStrategy() { + return HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; + } + + @Override + public Option> merge(HoodieRecord older, Schema oldSchema, HoodieRecord newer, Schema newSchema, TypedProperties props) throws IOException { + ValidationUtils.checkArgument(older.getRecordType() == HoodieRecordType.AVRO); + ValidationUtils.checkArgument(newer.getRecordType() == HoodieRecordType.AVRO); + Config.LegacyOperationMode legacyOperatingMode = Config.LegacyOperationMode.valueOf( + props.getString(Config.LEGACY_OPERATING_MODE.key(), Config.LEGACY_OPERATING_MODE.defaultValue())); + + switch (legacyOperatingMode) { + case PRE_COMBINING: + HoodieRecord res = preCombine(older, newer); + if (res == older) { + return Option.of(Pair.of(res, oldSchema)); + } else { + return Option.of(Pair.of(res, newSchema)); + } + + case COMBINING: + return combineAndGetUpdateValue(older, newer, newSchema, props) + .map(r -> Pair.of(r, (((HoodieAvroIndexedRecord) r).getData()).getSchema())); + + default: + throw new UnsupportedOperationException(String.format("Unsupported legacy operating mode (%s)", legacyOperatingMode)); + } + } + + @Override + public HoodieRecordType getRecordType() { + return HoodieRecordType.AVRO; + } + + private HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer) { + HoodieRecordPayload picked = unsafeCast(((HoodieAvroRecord) newer).getData().preCombine(((HoodieAvroRecord) older).getData())); + if (picked instanceof HoodieMetadataPayload) { + // NOTE: HoodieMetadataPayload return a new payload + return new HoodieAvroRecord(newer.getKey(), picked, newer.getOperation()); + } + return picked.equals(((HoodieAvroRecord) newer).getData()) ? newer : older; + } + + private Option combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException { + Option previousAvroData = older.toIndexedRecord(schema, props).map(HoodieAvroIndexedRecord::getData); + if (!previousAvroData.isPresent()) { + return Option.empty(); + } + + return ((HoodieAvroRecord) newer).getData().combineAndGetUpdateValue(previousAvroData.get(), schema, props) + .map(combinedAvroPayload -> new HoodieAvroIndexedRecord((IndexedRecord) combinedAvroPayload)); + } + + public static class Config { + + public enum LegacyOperationMode { + PRE_COMBINING, + COMBINING + } + + public static ConfigProperty LEGACY_OPERATING_MODE = + ConfigProperty.key("hoodie.datasource.write.merger.legacy.operation") + .defaultValue(LegacyOperationMode.COMBINING.name()) + .withDocumentation("Controls the mode of the merging operation performed by `HoodieAvroRecordMerger`. " + + "This is required to maintain backward-compatibility w/ the existing semantic of `HoodieRecordPayload` " + + "implementations providing `preCombine` and `combineAndGetUpdateValue` methods."); + + public static TypedProperties withLegacyOperatingModePreCombining(Properties props) { + TypedProperties newProps = new TypedProperties(); + newProps.putAll(props); + newProps.setProperty(Config.LEGACY_OPERATING_MODE.key(), Config.LegacyOperationMode.PRE_COMBINING.name()); + return newProps; + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java new file mode 100644 index 0000000000000..e2f9334f8a2ec --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.avro.Schema; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.BaseKeyGenerator; + +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + +public class HoodieEmptyRecord extends HoodieRecord { + + private HoodieRecordType type; + private Comparable orderingVal; + + public HoodieEmptyRecord(HoodieKey key, HoodieRecordType type) { + super(key, null); + this.type = type; + this.orderingVal = null; + } + + public HoodieEmptyRecord(HoodieKey key, HoodieOperation operation, Comparable orderingVal, HoodieRecordType type) { + super(key, null, operation, Option.empty()); + this.type = type; + this.orderingVal = orderingVal; + } + + @Override + public T getData() { + return null; + } + + @Override + public Comparable getOrderingValue(Schema recordSchema, Properties props) { + return orderingVal; + } + + @Override + public HoodieRecord newInstance() { + return this; + } + + @Override + public HoodieRecord newInstance(HoodieKey key, HoodieOperation op) { + return new HoodieEmptyRecord<>(key, op, orderingVal, type); + } + + @Override + public HoodieRecord newInstance(HoodieKey key) { + return new HoodieEmptyRecord<>(key, type); + } + + @Override + public HoodieRecordType getRecordType() { + return type; + } + + @Override + public String getRecordKey(Schema recordSchema, + Option keyGeneratorOpt) { + return key.getRecordKey(); + } + + @Override + public String getRecordKey(Schema recordSchema, String keyFieldName) { + return key.getRecordKey(); + } + + @Override + public Object[] getColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord truncateRecordKey(Schema recordSchema, Properties props, String keyFieldName) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isDelete(Schema recordSchema, Properties props) throws IOException { + return true; + } + + @Override + public boolean shouldIgnore(Schema recordSchema, Properties props) throws IOException { + return false; + } + + @Override + public HoodieRecord copy() { + return this; + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithParams(Schema recordSchema, Properties props, Option> simpleKeyGenFieldsOpt, + Boolean withOperation, Option partitionNameOp, Boolean populateMetaFieldsOp) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Schema recordSchema, + Properties props, Option keyGen) { + throw new UnsupportedOperationException(); + } + + @Override + public Option toIndexedRecord(Schema recordSchema, Properties props) throws IOException { + return Option.empty(); + } + + @Override + public Option> getMetadata() { + return Option.empty(); + } + + /** + * NOTE: This method is declared final to make sure there's no polymorphism and therefore + * JIT compiler could perform more aggressive optimizations + */ + @Override + protected final void writeRecordPayload(T payload, Kryo kryo, Output output) { + kryo.writeObject(output, type); + // NOTE: Since [[orderingVal]] is polymorphic we have to write out its class + // to be able to properly deserialize it + kryo.writeClassAndObject(output, orderingVal); + } + + /** + * NOTE: This method is declared final to make sure there's no polymorphism and therefore + * JIT compiler could perform more aggressive optimizations + */ + @Override + protected final T readRecordPayload(Kryo kryo, Input input) { + this.type = kryo.readObject(input, HoodieRecordType.class); + this.orderingVal = (Comparable) kryo.readClassAndObject(input); + // NOTE: [[EmptyRecord]]'s payload is always null + return null; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java index 9030204099ae8..5208cece1cb56 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java @@ -18,6 +18,11 @@ package org.apache.hudi.common.model; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoSerializable; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; + import java.io.Serializable; import java.util.Objects; @@ -27,13 +32,13 @@ * - recordKey : a recordKey that acts as primary key for a record. * - partitionPath : the partition path of a record. */ -public class HoodieKey implements Serializable { +public final class HoodieKey implements Serializable, KryoSerializable { private String recordKey; private String partitionPath; - public HoodieKey() { - } + // Required for serializer + public HoodieKey() {} public HoodieKey(String recordKey, String partitionPath) { this.recordKey = recordKey; @@ -81,4 +86,16 @@ public String toString() { sb.append('}'); return sb.toString(); } + + @Override + public void write(Kryo kryo, Output output) { + output.writeString(recordKey); + output.writeString(partitionPath); + } + + @Override + public void read(Kryo kryo, Input input) { + this.recordKey = input.readString(); + this.partitionPath = input.readString(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java index 2a3edafb8f27d..a23316a2ff9fd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -18,14 +18,24 @@ package org.apache.hudi.common.model; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoSerializable; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.BaseKeyGenerator; +import java.io.IOException; import java.io.Serializable; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Properties; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -33,15 +43,52 @@ /** * A Single Record managed by Hoodie. */ -public abstract class HoodieRecord implements Serializable { +public abstract class HoodieRecord implements HoodieRecordCompatibilityInterface, KryoSerializable, Serializable { + + public static final String COMMIT_TIME_METADATA_FIELD = HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.getFieldName(); + public static final String COMMIT_SEQNO_METADATA_FIELD = HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.getFieldName(); + public static final String RECORD_KEY_METADATA_FIELD = HoodieMetadataField.RECORD_KEY_METADATA_FIELD.getFieldName(); + public static final String PARTITION_PATH_METADATA_FIELD = HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.getFieldName(); + public static final String FILENAME_METADATA_FIELD = HoodieMetadataField.FILENAME_METADATA_FIELD.getFieldName(); + public static final String OPERATION_METADATA_FIELD = HoodieMetadataField.OPERATION_METADATA_FIELD.getFieldName(); + public static final String HOODIE_IS_DELETED_FIELD = "_hoodie_is_deleted"; + + public enum HoodieMetadataField { + COMMIT_TIME_METADATA_FIELD("_hoodie_commit_time"), + COMMIT_SEQNO_METADATA_FIELD("_hoodie_commit_seqno"), + RECORD_KEY_METADATA_FIELD("_hoodie_record_key"), + PARTITION_PATH_METADATA_FIELD("_hoodie_partition_path"), + FILENAME_METADATA_FIELD("_hoodie_file_name"), + OPERATION_METADATA_FIELD("_hoodie_operation"); + + private final String fieldName; + + HoodieMetadataField(String fieldName) { + this.fieldName = fieldName; + } + + public String getFieldName() { + return fieldName; + } + } - public static final String COMMIT_TIME_METADATA_FIELD = "_hoodie_commit_time"; - public static final String COMMIT_SEQNO_METADATA_FIELD = "_hoodie_commit_seqno"; - public static final String RECORD_KEY_METADATA_FIELD = "_hoodie_record_key"; - public static final String PARTITION_PATH_METADATA_FIELD = "_hoodie_partition_path"; - public static final String FILENAME_METADATA_FIELD = "_hoodie_file_name"; - public static final String OPERATION_METADATA_FIELD = "_hoodie_operation"; - public static final String HOODIE_IS_DELETED = "_hoodie_is_deleted"; + /** + * A special record returned by {@link HoodieRecordPayload}, which means we should just skip this record. + * This record is only used for {@link HoodieRecordPayload} currently, so it should not + * shuffle though network, we can compare the record locally by the equal method. + * The HoodieRecordPayload#combineAndGetUpdateValue and HoodieRecordPayload#getInsertValue + * have 3 kind of return: + * 1、Option.empty + * This means we should delete this record. + * 2、IGNORE_RECORD + * This means we should not process this record,just skip. + * 3、Other non-empty record + * This means we should process this record. + * + * We can see the usage of IGNORE_RECORD in + * org.apache.spark.sql.hudi.command.payload.ExpressionPayload + */ + public static final EmptyRecord SENTINEL = new EmptyRecord(); public static final List HOODIE_META_COLUMNS = CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD, @@ -67,7 +114,7 @@ public abstract class HoodieRecord implements Serializable { /** * Identifies the record across the table. */ - private HoodieKey key; + protected HoodieKey key; /** * Actual payload of the record. @@ -77,12 +124,12 @@ public abstract class HoodieRecord implements Serializable { /** * Current location of record on storage. Filled in by looking up index */ - private HoodieRecordLocation currentLocation; + protected HoodieRecordLocation currentLocation; /** * New location of record on storage, after written. */ - private HoodieRecordLocation newLocation; + protected HoodieRecordLocation newLocation; /** * Indicates whether the object is sealed. @@ -92,34 +139,55 @@ public abstract class HoodieRecord implements Serializable { /** * The cdc operation. */ - private HoodieOperation operation; + protected HoodieOperation operation; + + /** + * The metaData of the record. + */ + protected Option> metaData; public HoodieRecord(HoodieKey key, T data) { - this(key, data, null); + this(key, data, null, Option.empty()); } - public HoodieRecord(HoodieKey key, T data, HoodieOperation operation) { + public HoodieRecord(HoodieKey key, T data, HoodieOperation operation, Option> metaData) { this.key = key; this.data = data; this.currentLocation = null; this.newLocation = null; this.sealed = false; this.operation = operation; + this.metaData = metaData; + } + + public HoodieRecord( + HoodieKey key, + T data, + HoodieOperation operation, + HoodieRecordLocation currentLocation, + HoodieRecordLocation newLocation) { + this.key = key; + this.data = data; + this.currentLocation = currentLocation; + this.newLocation = newLocation; + this.operation = operation; } public HoodieRecord(HoodieRecord record) { - this(record.key, record.data); + this(record.key, record.data, record.operation, record.metaData); this.currentLocation = record.currentLocation; this.newLocation = record.newLocation; this.sealed = record.sealed; - this.operation = record.operation; } - public HoodieRecord() { - } + public HoodieRecord() {} public abstract HoodieRecord newInstance(); + public abstract HoodieRecord newInstance(HoodieKey key, HoodieOperation op); + + public abstract HoodieRecord newInstance(HoodieKey key); + public HoodieKey getKey() { return key; } @@ -128,6 +196,8 @@ public HoodieOperation getOperation() { return operation; } + public abstract Comparable getOrderingValue(Schema recordSchema, Properties props); + public T getData() { if (data == null) { throw new IllegalStateException("Payload already deflated for record."); @@ -203,10 +273,6 @@ public String toString() { return sb.toString(); } - public static String generateSequenceId(String instantTime, int partitionId, long recordIndex) { - return instantTime + "_" + partitionId + "_" + recordIndex; - } - public String getPartitionPath() { assert key != null; return key.getPartitionPath(); @@ -217,6 +283,12 @@ public String getRecordKey() { return key.getRecordKey(); } + public abstract HoodieRecordType getRecordType(); + + public abstract String getRecordKey(Schema recordSchema, Option keyGeneratorOpt); + + public abstract String getRecordKey(Schema recordSchema, String keyFieldName); + public void seal() { this.sealed = true; } @@ -230,4 +302,137 @@ public void checkState() { throw new UnsupportedOperationException("Not allowed to modify after sealed"); } } + + protected abstract void writeRecordPayload(T payload, Kryo kryo, Output output); + + protected abstract T readRecordPayload(Kryo kryo, Input input); + + /** + * NOTE: This method is declared final to make sure there's no polymorphism and therefore + * JIT compiler could perform more aggressive optimizations + */ + @Override + public final void write(Kryo kryo, Output output) { + kryo.writeObjectOrNull(output, key, HoodieKey.class); + kryo.writeObjectOrNull(output, operation, HoodieOperation.class); + // NOTE: We have to write actual class along with the object here, + // since [[HoodieRecordLocation]] has inheritors + kryo.writeClassAndObject(output, currentLocation); + kryo.writeClassAndObject(output, newLocation); + // NOTE: Writing out actual record payload is relegated to the actual + // implementation + writeRecordPayload(data, kryo, output); + } + + /** + * NOTE: This method is declared final to make sure there's no polymorphism and therefore + * JIT compiler could perform more aggressive optimizations + */ + @Override + public final void read(Kryo kryo, Input input) { + this.key = kryo.readObjectOrNull(input, HoodieKey.class); + this.operation = kryo.readObjectOrNull(input, HoodieOperation.class); + this.currentLocation = (HoodieRecordLocation) kryo.readClassAndObject(input); + this.newLocation = (HoodieRecordLocation) kryo.readClassAndObject(input); + // NOTE: Reading out actual record payload is relegated to the actual + // implementation + this.data = readRecordPayload(kryo, input); + + // NOTE: We're always seal object after deserialization + this.sealed = true; + } + + /** + * Get column in record to support RDDCustomColumnsSortPartitioner + * @return + */ + public abstract Object[] getColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled); + + /** + * Support bootstrap. + */ + public abstract HoodieRecord joinWith(HoodieRecord other, Schema targetSchema); + + /** + * Rewrite record into new schema(add meta columns) + */ + public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException; + + /** + * Support schema evolution. + */ + public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException; + + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema) throws IOException { + return rewriteRecordWithNewSchema(recordSchema, props, newSchema, Collections.emptyMap()); + } + + /** + * This method could change in the future. + * @temporary + */ + public abstract HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException; + + public abstract boolean isDelete(Schema recordSchema, Properties props) throws IOException; + + /** + * Is EmptyRecord. Generated by ExpressionPayload. + */ + public abstract boolean shouldIgnore(Schema recordSchema, Properties props) throws IOException; + + /** + * This is used to copy data. + */ + public abstract HoodieRecord copy(); + + public abstract Option> getMetadata(); + + public static String generateSequenceId(String instantTime, int partitionId, long recordIndex) { + return instantTime + "_" + partitionId + "_" + recordIndex; + } + + /** + * A special record returned by {@link HoodieRecordPayload}, which means we should just skip this record. + * This record is only used for {@link HoodieRecordPayload} currently, so it should not + * shuffle though network, we can compare the record locally by the equal method. + * The HoodieRecordPayload#combineAndGetUpdateValue and HoodieRecordPayload#getInsertValue + * have 3 kind of return: + * 1、Option.empty + * This means we should delete this record. + * 2、IGNORE_RECORD + * This means we should not process this record,just skip. + * 3、Other non-empty record + * This means we should process this record. + * + * We can see the usage of IGNORE_RECORD in + * org.apache.spark.sql.hudi.command.payload.ExpressionPayload + */ + private static class EmptyRecord implements GenericRecord { + private EmptyRecord() {} + + @Override + public void put(int i, Object v) {} + + @Override + public Object get(int i) { + return null; + } + + @Override + public Schema getSchema() { + return null; + } + + @Override + public void put(String key, Object v) {} + + @Override + public Object get(String key) { + return null; + } + } + + public enum HoodieRecordType { + AVRO, SPARK + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordCompatibilityInterface.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordCompatibilityInterface.java new file mode 100644 index 0000000000000..540244009d292 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordCompatibilityInterface.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import java.io.IOException; +import java.util.Properties; +import org.apache.avro.Schema; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.BaseKeyGenerator; + +public interface HoodieRecordCompatibilityInterface { + + /** + * This method used to extract HoodieKey not through keyGenerator. + */ + HoodieRecord wrapIntoHoodieRecordPayloadWithParams( + Schema recordSchema, + Properties props, + Option> simpleKeyGenFieldsOpt, + Boolean withOperation, + Option partitionNameOp, + Boolean populateMetaFieldsOp) throws IOException; + + /** + * This method used to extract HoodieKey through keyGenerator. This method used in ClusteringExecutionStrategy. + */ + HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Schema recordSchema, Properties props, Option keyGen); + + /** + * This method used to overwrite record key field. + */ + HoodieRecord truncateRecordKey(Schema recordSchema, Properties props, String keyFieldName) throws IOException; + + Option toIndexedRecord(Schema recordSchema, Properties props) throws IOException; +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordGlobalLocation.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordGlobalLocation.java index f469a1ab451c2..8c021d902a338 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordGlobalLocation.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordGlobalLocation.java @@ -18,18 +18,21 @@ package org.apache.hudi.common.model; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; + import java.util.Objects; /** * Similar with {@link org.apache.hudi.common.model.HoodieRecordLocation} but with partition path. */ -public class HoodieRecordGlobalLocation extends HoodieRecordLocation { +public final class HoodieRecordGlobalLocation extends HoodieRecordLocation { private static final long serialVersionUID = 1L; private String partitionPath; - public HoodieRecordGlobalLocation() { - } + public HoodieRecordGlobalLocation() {} public HoodieRecordGlobalLocation(String partitionPath, String instantTime, String fileId) { super(instantTime, fileId); @@ -93,5 +96,19 @@ public HoodieRecordLocation toLocal(String instantTime) { public HoodieRecordGlobalLocation copy(String partitionPath) { return new HoodieRecordGlobalLocation(partitionPath, instantTime, fileId); } + + @Override + public final void write(Kryo kryo, Output output) { + super.write(kryo, output); + + kryo.writeObjectOrNull(output, partitionPath, String.class); + } + + @Override + public void read(Kryo kryo, Input input) { + super.read(kryo, input); + + this.partitionPath = kryo.readObject(input, String.class); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordLocation.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordLocation.java index 2b1feab39b9cb..8b1dd2b378a1e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordLocation.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordLocation.java @@ -18,13 +18,18 @@ package org.apache.hudi.common.model; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoSerializable; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; + import java.io.Serializable; import java.util.Objects; /** * Location of a HoodieRecord within the partition it belongs to. Ultimately, this points to an actual file on disk */ -public class HoodieRecordLocation implements Serializable { +public class HoodieRecordLocation implements Serializable, KryoSerializable { protected String instantTime; protected String fileId; @@ -78,4 +83,16 @@ public String getFileId() { public void setFileId(String fileId) { this.fileId = fileId; } + + @Override + public void write(Kryo kryo, Output output) { + output.writeString(instantTime); + output.writeString(fileId); + } + + @Override + public void read(Kryo kryo, Input input) { + this.instantTime = input.readString(); + this.fileId = input.readString(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java new file mode 100644 index 0000000000000..da413592abc6e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.avro.Schema; +import org.apache.hudi.ApiMaturityLevel; +import org.apache.hudi.PublicAPIClass; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; + +import java.io.IOException; +import java.io.Serializable; + +/** + * HoodieMerge defines how to merge two records. It is a stateless component. + * It can implement the merging logic of HoodieRecord of different engines + * and avoid the performance consumption caused by the serialization/deserialization of Avro payload. + */ +@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) +public interface HoodieRecordMerger extends Serializable { + + String DEFAULT_MERGER_STRATEGY_UUID = "eeb8d96f-b1e4-49fd-bbf8-28ac514178e5"; + + /** + * This method converges combineAndGetUpdateValue and precombine from HoodiePayload. + * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C + * of the single record, both orders of operations applications have to yield the same result) + */ + Option> merge(HoodieRecord older, Schema oldSchema, HoodieRecord newer, Schema newSchema, TypedProperties props) throws IOException; + + /** + * The record type handled by the current merger. + * SPARK, AVRO, FLINK + */ + HoodieRecordType getRecordType(); + + /** + * The kind of merging strategy this recordMerger belongs to. An UUID represents merging strategy. + */ + String getMergingStrategy(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java index d4e61da9bbf63..5885c9702ab45 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java @@ -18,14 +18,13 @@ package org.apache.hudi.common.model; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.PublicAPIClass; import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.common.util.Option; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; - import java.io.IOException; import java.io.Serializable; import java.util.Map; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/MetadataValues.java b/hudi-common/src/main/java/org/apache/hudi/common/model/MetadataValues.java new file mode 100644 index 0000000000000..baee08163380b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/MetadataValues.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import java.util.HashMap; +import java.util.Map; + +public class MetadataValues { + private final Map kv; + + public MetadataValues() { + this.kv = new HashMap<>(); + } + + public MetadataValues setCommitTime(String value) { + this.kv.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, value); + return this; + } + + public MetadataValues setCommitSeqno(String value) { + this.kv.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, value); + return this; + } + + public MetadataValues setRecordKey(String value) { + this.kv.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, value); + return this; + } + + public MetadataValues setPartitionPath(String value) { + this.kv.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, value); + return this; + } + + public MetadataValues setFileName(String value) { + this.kv.put(HoodieRecord.FILENAME_METADATA_FIELD, value); + return this; + } + + public MetadataValues setOperation(String value) { + this.kv.put(HoodieRecord.OPERATION_METADATA_FIELD, value); + return this; + } + + public Map getKv() { + return kv; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java index 6bd945f389d09..70cc9360a3374 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java @@ -69,31 +69,11 @@ public Option combineAndGetUpdateValue(IndexedRecord currentValue @Override public Option getInsertValue(Schema schema) throws IOException { - if (recordBytes.length == 0) { + if (recordBytes.length == 0 || isDeletedRecord) { return Option.empty(); } - IndexedRecord indexedRecord = HoodieAvroUtils.bytesToAvro(recordBytes, schema); - if (isDeleteRecord((GenericRecord) indexedRecord)) { - return Option.empty(); - } else { - return Option.of(indexedRecord); - } - } - /** - * @param genericRecord instance of {@link GenericRecord} of interest. - * @returns {@code true} if record represents a delete record. {@code false} otherwise. - */ - protected boolean isDeleteRecord(GenericRecord genericRecord) { - final String isDeleteKey = HoodieRecord.HOODIE_IS_DELETED; - // Modify to be compatible with new version Avro. - // The new version Avro throws for GenericRecord.get if the field name - // does not exist in the schema. - if (genericRecord.getSchema().getField(isDeleteKey) == null) { - return false; - } - Object deleteMarker = genericRecord.get(isDeleteKey); - return (deleteMarker instanceof Boolean && (boolean) deleteMarker); + return Option.of((IndexedRecord) HoodieAvroUtils.bytesToAvro(recordBytes, schema)); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateAvroPayload.java index 217cf6cc88dee..08c020f63cacb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateAvroPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateAvroPayload.java @@ -143,7 +143,6 @@ private Option mergeOldRecord(IndexedRecord oldRecord, Schema schema, boolean isOldRecordNewer) throws IOException { Option recordOption = getInsertValue(schema); - if (!recordOption.isPresent()) { // use natural order for delete record return Option.empty(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 882b550f8845d..549d5559c2f45 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieTimelineTimeZone; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; @@ -172,6 +173,11 @@ public class HoodieTableConfig extends HoodieConfig { .withDocumentation("Payload class to use for performing compactions, i.e merge delta logs with current base file and then " + " produce a new base file."); + public static final ConfigProperty MERGER_STRATEGY = ConfigProperty + .key("hoodie.compaction.merger.strategy") + .defaultValue(HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID) + .withDocumentation("Id of merger strategy. Hudi will pick HoodieRecordMerger implementations in hoodie.datasource.write.merger.impls which has the same merger strategy id"); + public static final ConfigProperty ARCHIVELOG_FOLDER = ConfigProperty .key("hoodie.archivelog.folder") .defaultValue("archived") @@ -259,15 +265,24 @@ public class HoodieTableConfig extends HoodieConfig { private static final String TABLE_CHECKSUM_FORMAT = "%s.%s"; // . - public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) { + public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName, String mergerStrategyId) { super(); Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE); LOG.info("Loading table properties from " + propertyPath); try { fetchConfigs(fs, metaPath); + boolean needStore = false; if (contains(PAYLOAD_CLASS_NAME) && payloadClassName != null && !getString(PAYLOAD_CLASS_NAME).equals(payloadClassName)) { setValue(PAYLOAD_CLASS_NAME, payloadClassName); + needStore = true; + } + if (contains(MERGER_STRATEGY) && payloadClassName != null + && !getString(MERGER_STRATEGY).equals(mergerStrategyId)) { + setValue(MERGER_STRATEGY, mergerStrategyId); + needStore = true; + } + if (needStore) { // FIXME(vc): wonder if this can be removed. Need to look into history. try (FSDataOutputStream outputStream = fs.create(propertyPath)) { storeProperties(props, outputStream); @@ -434,6 +449,7 @@ public static void create(FileSystem fs, Path metadataFolder, Properties propert hoodieConfig.setDefaultValue(TYPE); if (hoodieConfig.getString(TYPE).equals(HoodieTableType.MERGE_ON_READ.name())) { hoodieConfig.setDefaultValue(PAYLOAD_CLASS_NAME); + hoodieConfig.setDefaultValue(MERGER_STRATEGY); } hoodieConfig.setDefaultValue(ARCHIVELOG_FOLDER); if (!hoodieConfig.contains(TIMELINE_LAYOUT_VERSION)) { @@ -503,6 +519,13 @@ public String getPayloadClass() { "org.apache.hudi"); } + /** + * Read the payload class for HoodieRecords from the table properties. + */ + public String getMergerStrategy() { + return getStringOrDefault(MERGER_STRATEGY); + } + public String getPreCombineField() { return getString(PRECOMBINE_FIELD); } 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 0938a61522a23..9dcd50c1cd838 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 @@ -102,7 +102,6 @@ public class HoodieTableMetaClient implements Serializable { // computations secured by its immutability protected SerializablePath basePath; protected SerializablePath metaPath; - private transient HoodieWrapperFileSystem fs; private boolean loadActiveTimelineOnLoad; protected SerializableConfiguration hadoopConf; @@ -123,7 +122,7 @@ public class HoodieTableMetaClient implements Serializable { */ protected HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad, ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, - String payloadClassName, FileSystemRetryConfig fileSystemRetryConfig) { + String payloadClassName, String mergerStrategy, FileSystemRetryConfig fileSystemRetryConfig) { LOG.info("Loading HoodieTableMetaClient from " + basePath); this.consistencyGuardConfig = consistencyGuardConfig; this.fileSystemRetryConfig = fileSystemRetryConfig; @@ -132,7 +131,7 @@ protected HoodieTableMetaClient(Configuration conf, String basePath, boolean loa this.metaPath = new SerializablePath(new CachingPath(basePath, METAFOLDER_NAME)); this.fs = getFs(); TableNotFoundException.checkTableValidity(fs, this.basePath.get(), metaPath.get()); - this.tableConfig = new HoodieTableConfig(fs, metaPath.toString(), payloadClassName); + this.tableConfig = new HoodieTableConfig(fs, metaPath.toString(), payloadClassName, mergerStrategy); this.tableType = tableConfig.getTableType(); Option tableConfigVersion = tableConfig.getTimelineLayoutVersion(); if (layoutVersion.isPresent() && tableConfigVersion.isPresent()) { @@ -167,6 +166,7 @@ public static HoodieTableMetaClient reload(HoodieTableMetaClient oldMetaClient) .setConsistencyGuardConfig(oldMetaClient.consistencyGuardConfig) .setLayoutVersion(Option.of(oldMetaClient.timelineLayoutVersion)) .setPayloadClassName(null) + .setMergerStrategy(null) .setFileSystemRetryConfig(oldMetaClient.fileSystemRetryConfig).build(); } @@ -651,7 +651,7 @@ public void initializeBootstrapDirsIfNotExists() throws IOException { private static HoodieTableMetaClient newMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad, ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, - String payloadClassName, FileSystemRetryConfig fileSystemRetryConfig, Properties props) { + String payloadClassName, String mergerStrategy, FileSystemRetryConfig fileSystemRetryConfig, Properties props) { HoodieMetastoreConfig metastoreConfig = null == props ? new HoodieMetastoreConfig.Builder().build() : new HoodieMetastoreConfig.Builder().fromProperties(props).build(); @@ -661,7 +661,7 @@ private static HoodieTableMetaClient newMetaClient(Configuration conf, String ba conf, consistencyGuardConfig, fileSystemRetryConfig, props.getProperty(HoodieTableConfig.DATABASE_NAME.key()), props.getProperty(HoodieTableConfig.NAME.key()), metastoreConfig) : new HoodieTableMetaClient(conf, basePath, - loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemRetryConfig); + loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, mergerStrategy, fileSystemRetryConfig); } public static Builder builder() { @@ -677,6 +677,7 @@ public static class Builder { private String basePath; private boolean loadActiveTimelineOnLoad = false; private String payloadClassName = null; + private String mergerStrategy = null; private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); private FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().build(); private Option layoutVersion = Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION); @@ -702,6 +703,11 @@ public Builder setPayloadClassName(String payloadClassName) { return this; } + public Builder setMergerStrategy(String mergerStrategy) { + this.mergerStrategy = mergerStrategy; + return this; + } + public Builder setConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) { this.consistencyGuardConfig = consistencyGuardConfig; return this; @@ -726,7 +732,8 @@ public HoodieTableMetaClient build() { ValidationUtils.checkArgument(conf != null, "Configuration needs to be set to init HoodieTableMetaClient"); ValidationUtils.checkArgument(basePath != null, "basePath needs to be set to init HoodieTableMetaClient"); return newMetaClient(conf, basePath, - loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemRetryConfig, props); + loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, + mergerStrategy, fileSystemRetryConfig, props); } } @@ -746,6 +753,7 @@ public static class PropertyBuilder { private String recordKeyFields; private String archiveLogFolder; private String payloadClassName; + private String mergerStrategy; private Integer timelineLayoutVersion; private String baseFileFormat; private String preCombineField; @@ -815,6 +823,11 @@ public PropertyBuilder setPayloadClassName(String payloadClassName) { return this; } + public PropertyBuilder setMergerStrategy(String mergerStrategy) { + this.mergerStrategy = mergerStrategy; + return this; + } + public PropertyBuilder setPayloadClass(Class payloadClass) { return setPayloadClassName(payloadClass.getName()); } @@ -934,7 +947,8 @@ public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) { return setTableType(metaClient.getTableType()) .setTableName(metaClient.getTableConfig().getTableName()) .setArchiveLogFolder(metaClient.getArchivePath()) - .setPayloadClassName(metaClient.getTableConfig().getPayloadClass()); + .setPayloadClassName(metaClient.getTableConfig().getPayloadClass()) + .setMergerStrategy(metaClient.getTableConfig().getMergerStrategy()); } public PropertyBuilder fromProperties(Properties properties) { @@ -964,6 +978,10 @@ public PropertyBuilder fromProperties(Properties properties) { setPayloadClassName( hoodieConfig.getString(HoodieTableConfig.PAYLOAD_CLASS_NAME)); } + if (hoodieConfig.contains(HoodieTableConfig.MERGER_STRATEGY)) { + setMergerStrategy( + hoodieConfig.getString(HoodieTableConfig.MERGER_STRATEGY)); + } if (hoodieConfig.contains(HoodieTableConfig.TIMELINE_LAYOUT_VERSION)) { setTimelineLayoutVersion(hoodieConfig.getInt(HoodieTableConfig.TIMELINE_LAYOUT_VERSION)); } @@ -1050,6 +1068,9 @@ public Properties build() { if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) { tableConfig.setValue(HoodieTableConfig.PAYLOAD_CLASS_NAME, payloadClassName); } + if (tableType == HoodieTableType.MERGE_ON_READ && mergerStrategy != null) { + tableConfig.setValue(HoodieTableConfig.MERGER_STRATEGY, mergerStrategy); + } if (null != tableCreateSchema) { tableConfig.setValue(HoodieTableConfig.CREATE_SCHEMA, tableCreateSchema); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 557bc929bf55e..4eddd6df0312b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -42,14 +42,13 @@ import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager; import org.apache.hudi.internal.schema.utils.SerDeHelper; -import org.apache.hudi.io.storage.HoodieHFileReader; -import org.apache.hudi.io.storage.HoodieOrcReader; +import org.apache.hudi.io.storage.HoodieAvroHFileReader; +import org.apache.hudi.io.storage.HoodieAvroOrcReader; import org.apache.hudi.util.Lazy; import org.apache.avro.JsonProperties; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -364,7 +363,7 @@ private MessageType readSchemaFromHFileBaseFile(Path hFilePath) throws IOExcepti FileSystem fs = metaClient.getRawFs(); CacheConfig cacheConfig = new CacheConfig(fs.getConf()); - HoodieHFileReader hFileReader = new HoodieHFileReader<>(fs.getConf(), hFilePath, cacheConfig); + HoodieAvroHFileReader hFileReader = new HoodieAvroHFileReader(fs.getConf(), hFilePath, cacheConfig); return convertAvroSchemaToParquet(hFileReader.getSchema()); } @@ -372,7 +371,7 @@ private MessageType readSchemaFromORCBaseFile(Path orcFilePath) throws IOExcepti LOG.info("Reading schema from " + orcFilePath); FileSystem fs = metaClient.getRawFs(); - HoodieOrcReader orcReader = new HoodieOrcReader<>(fs.getConf(), orcFilePath); + HoodieAvroOrcReader orcReader = new HoodieAvroOrcReader(fs.getConf(), orcFilePath); return convertAvroSchemaToParquet(orcReader.getSchema()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index 0b5066792f6e2..ca53802a3afee 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -18,12 +18,14 @@ package org.apache.hudi.common.table.log; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.DeleteRecord; -import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePayloadProps; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; @@ -35,9 +37,9 @@ import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.ClosableIteratorWithSchema; import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.CloseableMappingIterator; import org.apache.hudi.common.util.collection.Pair; @@ -48,8 +50,6 @@ import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -71,7 +71,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.apache.hudi.avro.HoodieAvroUtils.rewriteRecordWithNewSchema; import static org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.COMPACTED_BLOCK_TIMES; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; @@ -100,12 +99,14 @@ public abstract class AbstractHoodieLogRecordReader { // Latest valid instant time // Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark. private final String latestInstantTime; - private final HoodieTableMetaClient hoodieTableMetaClient; + protected final HoodieTableMetaClient hoodieTableMetaClient; // Merge strategy to use when combining records from log private final String payloadClassFQN; // preCombine field - private final String preCombineField; - private final Properties payloadProps = new Properties(); + protected final String preCombineField; + // Stateless component for merging records + protected final HoodieRecordMerger recordMerger; + private final TypedProperties payloadProps; // simple key gen fields private Option> simpleKeyGenFields = Option.empty(); // Log File Paths @@ -148,6 +149,8 @@ public abstract class AbstractHoodieLogRecordReader { private Option partitionName; // Populate meta fields for the records private boolean populateMetaFields = true; + // Record type read from log block + protected final HoodieRecordType recordType; // Collect all the block instants after scanning all the log files. private List validBlockInstants = new ArrayList<>(); // Use scanV2 method. @@ -157,16 +160,17 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List instantRange, - boolean withOperationField) { + boolean withOperationField, HoodieRecordMerger recordMerger) { this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, - instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), false); + instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), false, recordMerger); } protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize, Option instantRange, boolean withOperationField, boolean forceFullScan, - Option partitionName, InternalSchema internalSchema, boolean useScanV2) { + Option partitionName, InternalSchema internalSchema, + boolean useScanV2, HoodieRecordMerger recordMerger) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build(); @@ -174,9 +178,13 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List keySpecOpt) throws Exception { - try (ClosableIterator recordIterator = getRecordsIterator(dataBlock, keySpecOpt)) { + try (ClosableIteratorWithSchema recordIterator = getRecordsIterator(dataBlock, keySpecOpt)) { while (recordIterator.hasNext()) { - processNextRecord(createHoodieRecord(recordIterator.next(), this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN, - this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName)); + HoodieRecord completedRecord = recordIterator.next() + .wrapIntoHoodieRecordPayloadWithParams(recordIterator.getSchema(), + hoodieTableMetaClient.getTableConfig().getProps(), + this.simpleKeyGenFields, + this.withOperationField, + this.partitionName, + getPopulateMetaFields()); + processNextRecord(completedRecord); totalLogRecords.incrementAndGet(); } } } - /** - * Create @{@link HoodieRecord} from the @{@link IndexedRecord}. - * - * @param rec - IndexedRecord to create the HoodieRecord from - * @param hoodieTableConfig - Table config - * @param payloadClassFQN - Payload class fully qualified name - * @param preCombineField - PreCombine field - * @param withOperationField - Whether operation field is enabled - * @param simpleKeyGenFields - Key generator fields when populate meta fields is tuened off - * @param partitionName - Partition name - * @return HoodieRecord created from the IndexedRecord - */ - protected HoodieAvroRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, - final String payloadClassFQN, final String preCombineField, - final boolean withOperationField, - final Option> simpleKeyGenFields, - final Option partitionName) { - if (this.populateMetaFields) { - return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN, - preCombineField, withOperationField); - } else { - return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN, - preCombineField, simpleKeyGenFields.get(), withOperationField, partitionName); - } + protected boolean getPopulateMetaFields() { + return this.populateMetaFields; } /** @@ -669,7 +662,7 @@ protected HoodieAvroRecord createHoodieRecord(final IndexedRecord rec, final * * @param hoodieRecord Hoodie Record to process */ - protected abstract void processNextRecord(HoodieRecord hoodieRecord) throws Exception; + protected abstract void processNextRecord(HoodieRecord hoodieRecord) throws Exception; /** * Process next deleted record. @@ -750,7 +743,7 @@ public boolean isWithOperationField() { return withOperationField; } - protected Properties getPayloadProps() { + protected TypedProperties getPayloadProps() { return payloadProps; } @@ -775,23 +768,28 @@ public List getValidBlockInstants() { return validBlockInstants; } - private ClosableIterator getRecordsIterator(HoodieDataBlock dataBlock, Option keySpecOpt) throws IOException { - ClosableIterator blockRecordsIterator; + private ClosableIteratorWithSchema getRecordsIterator( + HoodieDataBlock dataBlock, Option keySpecOpt) throws IOException { + ClosableIterator blockRecordsIterator; if (keySpecOpt.isPresent()) { KeySpec keySpec = keySpecOpt.get(); - blockRecordsIterator = dataBlock.getRecordIterator(keySpec.keys, keySpec.fullKey); + blockRecordsIterator = (ClosableIterator) dataBlock + .getRecordIterator(keySpec.keys, keySpec.fullKey, recordType); } else { - blockRecordsIterator = dataBlock.getRecordIterator(); + blockRecordsIterator = (ClosableIterator) dataBlock.getRecordIterator(recordType); } - Option> schemaEvolutionTransformerOpt = + Option, Schema>> schemaEvolutionTransformerOpt = composeEvolvedSchemaTransformer(dataBlock); // In case when schema has been evolved original persisted records will have to be // transformed to adhere to the new schema if (schemaEvolutionTransformerOpt.isPresent()) { - return new CloseableMappingIterator<>(blockRecordsIterator, schemaEvolutionTransformerOpt.get()); + return ClosableIteratorWithSchema.newInstance( + new CloseableMappingIterator<>(blockRecordsIterator, + schemaEvolutionTransformerOpt.get().getLeft()), + schemaEvolutionTransformerOpt.get().getRight()); } else { - return blockRecordsIterator; + return ClosableIteratorWithSchema.newInstance(blockRecordsIterator, dataBlock.getSchema()); } } @@ -804,7 +802,8 @@ private ClosableIterator getRecordsIterator(HoodieDataBlock dataB * @param dataBlock current processed block * @return final read schema. */ - private Option> composeEvolvedSchemaTransformer(HoodieDataBlock dataBlock) { + private Option, Schema>> composeEvolvedSchemaTransformer( + HoodieDataBlock dataBlock) { if (internalSchema.isEmptySchema()) { return Option.empty(); } @@ -816,7 +815,18 @@ private Option> composeEvolvedSchemaTrans true, false).mergeSchema(); Schema mergedAvroSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema, readerSchema.getFullName()); - return Option.of((record) -> rewriteRecordWithNewSchema(record, mergedAvroSchema, Collections.emptyMap())); + return Option.of(Pair.of((record) -> { + try { + return record.rewriteRecordWithNewSchema( + dataBlock.getSchema(), + this.hoodieTableMetaClient.getTableConfig().getProps(), + mergedAvroSchema, + Collections.emptyMap()); + } catch (IOException e) { + LOG.error("Error rewrite record with new schema", e); + throw new HoodieException(e); + } + }, mergedAvroSchema)); } /** @@ -854,6 +864,10 @@ public Builder withOperationField(boolean withOperationField) { throw new UnsupportedOperationException(); } + public Builder withRecordMerger(HoodieRecordMerger recordMerger) { + throw new UnsupportedOperationException(); + } + public Builder withUseScanV2(boolean useScanV2) { throw new UnsupportedOperationException(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieCDCLogRecordIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieCDCLogRecordIterator.java index 230dae73de99a..0d247e09410cf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieCDCLogRecordIterator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieCDCLogRecordIterator.java @@ -18,9 +18,12 @@ package org.apache.hudi.common.table.log; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.exception.HoodieIOException; import org.apache.avro.Schema; @@ -92,7 +95,8 @@ private boolean loadReader() { private boolean loadItr() { HoodieDataBlock dataBlock = (HoodieDataBlock) reader.next(); closeItr(); - itr = dataBlock.getRecordIterator(); + // TODO support cdc with spark record. + itr = new MappingIterator(dataBlock.getRecordIterator(HoodieRecordType.AVRO), record -> ((HoodieAvroIndexedRecord) record).getData()); return itr.hasNext(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java index 3322a9a2f2779..adbc0a1e1b34a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java @@ -227,7 +227,7 @@ private HoodieLogBlock readBlock() throws IOException { String.format("Parquet block could not be of version (%d)", HoodieLogFormatVersion.DEFAULT_VERSION)); return new HoodieParquetDataBlock(inputStream, content, readBlockLazily, logBlockContentLoc, - Option.ofNullable(readerSchema), header, footer, keyField); + getTargetReaderSchemaForBlock(), header, footer, keyField); case DELETE_BLOCK: return new HoodieDeleteBlock(content, inputStream, readBlockLazily, Option.of(logBlockContentLoc), header, footer); @@ -284,7 +284,7 @@ private HoodieLogBlock createCorruptBlock(long blockStartPos) throws IOException private boolean isBlockCorrupted(int blocksize) throws IOException { long currentPos = inputStream.getPos(); long blockSizeFromFooter; - + try { // check if the blocksize mentioned in the footer is the same as the header; // by seeking and checking the length of a long. We do not seek `currentPos + blocksize` diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index 5fda4bb7def7f..3ff1432f64045 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -20,11 +20,11 @@ import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.model.DeleteRecord; -import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieEmptyRecord; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.cdc.HoodieCDCUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.DefaultSizeEstimator; @@ -33,6 +33,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.SpillableMapUtils; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.internal.schema.InternalSchema; @@ -66,13 +67,13 @@ */ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader - implements Iterable> { + implements Iterable { private static final Logger LOG = LogManager.getLogger(HoodieMergedLogRecordScanner.class); // A timer for calculating elapsed time in millis public final HoodieTimer timer = new HoodieTimer(); // Final map of compacted/merged records - protected final ExternalSpillableMap> records; + protected final ExternalSpillableMap records; // count of merged records in log private long numMergedRecordsInLog; private long maxMemorySizeInBytes; @@ -88,10 +89,10 @@ protected HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List partitionName, InternalSchema internalSchema, - boolean useScanV2) { + boolean useScanV2, HoodieRecordMerger recordMerger) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, withOperationField, - forceFullScan, partitionName, internalSchema, useScanV2); + forceFullScan, partitionName, internalSchema, useScanV2, recordMerger); try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(), @@ -123,16 +124,20 @@ protected void performScan() { } @Override - public Iterator> iterator() { + public Iterator iterator() { checkState(forceFullScan, "Record reader has to be in full-scan mode to use this API"); return records.iterator(); } - public Map> getRecords() { + public Map getRecords() { checkState(forceFullScan, "Record reader has to be in full-scan mode to use this API"); return records; } + public HoodieRecordType getRecordType() { + return recordMerger.getRecordType(); + } + public long getNumMergedRecordsInLog() { return numMergedRecordsInLog; } @@ -145,40 +150,46 @@ public static HoodieMergedLogRecordScanner.Builder newBuilder() { } @Override - protected void processNextRecord(HoodieRecord hoodieRecord) throws IOException { - String key = hoodieRecord.getRecordKey(); + protected void processNextRecord(HoodieRecord newRecord) throws IOException { + String key = newRecord.getRecordKey(); if (records.containsKey(key)) { // Merge and store the merged record. The HoodieRecordPayload implementation is free to decide what should be // done when a DELETE (empty payload) is encountered before or after an insert/update. - HoodieRecord oldRecord = records.get(key); - HoodieRecordPayload oldValue = oldRecord.getData(); - HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(oldValue, readerSchema, this.getPayloadProps()); + HoodieRecord oldRecord = records.get(key); + T oldValue = oldRecord.getData(); + HoodieRecord combinedRecord = (HoodieRecord) recordMerger.merge(oldRecord, readerSchema, + newRecord, readerSchema, this.getPayloadProps()).get().getLeft(); // If combinedValue is oldValue, no need rePut oldRecord - if (combinedValue != oldValue) { - HoodieOperation operation = hoodieRecord.getOperation(); - HoodieRecord latestHoodieRecord = new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation); + if (combinedRecord.getData() != oldValue) { + HoodieRecord latestHoodieRecord = combinedRecord.newInstance(new HoodieKey(key, newRecord.getPartitionPath()), newRecord.getOperation()); latestHoodieRecord.unseal(); - latestHoodieRecord.setCurrentLocation(hoodieRecord.getCurrentLocation()); + latestHoodieRecord.setCurrentLocation(newRecord.getCurrentLocation()); latestHoodieRecord.seal(); - records.put(key, latestHoodieRecord); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into records(Map). + records.put(key, latestHoodieRecord.copy()); } } else { // Put the record as is - records.put(key, hoodieRecord); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into records(Map). + records.put(key, newRecord.copy()); } } @Override protected void processNextDeletedRecord(DeleteRecord deleteRecord) { String key = deleteRecord.getRecordKey(); - HoodieRecord oldRecord = records.get(key); + HoodieRecord oldRecord = records.get(key); if (oldRecord != null) { // Merge and store the merged record. The ordering val is taken to decide whether the same key record // should be deleted or be kept. The old record is kept only if the DELETE record has smaller ordering val. // For same ordering values, uses the natural order(arrival time semantics). - Comparable curOrderingVal = oldRecord.getData().getOrderingValue(); + Comparable curOrderingVal = oldRecord.getOrderingValue(this.readerSchema, this.hoodieTableMetaClient.getTableConfig().getProps()); Comparable deleteOrderingVal = deleteRecord.getOrderingValue(); // Checks the ordering value does not equal to 0 // because we use 0 as the default value which means natural order @@ -191,8 +202,13 @@ protected void processNextDeletedRecord(DeleteRecord deleteRecord) { } } // Put the DELETE record - records.put(key, SpillableMapUtils.generateEmptyPayload(key, - deleteRecord.getPartitionPath(), deleteRecord.getOrderingValue(), getPayloadClassFQN())); + if (recordType == HoodieRecordType.AVRO) { + records.put(key, SpillableMapUtils.generateEmptyPayload(key, + deleteRecord.getPartitionPath(), deleteRecord.getOrderingValue(), getPayloadClassFQN())); + } else { + HoodieEmptyRecord record = new HoodieEmptyRecord<>(new HoodieKey(key, deleteRecord.getPartitionPath()), null, deleteRecord.getOrderingValue(), recordType); + records.put(key, record); + } } public long getTotalTimeTakenToReadAndMergeBlocks() { @@ -232,6 +248,7 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder { private boolean withOperationField = false; // Use scanV2 method. private boolean useScanV2 = false; + private HoodieRecordMerger recordMerger; @Override public Builder withFileSystem(FileSystem fs) { @@ -332,16 +349,24 @@ public Builder withUseScanV2(boolean useScanV2) { return this; } + @Override + public Builder withRecordMerger(HoodieRecordMerger recordMerger) { + this.recordMerger = recordMerger; + return this; + } + @Override public HoodieMergedLogRecordScanner build() { if (this.partitionName == null && CollectionUtils.nonEmpty(this.logFilePaths)) { this.partitionName = getRelativePartitionPath(new Path(basePath), new Path(this.logFilePaths.get(0)).getParent()); } + ValidationUtils.checkArgument(recordMerger != null); + return new HoodieMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader, bufferSize, spillableMapBasePath, instantRange, diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true, - Option.ofNullable(partitionName), internalSchema, useScanV2); + Option.ofNullable(partitionName), internalSchema, useScanV2, recordMerger); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java index 863fa8162b1a6..c5468f0051277 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java @@ -20,9 +20,10 @@ import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.cdc.HoodieCDCUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.avro.Schema; @@ -41,9 +42,9 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade private HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize, LogRecordScannerCallback callback, Option instantRange, InternalSchema internalSchema, - boolean useScanV2) { + boolean useScanV2, HoodieRecordMerger recordMerger) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, - false, true, Option.empty(), internalSchema, useScanV2); + false, true, Option.empty(), internalSchema, useScanV2, recordMerger); this.callback = callback; } @@ -55,9 +56,12 @@ public static HoodieUnMergedLogRecordScanner.Builder newBuilder() { } @Override - protected void processNextRecord(HoodieRecord hoodieRecord) throws Exception { + protected void processNextRecord(HoodieRecord hoodieRecord) throws Exception { + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be put into queue of BoundedInMemoryExecutor. // Just call callback without merging - callback.apply(hoodieRecord); + callback.apply(hoodieRecord.copy()); } @Override @@ -71,7 +75,7 @@ protected void processNextDeletedRecord(DeleteRecord deleteRecord) { @FunctionalInterface public interface LogRecordScannerCallback { - void apply(HoodieRecord record) throws Exception; + void apply(HoodieRecord record) throws Exception; } /** @@ -91,6 +95,7 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder { // specific configurations private LogRecordScannerCallback callback; private boolean useScanV2; + private HoodieRecordMerger recordMerger; public Builder withFileSystem(FileSystem fs) { this.fs = fs; @@ -156,11 +161,19 @@ public Builder withUseScanV2(boolean useScanV2) { return this; } + @Override + public Builder withRecordMerger(HoodieRecordMerger recordMerger) { + this.recordMerger = recordMerger; + return this; + } + @Override public HoodieUnMergedLogRecordScanner build() { + ValidationUtils.checkArgument(recordMerger != null); + return new HoodieUnMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, callback, instantRange, - internalSchema, useScanV2); + internalSchema, useScanV2, recordMerger); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java index d68ac4e203680..afc7937f2171a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java @@ -19,8 +19,14 @@ package org.apache.hudi.common.table.log.block; import org.apache.hudi.common.fs.SizeAwareDataInputStream; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockContentLocation; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.internal.schema.InternalSchema; @@ -53,9 +59,11 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; import static org.apache.hudi.common.util.ValidationUtils.checkState; /** @@ -76,9 +84,10 @@ public HoodieAvroDataBlock(FSDataInputStream inputStream, super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, keyField, false); } - public HoodieAvroDataBlock(@Nonnull List records, - @Nonnull Map header, - @Nonnull String keyField) { + public HoodieAvroDataBlock(@Nonnull List records, + @Nonnull Map header, + @Nonnull String keyField + ) { super(records, header, new HashMap<>(), keyField); } @@ -88,7 +97,7 @@ public HoodieLogBlockType getBlockType() { } @Override - protected byte[] serializeRecords(List records) throws IOException { + protected byte[] serializeRecords(List records) throws IOException { Schema schema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); GenericDatumWriter writer = new GenericDatumWriter<>(schema); ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -101,13 +110,15 @@ protected byte[] serializeRecords(List records) throws IOExceptio output.writeInt(records.size()); // 3. Write the records - for (IndexedRecord s : records) { + for (HoodieRecord s : records) { ByteArrayOutputStream temp = new ByteArrayOutputStream(); BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(temp, encoderCache.get()); encoderCache.set(encoder); try { // Encode the record into bytes - writer.write(s, encoder); + // Spark Record not support write avro log + IndexedRecord data = s.toIndexedRecord(schema, new Properties()).get().getData(); + writer.write(data, encoder); encoder.flush(); // Get the size of the bytes @@ -127,9 +138,12 @@ protected byte[] serializeRecords(List records) throws IOExceptio // TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used @Override - protected ClosableIterator deserializeRecords(byte[] content) throws IOException { + protected ClosableIterator> deserializeRecords(byte[] content, HoodieRecordType type) throws IOException { checkState(this.readerSchema != null, "Reader's schema has to be non-null"); - return RecordIterator.getInstance(this, content); + checkArgument(type != HoodieRecordType.SPARK, "Not support read avro to spark record"); + // TODO AvroSparkReader need + RecordIterator iterator = RecordIterator.getInstance(this, content); + return new MappingIterator<>(iterator, data -> (HoodieRecord) new HoodieAvroIndexedRecord(data)); } private static class RecordIterator implements ClosableIterator { @@ -206,7 +220,7 @@ public IndexedRecord next() { * HoodieLogFormat V1. */ @Deprecated - public HoodieAvroDataBlock(List records, Schema schema) { + public HoodieAvroDataBlock(List records, Schema schema) { super(records, Collections.singletonMap(HeaderMetadataType.SCHEMA, schema.toString()), new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD); } @@ -240,14 +254,14 @@ public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema, GenericDatumReader reader = new GenericDatumReader<>(writerSchema, readerSchema); // 2. Get the total records int totalRecords = dis.readInt(); - List records = new ArrayList<>(totalRecords); + List records = new ArrayList<>(totalRecords); // 3. Read the content for (int i = 0; i < totalRecords; i++) { int recordLength = dis.readInt(); Decoder decoder = DecoderFactory.get().binaryDecoder(content, dis.getNumberOfBytesRead(), recordLength, null); IndexedRecord record = reader.read(null, decoder); - records.add(record); + records.add(new HoodieAvroIndexedRecord(record)); dis.skipBytes(recordLength); } dis.close(); @@ -293,8 +307,8 @@ public byte[] getBytes(Schema schema) throws IOException { output.writeInt(schemaContent.length); output.write(schemaContent); - List records = new ArrayList<>(); - try (ClosableIterator recordItr = getRecordIterator()) { + List> records = new ArrayList<>(); + try (ClosableIterator> recordItr = getRecordIterator(HoodieRecordType.AVRO)) { recordItr.forEachRemaining(records::add); } @@ -302,9 +316,9 @@ public byte[] getBytes(Schema schema) throws IOException { output.writeInt(records.size()); // 3. Write the records - Iterator itr = records.iterator(); + Iterator> itr = records.iterator(); while (itr.hasNext()) { - IndexedRecord s = itr.next(); + IndexedRecord s = itr.next().toIndexedRecord(schema, new Properties()).get().getData(); ByteArrayOutputStream temp = new ByteArrayOutputStream(); Encoder encoder = EncoderFactory.get().binaryEncoder(temp, null); try { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCDCDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCDCDataBlock.java index 52acacf93b569..93bd41b88d0e5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCDCDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCDCDataBlock.java @@ -18,10 +18,10 @@ package org.apache.hudi.common.table.log.block; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FSDataInputStream; import java.util.HashMap; @@ -45,7 +45,7 @@ public HoodieCDCDataBlock( Option.of(readerSchema), header, new HashMap<>(), keyField); } - public HoodieCDCDataBlock(List records, + public HoodieCDCDataBlock(List records, Map header, String keyField) { super(records, header, keyField); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java index 0d5cc60a6ee8c..17e4d7a08e274 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java @@ -18,13 +18,14 @@ package org.apache.hudi.common.table.log.block; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hudi.common.model.HoodieRecord; import java.io.IOException; import java.util.HashSet; @@ -34,6 +35,7 @@ import java.util.Set; import java.util.function.Function; +import static org.apache.hudi.common.util.TypeUtils.unsafeCast; import static org.apache.hudi.common.util.ValidationUtils.checkState; /** @@ -49,7 +51,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock { // TODO rebase records/content to leverage Either to warrant // that they are mutex (used by read/write flows respectively) - private final Option> records; + private final Option> records; /** * Key field's name w/in the record's schema @@ -58,12 +60,12 @@ public abstract class HoodieDataBlock extends HoodieLogBlock { private final boolean enablePointLookups; - protected final Schema readerSchema; + protected Schema readerSchema; /** * NOTE: This ctor is used on the write-path (ie when records ought to be written into the log) */ - public HoodieDataBlock(List records, + public HoodieDataBlock(List records, Map header, Map footer, String keyFieldName) { @@ -116,13 +118,14 @@ protected static Schema getWriterSchema(Map logBlock /** * Returns all the records iterator contained w/in this block. */ - public final ClosableIterator getRecordIterator() { + public final ClosableIterator> getRecordIterator(HoodieRecordType type) { if (records.isPresent()) { - return list2Iterator(records.get()); + // TODO need convert record type + return list2Iterator(unsafeCast(records.get())); } try { // in case records are absent, read content lazily and then convert to IndexedRecords - return readRecordsFromBlockPayload(); + return readRecordsFromBlockPayload(type); } catch (IOException io) { throw new HoodieIOException("Unable to convert content bytes to records", io); } @@ -140,7 +143,7 @@ public Schema getSchema() { * @return List of IndexedRecords for the keys of interest. * @throws IOException in case of failures encountered when reading/parsing records */ - public final ClosableIterator getRecordIterator(List keys, boolean fullKey) throws IOException { + public final ClosableIterator> getRecordIterator(List keys, boolean fullKey, HoodieRecordType type) throws IOException { boolean fullScan = keys.isEmpty(); if (enablePointLookups && !fullScan) { return lookupRecords(keys, fullKey); @@ -148,7 +151,7 @@ public final ClosableIterator getRecordIterator(List keys // Otherwise, we fetch all the records and filter out all the records, but the // ones requested - ClosableIterator allRecords = getRecordIterator(); + ClosableIterator> allRecords = getRecordIterator(type); if (fullScan) { return allRecords; } @@ -157,29 +160,29 @@ public final ClosableIterator getRecordIterator(List keys return FilteringIterator.getInstance(allRecords, keySet, fullKey, this::getRecordKey); } - protected ClosableIterator readRecordsFromBlockPayload() throws IOException { + protected ClosableIterator> readRecordsFromBlockPayload(HoodieRecordType type) throws IOException { if (readBlockLazily && !getContent().isPresent()) { // read log block contents from disk inflate(); } try { - return deserializeRecords(getContent().get()); + return deserializeRecords(getContent().get(), type); } finally { // Free up content to be GC'd by deflating the block deflate(); } } - protected ClosableIterator lookupRecords(List keys, boolean fullKey) throws IOException { + protected ClosableIterator> lookupRecords(List keys, boolean fullKey) throws IOException { throw new UnsupportedOperationException( String.format("Point lookups are not supported by this Data block type (%s)", getBlockType()) ); } - protected abstract byte[] serializeRecords(List records) throws IOException; + protected abstract byte[] serializeRecords(List records) throws IOException; - protected abstract ClosableIterator deserializeRecords(byte[] content) throws IOException; + protected abstract ClosableIterator> deserializeRecords(byte[] content, HoodieRecordType type) throws IOException; public abstract HoodieLogBlockType getBlockType(); @@ -187,10 +190,8 @@ protected Option getKeyField(Schema schema) { return Option.ofNullable(schema.getField(keyFieldName)); } - protected Option getRecordKey(IndexedRecord record) { - return getKeyField(record.getSchema()) - .map(keyField -> record.get(keyField.pos())) - .map(Object::toString); + protected Option getRecordKey(HoodieRecord record) { + return Option.ofNullable(record.getRecordKey(readerSchema, keyFieldName)); } /** @@ -223,31 +224,29 @@ public T next() { /** * A {@link ClosableIterator} that supports filtering strategy with given keys. * User should supply the key extraction function for fetching string format keys. - * - * @param the element type */ - private static class FilteringIterator implements ClosableIterator { - private final ClosableIterator nested; // nested iterator + private static class FilteringIterator implements ClosableIterator> { + private final ClosableIterator> nested; // nested iterator private final Set keys; // the filtering keys private final boolean fullKey; - private final Function> keyExtract; // function to extract the key + private final Function, Option> keyExtract; // function to extract the key - private T next; + private HoodieRecord next; - private FilteringIterator(ClosableIterator nested, Set keys, boolean fullKey, Function> keyExtract) { + private FilteringIterator(ClosableIterator> nested, Set keys, boolean fullKey, Function, Option> keyExtract) { this.nested = nested; this.keys = keys; this.fullKey = fullKey; this.keyExtract = keyExtract; } - public static FilteringIterator getInstance( - ClosableIterator nested, + public static FilteringIterator getInstance( + ClosableIterator> nested, Set keys, boolean fullKey, - Function> keyExtract) { + Function, Option> keyExtract) { return new FilteringIterator<>(nested, keys, fullKey, keyExtract); } @@ -274,7 +273,7 @@ public boolean hasNext() { } @Override - public T next() { + public HoodieRecord next() { return this.next; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index 155e211141942..5de47ab70638c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -21,13 +21,15 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.inline.InLineFSUtils; import org.apache.hudi.common.fs.inline.InLineFileSystem; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.HoodieAvroHFileReader; import org.apache.hudi.io.storage.HoodieHBaseKVComparator; -import org.apache.hudi.io.storage.HoodieHFileReader; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -50,8 +52,10 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.TreeMap; +import static org.apache.hudi.common.util.TypeUtils.unsafeCast; import static org.apache.hudi.common.util.ValidationUtils.checkState; /** @@ -76,16 +80,16 @@ public HoodieHFileDataBlock(FSDataInputStream inputStream, Map footer, boolean enablePointLookups, Path pathForReader) { - super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, HoodieHFileReader.KEY_FIELD_NAME, enablePointLookups); + super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, HoodieAvroHFileReader.KEY_FIELD_NAME, enablePointLookups); this.compressionAlgorithm = Option.empty(); this.pathForReader = pathForReader; } - public HoodieHFileDataBlock(List records, + public HoodieHFileDataBlock(List records, Map header, Compression.Algorithm compressionAlgorithm, Path pathForReader) { - super(records, header, new HashMap<>(), HoodieHFileReader.KEY_FIELD_NAME); + super(records, header, new HashMap<>(), HoodieAvroHFileReader.KEY_FIELD_NAME); this.compressionAlgorithm = Option.of(compressionAlgorithm); this.pathForReader = pathForReader; } @@ -96,7 +100,7 @@ public HoodieLogBlockType getBlockType() { } @Override - protected byte[] serializeRecords(List records) throws IOException { + protected byte[] serializeRecords(List records) throws IOException { HFileContext context = new HFileContextBuilder() .withBlockSize(DEFAULT_BLOCK_SIZE) .withCompression(compressionAlgorithm.get()) @@ -115,11 +119,13 @@ protected byte[] serializeRecords(List records) throws IOExceptio // Serialize records into bytes Map sortedRecordsMap = new TreeMap<>(); - Iterator itr = records.iterator(); + // Get writer schema + Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); + Iterator itr = records.iterator(); int id = 0; while (itr.hasNext()) { - IndexedRecord record = itr.next(); + HoodieRecord record = itr.next(); String recordKey; if (useIntegerKey) { recordKey = String.format("%" + keyWidth + "s", id++); @@ -127,7 +133,7 @@ protected byte[] serializeRecords(List records) throws IOExceptio recordKey = getRecordKey(record).get(); } - final byte[] recordBytes = serializeRecord(record); + final byte[] recordBytes = serializeRecord(record, writerSchema); ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey), "Writing multiple records with same key not supported for " + this.getClass().getName()); sortedRecordsMap.put(recordKey, recordBytes); @@ -146,7 +152,7 @@ protected byte[] serializeRecords(List records) throws IOExceptio } }); - writer.appendFileInfo(HoodieHFileReader.SCHEMA_KEY.getBytes(), getSchema().toString().getBytes()); + writer.appendFileInfo(HoodieAvroHFileReader.SCHEMA_KEY.getBytes(), getSchema().toString().getBytes()); writer.close(); ostream.flush(); @@ -156,36 +162,20 @@ protected byte[] serializeRecords(List records) throws IOExceptio } @Override - protected ClosableIterator deserializeRecords(byte[] content) throws IOException { + protected ClosableIterator> deserializeRecords(byte[] content, HoodieRecordType type) throws IOException { checkState(readerSchema != null, "Reader's schema has to be non-null"); // Get schema from the header Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); // Read the content - HoodieHFileReader reader = new HoodieHFileReader<>(null, pathForReader, content, Option.of(writerSchema)); - Iterator recordIterator = reader.getRecordIterator(readerSchema); - return new ClosableIterator() { - @Override - public void close() { - reader.close(); - } - - @Override - public boolean hasNext() { - return recordIterator.hasNext(); - } - - @Override - public IndexedRecord next() { - return recordIterator.next(); - } - }; + HoodieAvroHFileReader reader = new HoodieAvroHFileReader(null, pathForReader, content, Option.of(writerSchema)); + return unsafeCast(reader.getRecordIterator(readerSchema)); } // TODO abstract this w/in HoodieDataBlock @Override - protected ClosableIterator lookupRecords(List keys, boolean fullKey) throws IOException { + protected ClosableIterator> lookupRecords(List keys, boolean fullKey) throws IOException { HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get(); // NOTE: It's important to extend Hadoop configuration here to make sure configuration @@ -205,38 +195,22 @@ protected ClosableIterator lookupRecords(List keys, boole List sortedKeys = new ArrayList<>(keys); Collections.sort(sortedKeys); - final HoodieHFileReader reader = - new HoodieHFileReader<>(inlineConf, inlinePath, new CacheConfig(inlineConf), inlinePath.getFileSystem(inlineConf)); + final HoodieAvroHFileReader reader = + new HoodieAvroHFileReader(inlineConf, inlinePath, new CacheConfig(inlineConf), inlinePath.getFileSystem(inlineConf)); // Get writer's schema from the header - final ClosableIterator recordIterator = + final ClosableIterator> recordIterator = fullKey ? reader.getRecordsByKeysIterator(sortedKeys, readerSchema) : reader.getRecordsByKeyPrefixIterator(sortedKeys, readerSchema); - return new ClosableIterator() { - @Override - public boolean hasNext() { - return recordIterator.hasNext(); - } - - @Override - public IndexedRecord next() { - return recordIterator.next(); - } - - @Override - public void close() { - recordIterator.close(); - reader.close(); - } - }; + return new MappingIterator<>(recordIterator, data -> (HoodieRecord) data); } - private byte[] serializeRecord(IndexedRecord record) { - Option keyField = getKeyField(record.getSchema()); + private byte[] serializeRecord(HoodieRecord record, Schema schema) throws IOException { + Option keyField = getKeyField(schema); // Reset key value w/in the record to avoid duplicating the key w/in payload if (keyField.isPresent()) { - record.put(keyField.get().pos(), StringUtils.EMPTY_STRING); + record.truncateRecordKey(schema, new Properties(), keyField.get().name()); } - return HoodieAvroUtils.indexedRecordToBytes(record); + return HoodieAvroUtils.recordToBytes(record, schema).get(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java index 2c0034c4d8bab..cdd2190d910e8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java @@ -18,29 +18,25 @@ package org.apache.hudi.common.table.log.block; -import org.apache.hudi.avro.HoodieAvroWriteSupport; +import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.fs.inline.InLineFSUtils; import org.apache.hudi.common.fs.inline.InLineFileSystem; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ParquetReaderIterator; -import org.apache.hudi.io.storage.HoodieParquetConfig; -import org.apache.hudi.io.storage.HoodieParquetStreamWriter; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.io.storage.HoodieFileWriter; +import org.apache.hudi.io.storage.HoodieFileWriterFactory; import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; -import org.apache.parquet.avro.AvroParquetReader; -import org.apache.parquet.avro.AvroReadSupport; -import org.apache.parquet.avro.AvroSchemaConverter; -import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; -import org.apache.parquet.hadoop.util.HadoopInputFile; -import org.apache.parquet.io.InputFile; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -48,6 +44,14 @@ import java.util.List; import java.util.Map; +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_BLOCK_SIZE; +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME; +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION; +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED; +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_MAX_FILE_SIZE; +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_PAGE_SIZE; +import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; + /** * HoodieParquetDataBlock contains a list of records serialized using Parquet. */ @@ -72,7 +76,7 @@ public HoodieParquetDataBlock(FSDataInputStream inputStream, this.useDictionaryEncoding = Option.empty(); } - public HoodieParquetDataBlock(List records, + public HoodieParquetDataBlock(List records, Map header, String keyField, CompressionCodecName compressionCodecName, @@ -92,60 +96,53 @@ public HoodieLogBlockType getBlockType() { } @Override - protected byte[] serializeRecords(List records) throws IOException { + protected byte[] serializeRecords(List records) throws IOException { if (records.size() == 0) { return new byte[0]; } Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); - - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( - new AvroSchemaConverter().convert(writerSchema), writerSchema, Option.empty()); - - HoodieParquetConfig avroParquetConfig = - new HoodieParquetConfig<>( - writeSupport, - compressionCodecName.get(), - ParquetWriter.DEFAULT_BLOCK_SIZE, - ParquetWriter.DEFAULT_PAGE_SIZE, - 1024 * 1024 * 1024, - new Configuration(), - expectedCompressionRatio.get(), - useDictionaryEncoding.get()); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - - try (FSDataOutputStream outputStream = new FSDataOutputStream(baos)) { - try (HoodieParquetStreamWriter parquetWriter = new HoodieParquetStreamWriter<>(outputStream, avroParquetConfig)) { - for (IndexedRecord record : records) { + try (FSDataOutputStream outputStream = new FSDataOutputStream(baos, null)) { + HoodieFileWriter parquetWriter = null; + HoodieConfig config = new HoodieConfig(); + config.setValue(PARQUET_COMPRESSION_CODEC_NAME.key(), compressionCodecName.get().name()); + config.setValue(PARQUET_BLOCK_SIZE.key(), String.valueOf(ParquetWriter.DEFAULT_BLOCK_SIZE)); + config.setValue(PARQUET_PAGE_SIZE.key(), String.valueOf(ParquetWriter.DEFAULT_PAGE_SIZE)); + config.setValue(PARQUET_MAX_FILE_SIZE.key(), String.valueOf(1024 * 1024 * 1024)); + config.setValue(PARQUET_COMPRESSION_RATIO_FRACTION.key(), String.valueOf(expectedCompressionRatio.get())); + config.setValue(PARQUET_DICTIONARY_ENABLED, String.valueOf(useDictionaryEncoding.get())); + HoodieRecordType recordType = records.iterator().next().getRecordType(); + try { + parquetWriter = HoodieFileWriterFactory.getFileWriter( + HoodieFileFormat.PARQUET, + outputStream, + new Configuration(), + config, + writerSchema, + recordType); + for (HoodieRecord record : records) { String recordKey = getRecordKey(record).orElse(null); - parquetWriter.writeAvro(recordKey, record); + parquetWriter.write(recordKey, record, writerSchema); } outputStream.flush(); + } finally { + if (parquetWriter != null) { + parquetWriter.close(); + } } } return baos.toByteArray(); } - public static ClosableIterator getProjectedParquetRecordsIterator(Configuration conf, - Schema readerSchema, - InputFile inputFile) throws IOException { - AvroReadSupport.setAvroReadSchema(conf, readerSchema); - AvroReadSupport.setRequestedProjection(conf, readerSchema); - - ParquetReader reader = - AvroParquetReader.builder(inputFile).withConf(conf).build(); - return new ParquetReaderIterator<>(reader); - } - /** * NOTE: We're overriding the whole reading sequence to make sure we properly respect * the requested Reader's schema and only fetch the columns that have been explicitly * requested by the caller (providing projected Reader's schema) */ @Override - protected ClosableIterator readRecordsFromBlockPayload() throws IOException { + protected ClosableIterator> readRecordsFromBlockPayload(HoodieRecordType type) throws IOException { HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get(); // NOTE: It's important to extend Hadoop configuration here to make sure configuration @@ -159,14 +156,15 @@ protected ClosableIterator readRecordsFromBlockPayload() throws I blockContentLoc.getContentPositionInLogFile(), blockContentLoc.getBlockSize()); - return getProjectedParquetRecordsIterator( - inlineConf, - readerSchema, - HadoopInputFile.fromPath(inlineLogFilePath, inlineConf)); + Schema writerSchema = new Schema.Parser().parse(this.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); + + ClosableIterator> iterator = HoodieFileReaderFactory.getReaderFactory(type).getFileReader(inlineConf, inlineLogFilePath, PARQUET) + .getRecordIterator(writerSchema, readerSchema); + return iterator; } @Override - protected ClosableIterator deserializeRecords(byte[] content) { + protected ClosableIterator> deserializeRecords(byte[] content, HoodieRecordType type) throws IOException { throw new UnsupportedOperationException("Should not be invoked"); } -} \ No newline at end of file +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index c98920e3ed8f0..0999778fb5392 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -24,6 +24,8 @@ import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; @@ -258,11 +260,12 @@ private List loadInstants(TimeRangeFilter filter, boolean loadIns HoodieAvroDataBlock avroBlock = (HoodieAvroDataBlock) block; // TODO If we can store additional metadata in datablock, we can skip parsing records // (such as startTime, endTime of records in the block) - try (ClosableIterator itr = avroBlock.getRecordIterator()) { + try (ClosableIterator> itr = avroBlock.getRecordIterator(HoodieRecordType.AVRO)) { StreamSupport.stream(Spliterators.spliteratorUnknownSize(itr, Spliterator.IMMUTABLE), true) // Filter blocks in desired time window - .filter(r -> commitsFilter.apply((GenericRecord) r)) - .map(r -> readCommit((GenericRecord) r, loadInstantDetails)) + .map(r -> (GenericRecord) r.getData()) + .filter(commitsFilter::apply) + .map(r -> readCommit(r, loadInstantDetails)) .filter(c -> filter == null || filter.isInRange(c)) .forEach(instantsInRange::add); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineLayout.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineLayout.java index 77d82f913e9f3..d8616fc409726 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineLayout.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineLayout.java @@ -27,7 +27,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; - /** * Timeline Layout responsible for applying specific filters when generating timeline instants. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClosableIteratorWithSchema.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClosableIteratorWithSchema.java new file mode 100644 index 0000000000000..6f7c09a7f6436 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClosableIteratorWithSchema.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import org.apache.avro.Schema; + +public class ClosableIteratorWithSchema implements ClosableIterator { + + private final ClosableIterator iter; + private final Schema schema; + + public ClosableIteratorWithSchema(ClosableIterator iter, Schema schema) { + this.iter = iter; + this.schema = schema; + } + + public static ClosableIteratorWithSchema newInstance(ClosableIterator iter, Schema schema) { + return new ClosableIteratorWithSchema<>(iter, schema); + } + + public Schema getSchema() { + return schema; + } + + @Override + public void close() { + iter.close(); + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public R next() { + return iter.next(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java index 27ced0f353b62..f491adc6cabf0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java @@ -118,7 +118,6 @@ public static T[] append(T[] array, T elem) { return combined; } - /** * Combines provided {@link List}s into one, returning new instance of {@link ArrayList} */ @@ -250,6 +249,15 @@ public static Map createImmutableMap(final Pair... elements) { return Collections.unmodifiableMap(map); } + @SafeVarargs + public static HashMap createHashMap(final Pair... elements) { + HashMap map = new HashMap<>(); + for (Pair pair: elements) { + map.put(pair.getLeft(), pair.getRight()); + } + return map; + } + @SafeVarargs public static Set createImmutableSet(final T... elements) { return Collections.unmodifiableSet(createSet(elements)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java new file mode 100644 index 0000000000000..b801ccc897a46 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import org.apache.hudi.common.model.HoodiePayloadProps; +import org.apache.hudi.common.table.HoodieTableConfig; + +import java.util.Arrays; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; + +public class ConfigUtils { + + /** + * Get ordering field. + */ + public static String getOrderingField(Properties properties) { + String orderField = null; + if (properties.containsKey(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY)) { + orderField = properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY); + } else if (properties.containsKey("hoodie.datasource.write.precombine.field")) { + orderField = properties.getProperty("hoodie.datasource.write.precombine.field"); + } else if (properties.containsKey(HoodieTableConfig.PRECOMBINE_FIELD.key())) { + orderField = properties.getProperty(HoodieTableConfig.PRECOMBINE_FIELD.key()); + } + return orderField; + } + + /** + * Get payload class. + */ + public static String getPayloadClass(Properties properties) { + String payloadClass = null; + if (properties.containsKey(HoodieTableConfig.PAYLOAD_CLASS_NAME.key())) { + payloadClass = properties.getProperty(HoodieTableConfig.PAYLOAD_CLASS_NAME.key()); + } else if (properties.containsKey("hoodie.datasource.write.payload.class")) { + payloadClass = properties.getProperty("hoodie.datasource.write.payload.class"); + } + return payloadClass; + } + + public static List split2List(String param) { + return Arrays.stream(param.split(",")) + .map(String::trim).distinct().collect(Collectors.toList()); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieCommonKryoProvider.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieCommonKryoProvider.java new file mode 100644 index 0000000000000..b25efb632582a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieCommonKryoProvider.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import org.apache.hudi.common.HoodieJsonPayload; +import org.apache.hudi.common.model.AWSDmsAvroPayload; +import org.apache.hudi.common.model.DefaultHoodieRecordPayload; +import org.apache.hudi.common.model.EventTimeAvroPayload; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieEmptyRecord; +import org.apache.hudi.common.model.HoodieRecordGlobalLocation; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.OverwriteNonDefaultsWithLatestAvroPayload; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.common.model.PartialUpdateAvroPayload; +import org.apache.hudi.common.model.RewriteAvroPayload; +import org.apache.hudi.common.model.debezium.MySqlDebeziumAvroPayload; +import org.apache.hudi.common.model.debezium.PostgresDebeziumAvroPayload; +import org.apache.hudi.metadata.HoodieMetadataPayload; + +/** + * NOTE: PLEASE READ CAREFULLY BEFORE CHANGING + * + * This class is responsible for registering Hudi specific components that are often + * serialized by Kryo (for ex, during Spark's Shuffling operations) to make sure Kryo + * doesn't need to serialize their full class-names (for every object) which will quickly + * add up to considerable amount of overhead. + * + * Please note of the following: + *
    + *
  1. Ordering of the registration COULD NOT change as it's directly impacting + * associated class ids (on the Kryo side)
  2. + *
  3. This class might be loaded up using reflection and as such should not be relocated + * or renamed (w/o correspondingly updating such usages)
  4. + *
+ */ +public class HoodieCommonKryoProvider { + + public Class[] registerClasses() { + /////////////////////////////////////////////////////////////////////////// + // NOTE: DO NOT REORDER REGISTRATIONS + /////////////////////////////////////////////////////////////////////////// + + return new Class[] { + HoodieAvroRecord.class, + HoodieAvroIndexedRecord.class, + HoodieEmptyRecord.class, + + OverwriteWithLatestAvroPayload.class, + DefaultHoodieRecordPayload.class, + OverwriteNonDefaultsWithLatestAvroPayload.class, + RewriteAvroPayload.class, + EventTimeAvroPayload.class, + PartialUpdateAvroPayload.class, + MySqlDebeziumAvroPayload.class, + PostgresDebeziumAvroPayload.class, + // TODO need to relocate to hudi-common + //kryo.register(BootstrapRecordPayload.class); + AWSDmsAvroPayload.class, + HoodieAvroPayload.class, + HoodieJsonPayload.class, + HoodieMetadataPayload.class, + + HoodieRecordLocation.class, + HoodieRecordGlobalLocation.class + }; + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java index feebf6c48224a..32955abbf0c44 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java @@ -19,21 +19,18 @@ package org.apache.hudi.common.util; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.Serializable; - /** * Size Estimator for Hoodie record payload. * * @param */ -public class HoodieRecordSizeEstimator implements SizeEstimator>, Serializable { +public class HoodieRecordSizeEstimator implements SizeEstimator> { private static final Logger LOG = LogManager.getLogger(HoodieRecordSizeEstimator.class); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java new file mode 100644 index 0000000000000..666a084877bc6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import java.lang.reflect.InvocationTargetException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * A utility class for HoodieRecord. + */ +public class HoodieRecordUtils { + + private static final Map INSTANCE_CACHE = new HashMap<>(); + private static final Logger LOG = LogManager.getLogger(HoodieRecordUtils.class); + + /** + * Instantiate a given class with a record merge. + */ + public static HoodieRecordMerger loadRecordMerger(String mergerClass) { + try { + HoodieRecordMerger recordMerger = (HoodieRecordMerger) INSTANCE_CACHE.get(mergerClass); + if (null == recordMerger) { + synchronized (HoodieRecordMerger.class) { + recordMerger = (HoodieRecordMerger) INSTANCE_CACHE.get(mergerClass); + if (null == recordMerger) { + recordMerger = (HoodieRecordMerger) ReflectionUtils.loadClass(mergerClass, + new Object[]{}); + INSTANCE_CACHE.put(mergerClass, recordMerger); + } + } + } + return recordMerger; + } catch (HoodieException e) { + throw new HoodieException("Unable to instantiate hoodie merge class ", e); + } + } + + /** + * Instantiate a given class with a record merge. + */ + public static HoodieRecordMerger createRecordMerger(String basePath, EngineType engineType, + List mergerClassList, String mergerStrategy) { + if (mergerClassList.isEmpty() || HoodieTableMetadata.isMetadataTable(basePath)) { + return HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()); + } else { + return mergerClassList.stream() + .map(clazz -> { + try { + return loadRecordMerger(clazz); + } catch (HoodieException e) { + LOG.warn(String.format("Unable to init %s", clazz), e); + return null; + } + }) + .filter(Objects::nonNull) + .filter(merger -> merger.getMergingStrategy().equals(mergerStrategy)) + .filter(merger -> recordTypeCompatibleEngine(merger.getRecordType(), engineType)) + .findFirst() + .orElse(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())); + } + } + + /** + * Instantiate a given class with an avro record payload. + */ + public static T loadPayload(String recordPayloadClass, + Object[] payloadArgs, + Class... constructorArgTypes) { + try { + return (T) ReflectionUtils.getClass(recordPayloadClass).getConstructor(constructorArgTypes) + .newInstance(payloadArgs); + } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + throw new HoodieException("Unable to instantiate payload class ", e); + } + } + + public static boolean recordTypeCompatibleEngine(HoodieRecordType recordType, EngineType engineType) { + if (engineType == EngineType.SPARK && recordType == HoodieRecordType.SPARK) { + return true; + } else { + return false; + } + } +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/IdentityIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/IdentityIterator.java new file mode 100644 index 0000000000000..feafd0762dd64 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/IdentityIterator.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import java.util.Iterator; + +public class IdentityIterator implements ClosableIterator { + + private Iterator itr; + + public IdentityIterator(Iterator itr) { + this.itr = itr; + } + + @Override + public void close() { + } + + @Override + public boolean hasNext() { + return itr.hasNext(); + } + + @Override + public R next() { + return itr.next(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MappingIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MappingIterator.java new file mode 100644 index 0000000000000..3ab80d9634fae --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MappingIterator.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import java.util.function.Function; + +public class MappingIterator implements ClosableIterator { + + private final ClosableIterator sourceIterator; + private final Function mapper; + + public MappingIterator(ClosableIterator sourceIterator, Function mapper) { + this.sourceIterator = sourceIterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return sourceIterator.hasNext(); + } + + @Override + public R next() { + return mapper.apply(sourceIterator.next()); + } + + @Override + public void close() { + sourceIterator.close(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java index 03bd471b606f1..f681bdfe844d3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java @@ -32,11 +32,11 @@ public class ParquetReaderIterator implements ClosableIterator { // Parquet reader for an existing parquet file - private final ParquetReader parquetReader; + private final ParquetReader parquetReader; // Holds the next entry returned by the parquet reader private T next; - public ParquetReaderIterator(ParquetReader parquetReader) { + public ParquetReaderIterator(ParquetReader parquetReader) { this.parquetReader = parquetReader; } @@ -64,7 +64,7 @@ public T next() { } } T retVal = this.next; - this.next = parquetReader.read(); + this.next = null; return retVal; } catch (Exception e) { FileIOUtils.closeQuietly(parquetReader); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java index 6ee7928c759da..b3e178320b829 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java @@ -18,7 +18,6 @@ package org.apache.hudi.common.util; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; @@ -69,18 +68,6 @@ public static T loadClass(String className) { } } - /** - * Instantiate a given class with a generic record payload. - */ - public static T loadPayload(String recordPayloadClass, Object[] payloadArgs, - Class... constructorArgTypes) { - try { - return (T) getClass(recordPayloadClass).getConstructor(constructorArgTypes).newInstance(payloadArgs); - } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { - throw new HoodieException("Unable to instantiate payload class ", e); - } - } - /** * Creates an instance of the given class. Use this version when dealing with interface types as constructor args. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java index 872848a5d4979..5962cd6e3c734 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java @@ -28,6 +28,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.Serializable; +import java.util.Arrays; /** * {@link SerializationUtils} class internally uses {@link Kryo} serializer for serializing / deserializing objects. @@ -117,6 +118,10 @@ public Kryo newKryo() { // for hadoop kryo.setClassLoader(Thread.currentThread().getContextClassLoader()); + // Register Hudi's classes + Arrays.stream(new HoodieCommonKryoProvider().registerClasses()) + .forEach(kryo::register); + // Register serializers kryo.register(Utf8.class, new AvroUtf8Serializer()); @@ -129,7 +134,7 @@ public Kryo newKryo() { * NOTE: This {@link Serializer} could deserialize instance of {@link Utf8} serialized * by implicitly generated Kryo serializer (based on {@link com.esotericsoftware.kryo.serializers.FieldSerializer} */ - private static class AvroUtf8Serializer extends Serializer { + public static class AvroUtf8Serializer extends Serializer { @SuppressWarnings("unchecked") @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java index d4bafd9c9feee..d8cf85cfd4533 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java @@ -106,13 +106,13 @@ public static long computePayloadSize(R value, SizeEstimator valueSizeEst /** * Utility method to convert bytes to HoodieRecord using schema and payload class. */ - public static R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz, String preCombineField, boolean withOperationField) { + public static HoodieRecord convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz, String preCombineField, boolean withOperationField) { return convertToHoodieRecordPayload(rec, payloadClazz, preCombineField, Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), withOperationField, Option.empty()); } - public static R convertToHoodieRecordPayload(GenericRecord record, String payloadClazz, + public static HoodieRecord convertToHoodieRecordPayload(GenericRecord record, String payloadClazz, String preCombineField, boolean withOperationField, Option partitionName) { @@ -124,7 +124,7 @@ public static R convertToHoodieRecordPayload(GenericRecord record, String pa /** * Utility method to convert bytes to HoodieRecord using schema and payload class. */ - public static R convertToHoodieRecordPayload(GenericRecord record, String payloadClazz, + public static HoodieRecord convertToHoodieRecordPayload(GenericRecord record, String payloadClazz, String preCombineField, Pair recordKeyPartitionPathFieldPair, boolean withOperationField, @@ -137,10 +137,10 @@ public static R convertToHoodieRecordPayload(GenericRecord record, String pa HoodieOperation operation = withOperationField ? HoodieOperation.fromName(getNullableValAsString(record, HoodieRecord.OPERATION_METADATA_FIELD)) : null; HoodieRecord hoodieRecord = new HoodieAvroRecord<>(new HoodieKey(recKey, partitionPath), - ReflectionUtils.loadPayload(payloadClazz, new Object[]{record, preCombineVal}, GenericRecord.class, + HoodieRecordUtils.loadPayload(payloadClazz, new Object[]{record, preCombineVal}, GenericRecord.class, Comparable.class), operation); - return (R) hoodieRecord; + return (HoodieRecord) hoodieRecord; } /** @@ -163,7 +163,7 @@ private static Object getPreCombineVal(GenericRecord rec, String preCombineField */ public static R generateEmptyPayload(String recKey, String partitionPath, Comparable orderingVal, String payloadClazz) { HoodieRecord hoodieRecord = new HoodieAvroRecord<>(new HoodieKey(recKey, partitionPath), - ReflectionUtils.loadPayload(payloadClazz, new Object[] {null, orderingVal}, GenericRecord.class, Comparable.class)); + HoodieRecordUtils.loadPayload(payloadClazz, new Object[] {null, orderingVal}, GenericRecord.class, Comparable.class)); return (R) hoodieRecord; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/VisibleForTesting.java b/hudi-common/src/main/java/org/apache/hudi/common/util/VisibleForTesting.java new file mode 100644 index 0000000000000..c6e4e488f0959 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/VisibleForTesting.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +/** + * Annotation designating a field or a method as visible for the testing purposes + */ +public @interface VisibleForTesting { +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/FlatLists.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/FlatLists.java new file mode 100644 index 0000000000000..ece503c0cd2de --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/FlatLists.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util.collection; + +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoSerializable; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; + +/** + * Space-efficient, comparable, immutable lists, copied from calcite core. + */ +public class FlatLists { + private FlatLists() { + } + + /** + * Creates a memory-, CPU- and cache-efficient immutable list from an + * existing list. The list is always copied. + * + * @param t Array of members of list + * @param Element type + * @return List containing the given members + */ + public static List of(List t) { + return of_(t); + } + + public static ComparableList ofComparable(List t) { + return of_(t); + } + + public static ComparableList ofComparableArray(Object[] t) { + return ofComparable(Arrays.stream(t).map(v -> (T)v).collect(Collectors.toList())); + } + + private static ComparableList of_(List t) { + return new ComparableListImpl(new ArrayList<>(t)); + } + + /** List that is also comparable. + * + *

You can create an instance whose type + * parameter {@code T} does not extend {@link Comparable}, but you will get a + * {@link ClassCastException} at runtime when you call + * {@link #compareTo(Object)} if the elements of the list do not implement + * {@code Comparable}. + */ + public interface ComparableList extends List, Comparable { + } + + /** Wrapper around a list that makes it implement the {@link Comparable} + * interface using lexical ordering. The elements must be comparable. */ + static class ComparableListImpl> + extends AbstractList + implements ComparableList, KryoSerializable { + private List list; + + protected ComparableListImpl(List list) { + this.list = list; + } + + public T get(int index) { + return list.get(index); + } + + public int size() { + return list.size(); + } + + public int compareTo(List o) { + return compare(list, o); + } + + static > int compare(List list0, List list1) { + final int size0 = list0.size(); + final int size1 = list1.size(); + if (size1 == size0) { + return compare(list0, list1, size0); + } + final int c = compare(list0, list1, Math.min(size0, size1)); + if (c != 0) { + return c; + } + return size0 - size1; + } + + static > int compare(List list0, List list1, int size) { + for (int i = 0; i < size; i++) { + Comparable o0 = list0.get(i); + Comparable o1 = list1.get(i); + int c = compare(o0, o1); + if (c != 0) { + return c; + } + } + return 0; + } + + static > int compare(T a, T b) { + if (a == b) { + return 0; + } + if (a == null) { + return -1; + } + if (b == null) { + return 1; + } + return a.compareTo(b); + } + + @Override + public void write(Kryo kryo, Output output) { + kryo.writeClassAndObject(output, list); + } + + @Override + public void read(Kryo kryo, Input input) { + list = (List) kryo.readClassAndObject(input); + } + } + +} + + diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieException.java index 9bf01b34cdb50..c13b03df1c45b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieException.java +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieException.java @@ -18,7 +18,6 @@ package org.apache.hudi.exception; - /** *

* Exception thrown for Hoodie failures. The root of the exception hierarchy. diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchemaBuilder.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchemaBuilder.java index 5fc86ef723958..ff54116a90c24 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchemaBuilder.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchemaBuilder.java @@ -43,7 +43,6 @@ public static InternalSchemaBuilder getBuilder() { private InternalSchemaBuilder() { } - /** * Build a mapping from id to full field name for a internal Type. * if a field y belong to a struct filed x, then the full name of y is x.y diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReader.java new file mode 100644 index 0000000000000..a829880d5f948 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReader.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.avro.generic.IndexedRecord; + +/** + * Marker interface for every {@link HoodieFileReader} reading in Avro (ie + * producing {@link IndexedRecord}s) + */ +public interface HoodieAvroFileReader extends HoodieFileReader {} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderBase.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderBase.java new file mode 100644 index 0000000000000..5cee50449a443 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderBase.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.MappingIterator; + +import java.io.IOException; + +import static org.apache.hudi.common.util.TypeUtils.unsafeCast; + +/** + * Base class for every {@link HoodieAvroFileReader} + */ +abstract class HoodieAvroFileReaderBase implements HoodieAvroFileReader { + + @Override + public ClosableIterator> getRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { + ClosableIterator iterator = getIndexedRecordIterator(readerSchema, requestedSchema); + return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieAvroIndexedRecord(data))); + } + + protected ClosableIterator getIndexedRecordIterator(Schema readerSchema) throws IOException { + return getIndexedRecordIterator(readerSchema, readerSchema); + } + + protected abstract ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException; +} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java new file mode 100644 index 0000000000000..3834574cd04ca --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; + +import java.io.IOException; + +public class HoodieAvroFileReaderFactory extends HoodieFileReaderFactory { + + protected HoodieFileReader newParquetFileReader(Configuration conf, Path path) { + return new HoodieAvroParquetReader(conf, path); + } + + protected HoodieFileReader newHFileFileReader(Configuration conf, Path path) throws IOException { + CacheConfig cacheConfig = new CacheConfig(conf); + return new HoodieAvroHFileReader(conf, path, cacheConfig); + } + + @Override + protected HoodieFileReader newOrcFileReader(Configuration conf, Path path) { + return new HoodieAvroOrcReader(conf, path); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriter.java similarity index 60% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriter.java index cce59d3b6624a..e53a7b095ff2b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriter.java @@ -18,28 +18,42 @@ package org.apache.hudi.io.storage; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import java.io.IOException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; -public interface HoodieFileWriter { +import java.io.IOException; +import java.util.Properties; - void writeAvroWithMetadata(HoodieKey key, R newRecord) throws IOException; +public interface HoodieAvroFileWriter extends HoodieFileWriter { boolean canWrite(); void close() throws IOException; - void writeAvro(String key, R oldRecord) throws IOException; + void writeAvroWithMetadata(HoodieKey key, IndexedRecord avroRecord) throws IOException; + + void writeAvro(String recordKey, IndexedRecord record) throws IOException; + + @Override + default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException { + IndexedRecord avroPayload = record.toIndexedRecord(schema, props).get().getData(); + writeAvroWithMetadata(key, avroPayload); + } + + @Override + default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException { + IndexedRecord avroPayload = record.toIndexedRecord(schema, props).get().getData(); + writeAvro(recordKey, avroPayload); + } - default void prepRecordWithMetadata(HoodieKey key, R avroRecord, String instantTime, Integer partitionId, long recordIndex, String fileName) { + default void prepRecordWithMetadata(HoodieKey key, IndexedRecord avroRecord, String instantTime, Integer partitionId, long recordIndex, String fileName) { String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex); HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, key.getRecordKey(), key.getPartitionPath(), fileName); HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, instantTime, seqId); - return; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java new file mode 100644 index 0000000000000..8c68a2fde7b59 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.avro.HoodieAvroWriteSupport; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.orc.CompressionKind; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; + +import static org.apache.hudi.io.storage.HoodieHFileConfig.CACHE_DATA_IN_L1; +import static org.apache.hudi.io.storage.HoodieHFileConfig.DROP_BEHIND_CACHE_COMPACTION; +import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR; +import static org.apache.hudi.io.storage.HoodieHFileConfig.PREFETCH_ON_OPEN; + +public class HoodieAvroFileWriterFactory extends HoodieFileWriterFactory { + + protected HoodieFileWriter newParquetFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS); + boolean enableBloomFilter = populateMetaFields; + Option filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(conf).convert(schema), schema, filter); + String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); + // Support PARQUET_COMPRESSION_CODEC_NAME is "" + if (compressionCodecName.isEmpty()) { + compressionCodecName = null; + } + HoodieParquetConfig parquetConfig = new HoodieParquetConfig(writeSupport, + CompressionCodecName.fromConf(compressionCodecName), + config.getIntOrDefault(HoodieStorageConfig.PARQUET_BLOCK_SIZE), + config.getIntOrDefault(HoodieStorageConfig.PARQUET_PAGE_SIZE), + config.getLongOrDefault(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE), + conf, config.getDoubleOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION), + config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED)); + return new HoodieAvroParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, populateMetaFields); + } + + protected HoodieFileWriter newParquetFileWriter( + FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException { + boolean enableBloomFilter = false; + Option filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(conf).convert(schema), schema, filter); + HoodieParquetConfig parquetConfig = new HoodieParquetConfig<>(writeSupport, + CompressionCodecName.fromConf(config.getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME)), + config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE), + config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE), + config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE), // todo: 1024*1024*1024 + conf, config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION), + config.getBoolean(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED)); + return new HoodieParquetStreamWriter(outputStream, parquetConfig); + } + + protected HoodieFileWriter newHFileFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + BloomFilter filter = createBloomFilter(config); + HoodieHFileConfig hfileConfig = new HoodieHFileConfig(conf, + Compression.Algorithm.valueOf(config.getString(HoodieStorageConfig.HFILE_COMPRESSION_ALGORITHM_NAME)), + config.getInt(HoodieStorageConfig.HFILE_BLOCK_SIZE), + config.getLong(HoodieStorageConfig.HFILE_MAX_FILE_SIZE), + HoodieAvroHFileReader.KEY_FIELD_NAME, PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, + filter, HFILE_COMPARATOR); + + return new HoodieAvroHFileWriter(instantTime, path, hfileConfig, schema, taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)); + } + + protected HoodieFileWriter newOrcFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + BloomFilter filter = createBloomFilter(config); + HoodieOrcConfig orcConfig = new HoodieOrcConfig(conf, + CompressionKind.valueOf(config.getString(HoodieStorageConfig.ORC_COMPRESSION_CODEC_NAME)), + config.getInt(HoodieStorageConfig.ORC_STRIPE_SIZE), + config.getInt(HoodieStorageConfig.ORC_BLOCK_SIZE), + config.getLong(HoodieStorageConfig.ORC_FILE_MAX_SIZE), filter); + return new HoodieAvroOrcWriter(instantTime, path, orcConfig, schema, taskContextSupplier); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java similarity index 82% rename from hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java index 997d625aa2778..f9a72878b6226 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java @@ -22,8 +22,12 @@ import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.io.ByteBufferBackedInputStream; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -58,6 +62,7 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.util.CollectionUtils.toStream; +import static org.apache.hudi.common.util.TypeUtils.unsafeCast; import static org.apache.hudi.common.util.ValidationUtils.checkState; /** @@ -65,7 +70,7 @@ *

* {@link HoodieFileReader} implementation allowing to read from {@link HFile}. */ -public class HoodieHFileReader implements HoodieFileReader { +public class HoodieAvroHFileReader extends HoodieAvroFileReaderBase implements HoodieSeekingFileReader { // TODO HoodieHFileReader right now tightly coupled to MT, we should break that coupling public static final String SCHEMA_KEY = "schema"; @@ -76,7 +81,7 @@ public class HoodieHFileReader implements HoodieFileRea public static final String KEY_MIN_RECORD = "minRecordKey"; public static final String KEY_MAX_RECORD = "maxRecordKey"; - private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class); + private static final Logger LOG = LogManager.getLogger(HoodieAvroHFileReader.class); private final Path path; @@ -91,21 +96,21 @@ public class HoodieHFileReader implements HoodieFileRea private final Object sharedScannerLock = new Object(); - public HoodieHFileReader(Configuration hadoopConf, Path path, CacheConfig cacheConfig) throws IOException { + public HoodieAvroHFileReader(Configuration hadoopConf, Path path, CacheConfig cacheConfig) throws IOException { this(path, HoodieHFileUtils.createHFileReader(FSUtils.getFs(path.toString(), hadoopConf), path, cacheConfig, hadoopConf), Option.empty()); } - public HoodieHFileReader(Configuration hadoopConf, Path path, CacheConfig cacheConfig, FileSystem fs) throws IOException { + public HoodieAvroHFileReader(Configuration hadoopConf, Path path, CacheConfig cacheConfig, FileSystem fs) throws IOException { this(path, HoodieHFileUtils.createHFileReader(fs, path, cacheConfig, hadoopConf), Option.empty()); } - public HoodieHFileReader(FileSystem fs, Path dummyPath, byte[] content, Option schemaOpt) throws IOException { + public HoodieAvroHFileReader(FileSystem fs, Path dummyPath, byte[] content, Option schemaOpt) throws IOException { this(null, HoodieHFileUtils.createHFileReader(fs, dummyPath, content), schemaOpt); } - public HoodieHFileReader(Path path, HFile.Reader reader, Option schemaOpt) throws IOException { + public HoodieAvroHFileReader(Path path, HFile.Reader reader, Option schemaOpt) throws IOException { this.path = path; this.reader = reader; // For shared scanner, which is primarily used for point-lookups, we're caching blocks @@ -115,6 +120,30 @@ public HoodieHFileReader(Path path, HFile.Reader reader, Option schemaOp .orElseGet(() -> Lazy.lazily(() -> fetchSchema(reader))); } + @Override + public Option> getRecordByKey(String key, Schema readerSchema) throws IOException { + synchronized (sharedScannerLock) { + return fetchRecordByKeyInternal(sharedScanner, key, getSchema(), readerSchema) + .map(data -> unsafeCast(new HoodieAvroIndexedRecord(data))); + } + } + + @Override + public ClosableIterator> getRecordsByKeysIterator(List keys, Schema schema) throws IOException { + // We're caching blocks for this scanner to minimize amount of traffic + // to the underlying storage as we fetched (potentially) sparsely distributed + // keys + HFileScanner scanner = getHFileScanner(reader, true); + ClosableIterator iterator = new RecordByKeyIterator(scanner, keys, getSchema(), schema); + return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieAvroIndexedRecord(data))); + } + + @Override + public ClosableIterator> getRecordsByKeyPrefixIterator(List keyPrefixes, Schema schema) throws IOException { + ClosableIterator iterator = getIndexedRecordsByKeyPrefixIterator(keyPrefixes, schema); + return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieAvroIndexedRecord(data))); + } + @Override public String[] readMinMaxRecordKeys() { // NOTE: This access to reader is thread-safe @@ -170,40 +199,33 @@ public Set filterRowKeys(Set candidateRowKeys) { } } - @SuppressWarnings("unchecked") @Override - public Option getRecordByKey(String key, Schema readerSchema) throws IOException { - synchronized (sharedScannerLock) { - return (Option) fetchRecordByKeyInternal(sharedScanner, key, getSchema(), readerSchema); + protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) { + if (!Objects.equals(readerSchema, requestedSchema)) { + throw new UnsupportedOperationException("Schema projections are not supported in HFile reader"); } - } - @SuppressWarnings("unchecked") - @Override - public ClosableIterator getRecordIterator(Schema readerSchema) throws IOException { // TODO eval whether seeking scanner would be faster than pread HFileScanner scanner = getHFileScanner(reader, false); - return (ClosableIterator) new RecordIterator(scanner, getSchema(), readerSchema); + return new RecordIterator(scanner, getSchema(), readerSchema); } - @SuppressWarnings("unchecked") - @Override - public ClosableIterator getRecordsByKeysIterator(List keys, Schema readerSchema) throws IOException { + @VisibleForTesting + protected ClosableIterator getIndexedRecordsByKeysIterator(List keys, Schema readerSchema) throws IOException { // We're caching blocks for this scanner to minimize amount of traffic // to the underlying storage as we fetched (potentially) sparsely distributed // keys HFileScanner scanner = getHFileScanner(reader, true); - return (ClosableIterator) new RecordByKeyIterator(scanner, keys, getSchema(), readerSchema); + return new RecordByKeyIterator(scanner, keys, getSchema(), readerSchema); } - @SuppressWarnings("unchecked") - @Override - public ClosableIterator getRecordsByKeyPrefixIterator(List keyPrefixes, Schema readerSchema) throws IOException { + @VisibleForTesting + protected ClosableIterator getIndexedRecordsByKeyPrefixIterator(List keyPrefixes, Schema readerSchema) throws IOException { // We're caching blocks for this scanner to minimize amount of traffic // to the underlying storage as we fetched (potentially) sparsely distributed // keys HFileScanner scanner = getHFileScanner(reader, true); - return (ClosableIterator) new RecordByKeyPrefixIterator(scanner, keyPrefixes, getSchema(), readerSchema); + return new RecordByKeyPrefixIterator(scanner, keyPrefixes, getSchema(), readerSchema); } @Override @@ -228,7 +250,7 @@ private boolean isKeyAvailable(String key, HFileScanner keyScanner) throws IOExc return keyScanner.seekTo(kv) == 0; } - private static Iterator getRecordByKeyPrefixIteratorInternal(HFileScanner scanner, + private static Iterator getRecordByKeyPrefixIteratorInternal(HFileScanner scanner, String keyPrefix, Schema writerSchema, Schema readerSchema) throws IOException { @@ -265,8 +287,8 @@ private static Iterator getRecordByKeyPrefixIteratorInternal(HFil scanner.seekTo(); } - class KeyPrefixIterator implements Iterator { - private GenericRecord next = null; + class KeyPrefixIterator implements Iterator { + private IndexedRecord next = null; private boolean eof = false; @Override @@ -299,8 +321,8 @@ public boolean hasNext() { } @Override - public GenericRecord next() { - GenericRecord next = this.next; + public IndexedRecord next() { + IndexedRecord next = this.next; this.next = null; return next; } @@ -309,7 +331,7 @@ public GenericRecord next() { return new KeyPrefixIterator(); } - private static Option fetchRecordByKeyInternal(HFileScanner scanner, String key, Schema writerSchema, Schema readerSchema) throws IOException { + private static Option fetchRecordByKeyInternal(HFileScanner scanner, String key, Schema writerSchema, Schema readerSchema) throws IOException { KeyValue kv = new KeyValue(key.getBytes(), null, null, null); if (scanner.seekTo(kv) != 0) { return Option.empty(); @@ -373,9 +395,9 @@ private static byte[] copyValueFromCell(Cell c) { *

* Reads all the records with given schema */ - public static List readAllRecords(HoodieHFileReader reader) throws IOException { + public static List readAllRecords(HoodieAvroHFileReader reader) throws IOException { Schema schema = reader.getSchema(); - return toStream(reader.getRecordIterator(schema)) + return toStream(reader.getIndexedRecordIterator(schema)) .collect(Collectors.toList()); } @@ -384,7 +406,7 @@ public static List readAllRecords(HoodieHFileReader *

* Reads all the records with given schema and filtering keys. */ - public static List readRecords(HoodieHFileReader reader, + public static List readRecords(HoodieAvroHFileReader reader, List keys) throws IOException { return readRecords(reader, keys, reader.getSchema()); } @@ -394,11 +416,11 @@ public static List readRecords(HoodieHFileReader *

* Reads all the records with given schema and filtering keys. */ - public static List readRecords(HoodieHFileReader reader, + public static List readRecords(HoodieAvroHFileReader reader, List keys, Schema schema) throws IOException { Collections.sort(keys); - return toStream(reader.getRecordsByKeysIterator(keys, schema)) + return toStream(reader.getIndexedRecordsByKeysIterator(keys, schema)) .collect(Collectors.toList()); } @@ -412,16 +434,16 @@ private static Option getKeySchema(Schema schema) { return Option.ofNullable(schema.getField(KEY_FIELD_NAME)); } - private static class RecordByKeyPrefixIterator implements ClosableIterator { + private static class RecordByKeyPrefixIterator implements ClosableIterator { private final Iterator keyPrefixesIterator; - private Iterator recordsIterator; + private Iterator recordsIterator; private final HFileScanner scanner; private final Schema writerSchema; private final Schema readerSchema; - private GenericRecord next = null; + private IndexedRecord next = null; RecordByKeyPrefixIterator(HFileScanner scanner, List keyPrefixes, Schema writerSchema, Schema readerSchema) throws IOException { this.keyPrefixesIterator = keyPrefixes.iterator(); @@ -457,8 +479,8 @@ public boolean hasNext() { } @Override - public GenericRecord next() { - GenericRecord next = this.next; + public IndexedRecord next() { + IndexedRecord next = this.next; this.next = null; return next; } @@ -469,7 +491,7 @@ public void close() { } } - private static class RecordByKeyIterator implements ClosableIterator { + private static class RecordByKeyIterator implements ClosableIterator { private final Iterator keyIterator; private final HFileScanner scanner; @@ -477,7 +499,7 @@ private static class RecordByKeyIterator implements ClosableIterator keys, Schema writerSchema, Schema readerSchema) throws IOException { this.keyIterator = keys.iterator(); @@ -498,7 +520,7 @@ public boolean hasNext() { } while (keyIterator.hasNext()) { - Option value = fetchRecordByKeyInternal(scanner, keyIterator.next(), writerSchema, readerSchema); + Option value = fetchRecordByKeyInternal(scanner, keyIterator.next(), writerSchema, readerSchema); if (value.isPresent()) { next = value.get(); return true; @@ -511,8 +533,8 @@ public boolean hasNext() { } @Override - public GenericRecord next() { - GenericRecord next = this.next; + public IndexedRecord next() { + IndexedRecord next = this.next; this.next = null; return next; } @@ -523,13 +545,13 @@ public void close() { } } - private static class RecordIterator implements ClosableIterator { + private static class RecordIterator implements ClosableIterator { private final HFileScanner scanner; private final Schema writerSchema; private final Schema readerSchema; - private GenericRecord next = null; + private IndexedRecord next = null; RecordIterator(HFileScanner scanner, Schema writerSchema, Schema readerSchema) { this.scanner = scanner; @@ -564,8 +586,8 @@ public boolean hasNext() { } @Override - public GenericRecord next() { - GenericRecord next = this.next; + public IndexedRecord next() { + IndexedRecord next = this.next; this.next = null; return next; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java similarity index 87% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java index f065608b29bd5..c143f782d4839 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java @@ -36,7 +36,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -53,8 +52,8 @@ * 1. Records should be added in order of keys * 2. There are no column stats */ -public class HoodieHFileWriter - implements HoodieFileWriter { +public class HoodieAvroHFileWriter + implements HoodieAvroFileWriter { private static AtomicLong recordIndex = new AtomicLong(1); private final Path file; @@ -73,8 +72,8 @@ public class HoodieHFileWriter Record implementation that permits field access by integer index. */ -public class HoodieOrcReader implements HoodieFileReader { - private Path path; - private Configuration conf; +public class HoodieAvroOrcReader extends HoodieAvroFileReaderBase { + + private final Path path; + private final Configuration conf; private final BaseFileUtils orcUtils; - public HoodieOrcReader(Configuration configuration, Path path) { + public HoodieAvroOrcReader(Configuration configuration, Path path) { this.conf = configuration; this.path = path; this.orcUtils = BaseFileUtils.getInstance(HoodieFileFormat.ORC); @@ -71,12 +73,16 @@ public Set filterRowKeys(Set candidateRowKeys) { } @Override - public ClosableIterator getRecordIterator(Schema schema) throws IOException { + protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { + if (!Objects.equals(readerSchema, requestedSchema)) { + throw new UnsupportedOperationException("Schema projections are not supported in HFile reader"); + } + try { Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf)); - TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(schema); + TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(readerSchema); RecordReader recordReader = reader.rows(new Options(conf).schema(orcSchema)); - return new OrcReaderIterator<>(recordReader, schema, orcSchema); + return new OrcReaderIterator<>(recordReader, readerSchema, orcSchema); } catch (IOException io) { throw new HoodieIOException("Unable to create an ORC reader.", io); } @@ -88,8 +94,7 @@ public Schema getSchema() { } @Override - public void close() { - } + public void close() {} @Override public long getTotalRecords() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcWriter.java similarity index 93% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcWriter.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcWriter.java index 0923d151cb1dd..f0c796ff6c6b7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcWriter.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.AvroOrcUtils; import org.apache.orc.OrcFile; import org.apache.orc.TypeDescription; @@ -46,8 +45,7 @@ import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY; -public class HoodieOrcWriter - implements HoodieFileWriter, Closeable { +public class HoodieAvroOrcWriter implements HoodieAvroFileWriter, Closeable { private static final AtomicLong RECORD_INDEX = new AtomicLong(1); private final long maxFileSize; @@ -66,8 +64,8 @@ public class HoodieOrcWriter Record implementation that permits field access by integer index. */ -public class HoodieParquetReader implements HoodieFileReader { +public class HoodieAvroParquetReader extends HoodieAvroFileReaderBase { private final Path path; private final Configuration conf; private final BaseFileUtils parquetUtils; private final List readerIterators = new ArrayList<>(); - public HoodieParquetReader(Configuration configuration, Path path) { + public HoodieAvroParquetReader(Configuration configuration, Path path) { // We have to clone the Hadoop Config as it might be subsequently modified // by the Reader (for proper config propagation to Parquet components) this.conf = tryOverrideDefaultConfigs(new Configuration(configuration)); @@ -59,6 +65,15 @@ public HoodieParquetReader(Configuration configuration, Path path) { this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); } + @Override + public ClosableIterator> getRecordIterator(Schema readerSchema) throws IOException { + // TODO(HUDI-4588) remove after HUDI-4588 is resolved + // NOTE: This is a workaround to avoid leveraging projection w/in [[AvroParquetReader]], + // until schema handling issues (nullability canonicalization, etc) are resolved + ClosableIterator iterator = getIndexedRecordIterator(readerSchema); + return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieAvroIndexedRecord(data))); + } + @Override public String[] readMinMaxRecordKeys() { return parquetUtils.readMinMaxRecordKeys(conf, path); @@ -75,17 +90,13 @@ public Set filterRowKeys(Set candidateRowKeys) { } @Override - public ClosableIterator getRecordIterator(Schema schema) throws IOException { - // NOTE: We have to set both Avro read-schema and projection schema to make - // sure that in case the file-schema is not equal to read-schema we'd still - // be able to read that file (in case projection is a proper one) - AvroReadSupport.setAvroReadSchema(conf, schema); - AvroReadSupport.setRequestedProjection(conf, schema); + protected ClosableIterator getIndexedRecordIterator(Schema schema) throws IOException { + return getIndexedRecordIteratorInternal(schema, Option.empty()); + } - ParquetReader reader = AvroParquetReader.builder(path).withConf(conf).build(); - ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader); - readerIterators.add(parquetReaderIterator); - return parquetReaderIterator; + @Override + protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { + return getIndexedRecordIteratorInternal(readerSchema, Option.of(requestedSchema)); } @Override @@ -142,4 +153,21 @@ private static Configuration tryOverrideDefaultConfigs(Configuration conf) { return conf; } + + private ClosableIterator getIndexedRecordIteratorInternal(Schema schema, Option requestedSchema) throws IOException { + // NOTE: We have to set both Avro read-schema and projection schema to make + // sure that in case the file-schema is not equal to read-schema we'd still + // be able to read that file (in case projection is a proper one) + if (!requestedSchema.isPresent()) { + AvroReadSupport.setAvroReadSchema(conf, schema); + AvroReadSupport.setRequestedProjection(conf, schema); + } else { + AvroReadSupport.setAvroReadSchema(conf, requestedSchema.get()); + AvroReadSupport.setRequestedProjection(conf, requestedSchema.get()); + } + ParquetReader reader = AvroParquetReader.builder(path).withConf(conf).build(); + ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader); + readerIterators.add(parquetReaderIterator); + return parquetReaderIterator; + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java similarity index 94% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java index 06631dc53fb1c..36033d26b06cd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java @@ -35,9 +35,9 @@ * ATTENTION: HoodieParquetWriter is not thread safe and developer should take care of the order of write and close */ @NotThreadSafe -public class HoodieAvroParquetWriter +public class HoodieAvroParquetWriter extends HoodieBaseParquetWriter - implements HoodieFileWriter { + implements HoodieAvroFileWriter { private final String fileName; private final String instantTime; @@ -60,7 +60,7 @@ public HoodieAvroParquetWriter(Path file, } @Override - public void writeAvroWithMetadata(HoodieKey key, R avroRecord) throws IOException { + public void writeAvroWithMetadata(HoodieKey key, IndexedRecord avroRecord) throws IOException { if (populateMetaFields) { prepRecordWithMetadata(key, avroRecord, instantTime, taskContextSupplier.getPartitionIdSupplier().get(), getWrittenRecordCount(), fileName); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java similarity index 100% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java index 36fafe3e9c8dd..30dbc5b94ea57 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java @@ -18,24 +18,28 @@ package org.apache.hudi.io.storage; +import org.apache.avro.Schema; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.ClosableIterator; -import org.apache.hudi.common.util.Option; - -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.List; import java.util.Set; /** - * Hudi base file reader interface. + * Hudi's File Reader interface providing common set of APIs to fetch + * + *

    + *
  • {@link HoodieRecord}s
  • + *
  • Metadata (statistics, bloom-filters, etc)
  • + *
+ * + * from a file persisted in storage. * - * @param Record implementation that permits field access by integer index. + * @param target engine-specific representation of the raw data ({@code IndexedRecord} for Avro, + * {@code InternalRow} for Spark, etc) */ -public interface HoodieFileReader extends AutoCloseable { +public interface HoodieFileReader extends AutoCloseable { String[] readMinMaxRecordKeys(); @@ -43,34 +47,14 @@ public interface HoodieFileReader extends AutoCloseable Set filterRowKeys(Set candidateRowKeys); - ClosableIterator getRecordIterator(Schema readerSchema) throws IOException; + ClosableIterator> getRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException; - default ClosableIterator getRecordIterator() throws IOException { - return getRecordIterator(getSchema()); - } - - default Option getRecordByKey(String key, Schema readerSchema) throws IOException { - throw new UnsupportedOperationException(); + default ClosableIterator> getRecordIterator(Schema readerSchema) throws IOException { + return getRecordIterator(readerSchema, readerSchema); } - default Option getRecordByKey(String key) throws IOException { - return getRecordByKey(key, getSchema()); - } - - default ClosableIterator getRecordsByKeysIterator(List keys, Schema schema) throws IOException { - throw new UnsupportedOperationException(); - } - - default ClosableIterator getRecordsByKeysIterator(List keys) throws IOException { - return getRecordsByKeysIterator(keys, getSchema()); - } - - default ClosableIterator getRecordsByKeyPrefixIterator(List keyPrefixes, Schema schema) throws IOException { - throw new UnsupportedEncodingException(); - } - - default ClosableIterator getRecordsByKeyPrefixIterator(List keyPrefixes) throws IOException { - return getRecordsByKeyPrefixIterator(keyPrefixes, getSchema()); + default ClosableIterator> getRecordIterator() throws IOException { + return getRecordIterator(getSchema()); } Schema getSchema(); diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java index 3542956df3d28..36b3f3a57a31a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java @@ -18,12 +18,14 @@ package org.apache.hudi.io.storage; -import org.apache.hudi.common.fs.FSUtils; - -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.exception.HoodieException; import java.io.IOException; @@ -36,7 +38,23 @@ */ public class HoodieFileReaderFactory { - public static HoodieFileReader getFileReader(Configuration conf, Path path) throws IOException { + public static HoodieFileReaderFactory getReaderFactory(HoodieRecord.HoodieRecordType recordType) { + switch (recordType) { + case AVRO: + return new HoodieAvroFileReaderFactory(); + case SPARK: + try { + Class clazz = ReflectionUtils.getClass("org.apache.hudi.io.storage.HoodieSparkFileReaderFactory"); + return (HoodieFileReaderFactory) clazz.newInstance(); + } catch (IllegalArgumentException | IllegalAccessException | InstantiationException e) { + throw new HoodieException("Unable to create hoodie spark file writer factory", e); + } + default: + throw new UnsupportedOperationException(recordType + " record type not supported yet."); + } + } + + public HoodieFileReader getFileReader(Configuration conf, Path path) throws IOException { final String extension = FSUtils.getFileExtension(path.toString()); if (PARQUET.getFileExtension().equals(extension)) { return newParquetFileReader(conf, path); @@ -50,16 +68,19 @@ public static HoodieFileReader getFileReader(Config throw new UnsupportedOperationException(extension + " format not supported yet."); } - private static HoodieFileReader newParquetFileReader(Configuration conf, Path path) { - return new HoodieParquetReader<>(conf, path); + public HoodieFileReader getFileReader(Configuration conf, Path path, HoodieFileFormat format) throws IOException { + return this.newParquetFileReader(conf, path); + } + + protected HoodieFileReader newParquetFileReader(Configuration conf, Path path) { + throw new UnsupportedOperationException(); } - private static HoodieFileReader newHFileFileReader(Configuration conf, Path path) throws IOException { - CacheConfig cacheConfig = new CacheConfig(conf); - return new HoodieHFileReader<>(conf, path, cacheConfig); + protected HoodieFileReader newHFileFileReader(Configuration conf, Path path) throws IOException { + throw new UnsupportedOperationException(); } - private static HoodieFileReader newOrcFileReader(Configuration conf, Path path) { - return new HoodieOrcReader<>(conf, path); + protected HoodieFileReader newOrcFileReader(Configuration conf, Path path) { + throw new UnsupportedOperationException(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java new file mode 100644 index 0000000000000..d0f2ef025102f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.avro.Schema; + +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; + +import java.io.IOException; +import java.util.Properties; + +public interface HoodieFileWriter { + boolean canWrite(); + + void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException; + + void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException; + + void close() throws IOException; + + default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema) throws IOException { + writeWithMetadata(key, record, schema, new Properties()); + } + + default void write(String recordKey, HoodieRecord record, Schema schema) throws IOException { + write(recordKey, record, schema, new Properties()); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java new file mode 100644 index 0000000000000..456383d3741fb --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +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.HoodieConfig; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.exception.HoodieException; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; + +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; + +public class HoodieFileWriterFactory { + + private static HoodieFileWriterFactory getWriterFactory(HoodieRecord.HoodieRecordType recordType) { + switch (recordType) { + case AVRO: + return new HoodieAvroFileWriterFactory(); + case SPARK: + try { + Class clazz = ReflectionUtils.getClass("org.apache.hudi.io.storage.HoodieSparkFileWriterFactory"); + return (HoodieFileWriterFactory) clazz.newInstance(); + } catch (IllegalAccessException | IllegalArgumentException | InstantiationException e) { + throw new HoodieException("Unable to create hoodie spark file writer factory", e); + } + default: + throw new UnsupportedOperationException(recordType + " record type not supported yet."); + } + } + + public static HoodieFileWriter getFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier, HoodieRecordType recordType) throws IOException { + final String extension = FSUtils.getFileExtension(path.getName()); + HoodieFileWriterFactory factory = getWriterFactory(recordType); + return factory.getFileWriterByFormat(extension, instantTime, path, conf, config, schema, taskContextSupplier); + } + + public static HoodieFileWriter getFileWriter(HoodieFileFormat format, + FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema, HoodieRecordType recordType) throws IOException { + HoodieFileWriterFactory factory = getWriterFactory(recordType); + return factory.getFileWriterByFormat(format, outputStream, conf, config, schema); + } + + protected HoodieFileWriter getFileWriterByFormat( + String extension, String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + if (PARQUET.getFileExtension().equals(extension)) { + return newParquetFileWriter(instantTime, path, conf, config, schema, taskContextSupplier); + } + if (HFILE.getFileExtension().equals(extension)) { + return newHFileFileWriter(instantTime, path, conf, config, schema, taskContextSupplier); + } + if (ORC.getFileExtension().equals(extension)) { + return newOrcFileWriter(instantTime, path, conf, config, schema, taskContextSupplier); + } + throw new UnsupportedOperationException(extension + " format not supported yet."); + } + + protected HoodieFileWriter getFileWriterByFormat(HoodieFileFormat format, + FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException { + switch (format) { + case PARQUET: + return newParquetFileWriter(outputStream, conf, config, schema); + default: + throw new UnsupportedOperationException(format + " format not supported yet."); + } + } + + protected HoodieFileWriter newParquetFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + throw new UnsupportedOperationException(); + } + + protected HoodieFileWriter newParquetFileWriter( + FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException { + throw new UnsupportedOperationException(); + } + + protected HoodieFileWriter newHFileFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + throw new UnsupportedOperationException(); + } + + protected HoodieFileWriter newOrcFileWriter( + String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema, + TaskContextSupplier taskContextSupplier) throws IOException { + throw new UnsupportedOperationException(); + } + + protected BloomFilter createBloomFilter(HoodieConfig config) { + return BloomFilterFactory.createBloomFilter(60000, 0.000000001, 100000, + BloomFilterTypeCode.DYNAMIC_V0.name()); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java similarity index 100% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java index 878a3c563b6f5..7e888842e6607 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java @@ -70,7 +70,7 @@ public static HFile.Reader createHFileReader( // Avoid loading default configs, from the FS, since this configuration is mostly // used as a stub to initialize HFile reader Configuration conf = new Configuration(false); - HoodieHFileReader.SeekableByteArrayInputStream bis = new HoodieHFileReader.SeekableByteArrayInputStream(content); + HoodieAvroHFileReader.SeekableByteArrayInputStream bis = new HoodieAvroHFileReader.SeekableByteArrayInputStream(content); FSDataInputStream fsdis = new FSDataInputStream(bis); FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fsdis); ReaderContext context = new ReaderContextBuilder() diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcConfig.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcConfig.java similarity index 100% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcConfig.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcConfig.java diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetStreamWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetStreamWriter.java index 01460859e02fd..226266bf6cf97 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetStreamWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetStreamWriter.java @@ -19,6 +19,7 @@ package org.apache.hudi.io.storage; import org.apache.hudi.avro.HoodieAvroWriteSupport; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.parquet.io.OutputStreamBackedOutputFile; import org.apache.avro.generic.IndexedRecord; @@ -36,18 +37,16 @@ * Hudi log block writer for parquet format. *

* TODO(HUDI-3035) unify w/ HoodieParquetWriter. - * - * @param Record implementation that permits field access by integer index. */ -public class HoodieParquetStreamWriter implements AutoCloseable { +public class HoodieParquetStreamWriter implements HoodieAvroFileWriter, AutoCloseable { - private final ParquetWriter writer; + private final ParquetWriter writer; private final HoodieAvroWriteSupport writeSupport; public HoodieParquetStreamWriter(FSDataOutputStream outputStream, HoodieParquetConfig parquetConfig) throws IOException { this.writeSupport = parquetConfig.getWriteSupport(); - this.writer = new Builder(new OutputStreamBackedOutputFile(outputStream), writeSupport) + this.writer = new Builder(new OutputStreamBackedOutputFile(outputStream), writeSupport) .withWriteMode(ParquetFileWriter.Mode.CREATE) .withCompressionCodec(parquetConfig.getCompressionCodecName()) .withRowGroupSize(parquetConfig.getBlockSize()) @@ -59,11 +58,23 @@ public HoodieParquetStreamWriter(FSDataOutputStream outputStream, .build(); } - public void writeAvro(String key, R object) throws IOException { - writer.write(object); + @Override + public boolean canWrite() { + return true; + } + + @Override + public void writeAvro(String key, IndexedRecord record) throws IOException { + writer.write(record); writeSupport.add(key); } + @Override + public void writeAvroWithMetadata(HoodieKey key, IndexedRecord avroRecord) throws IOException { + // TODO support populating the metadata + this.writeAvro(key.getRecordKey(), avroRecord); + } + @Override public void close() throws IOException { writer.close(); diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieSeekingFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieSeekingFileReader.java new file mode 100644 index 0000000000000..ee4a5f5e7a0f1 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieSeekingFileReader.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.avro.Schema; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.Option; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.List; + +public interface HoodieSeekingFileReader extends HoodieFileReader { + + default Option> getRecordByKey(String key, Schema readerSchema) throws IOException { + throw new UnsupportedOperationException(); + } + + default Option> getRecordByKey(String key) throws IOException { + return getRecordByKey(key, getSchema()); + } + + default ClosableIterator> getRecordsByKeysIterator(List keys, Schema schema) throws IOException { + throw new UnsupportedOperationException(); + } + + default ClosableIterator> getRecordsByKeysIterator(List keys) throws IOException { + return getRecordsByKeysIterator(keys, getSchema()); + } + + default ClosableIterator> getRecordsByKeyPrefixIterator(List keyPrefixes, Schema schema) throws IOException { + throw new UnsupportedEncodingException(); + } + + default ClosableIterator> getRecordsByKeyPrefixIterator(List keyPrefixes) throws IOException { + return getRecordsByKeyPrefixIterator(keyPrefixes, getSchema()); + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index e2fbc4e6716c3..2721795081cdf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -18,6 +18,9 @@ package org.apache.hudi.metadata; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -34,6 +37,7 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -50,12 +54,8 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.TableNotFoundException; -import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.Path; +import org.apache.hudi.io.storage.HoodieSeekingFileReader; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -99,7 +99,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { private final boolean reuse; // Readers for the latest file slice corresponding to file groups in the metadata partition - private Map, Pair> partitionReaders = + private final Map, Pair, HoodieMetadataMergedLogRecordReader>> partitionReaders = new ConcurrentHashMap<>(); public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, @@ -171,44 +171,45 @@ public HoodieData> getRecordsByKeyPrefixes(L HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices( metadataMetaClient, metadataFileSystemView, partitionName); - return (shouldLoadInMemory ? HoodieListData.lazy(partitionFileSlices) : engineContext.parallelize(partitionFileSlices)) - .flatMap((SerializableFunction>>) fileSlice -> { - // NOTE: Since this will be executed by executors, we can't access previously cached - // readers, and therefore have to always open new ones - Pair readers = - openReaders(partitionName, fileSlice); - - try { - List timings = new ArrayList<>(); + return (shouldLoadInMemory ? HoodieListData.lazy(partitionFileSlices) : + engineContext.parallelize(partitionFileSlices)) + .flatMap( + (SerializableFunction>>) fileSlice -> { + // NOTE: Since this will be executed by executors, we can't access previously cached + // readers, and therefore have to always open new ones + Pair, HoodieMetadataMergedLogRecordReader> readers = + openReaders(partitionName, fileSlice); + try { + List timings = new ArrayList<>(); - HoodieFileReader baseFileReader = readers.getKey(); - HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight(); + HoodieSeekingFileReader baseFileReader = readers.getKey(); + HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight(); - if (baseFileReader == null && logRecordScanner == null) { - // TODO: what do we do if both does not exist? should we throw an exception and let caller do the fallback ? - return Collections.emptyIterator(); - } + if (baseFileReader == null && logRecordScanner == null) { + // TODO: what do we do if both does not exist? should we throw an exception and let caller do the fallback ? + return Collections.emptyIterator(); + } - boolean fullKeys = false; + boolean fullKeys = false; - Map>> logRecords = - readLogRecords(logRecordScanner, sortedKeyPrefixes, fullKeys, timings); + Map>> logRecords = + readLogRecords(logRecordScanner, sortedKeyPrefixes, fullKeys, timings); - List>>> mergedRecords = - readFromBaseAndMergeWithLogRecords(baseFileReader, sortedKeyPrefixes, fullKeys, logRecords, timings, partitionName); + List>>> mergedRecords = + readFromBaseAndMergeWithLogRecords(baseFileReader, sortedKeyPrefixes, fullKeys, logRecords, timings, partitionName); - LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", - sortedKeyPrefixes.size(), timings)); + LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", + sortedKeyPrefixes.size(), timings)); - return mergedRecords.stream() + return mergedRecords.stream() .map(keyRecordPair -> keyRecordPair.getValue().orElse(null)) .iterator(); - } catch (IOException ioe) { - throw new HoodieIOException("Error merging records from metadata table for " + sortedKeyPrefixes.size() + " key : ", ioe); - } finally { - closeReader(readers); - } - }) + } catch (IOException ioe) { + throw new HoodieIOException("Error merging records from metadata table for " + sortedKeyPrefixes.size() + " key : ", ioe); + } finally { + closeReader(readers); + } + }) .filter(Objects::nonNull); } @@ -222,11 +223,11 @@ public List>>> getRecord List>>> result = new ArrayList<>(); AtomicInteger fileSlicesKeysCount = new AtomicInteger(); partitionFileSliceToKeysMap.forEach((partitionFileSlicePair, fileSliceKeys) -> { - Pair readers = + Pair, HoodieMetadataMergedLogRecordReader> readers = getOrCreateReaders(partitionName, partitionFileSlicePair.getRight()); try { List timings = new ArrayList<>(); - HoodieFileReader baseFileReader = readers.getKey(); + HoodieSeekingFileReader baseFileReader = readers.getKey(); HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight(); if (baseFileReader == null && logRecordScanner == null) { return; @@ -292,7 +293,7 @@ private Map>> readLogRecords( return logRecords; } - private List>>> readFromBaseAndMergeWithLogRecords(HoodieFileReader baseFileReader, + private List>>> readFromBaseAndMergeWithLogRecords(HoodieSeekingFileReader reader, List keys, boolean fullKeys, Map>> logRecords, @@ -300,7 +301,7 @@ private List>>> readFrom String partitionName) throws IOException { HoodieTimer timer = HoodieTimer.start(); - if (baseFileReader == null) { + if (reader == null) { // No base file at all timings.add(timer.endTimer()); if (fullKeys) { @@ -319,7 +320,7 @@ private List>>> readFrom HoodieTimer readTimer = HoodieTimer.start(); Map> records = - fetchBaseFileRecordsByKeys(baseFileReader, keys, fullKeys, partitionName); + fetchBaseFileRecordsByKeys(reader, keys, fullKeys, partitionName); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer())); @@ -351,17 +352,22 @@ private List>>> readFrom } } - private Map> fetchBaseFileRecordsByKeys(HoodieFileReader baseFileReader, + @SuppressWarnings("unchecked") + private Map> fetchBaseFileRecordsByKeys(HoodieSeekingFileReader reader, List keys, boolean fullKeys, String partitionName) throws IOException { - ClosableIterator records = fullKeys ? baseFileReader.getRecordsByKeysIterator(keys) - : baseFileReader.getRecordsByKeyPrefixIterator(keys); + ClosableIterator> records = fullKeys + ? reader.getRecordsByKeysIterator(keys) + : reader.getRecordsByKeyPrefixIterator(keys); return toStream(records) - .map(record -> Pair.of( - (String) record.get(HoodieMetadataPayload.KEY_FIELD_NAME), - composeRecord(record, partitionName))) + .map(record -> { + GenericRecord data = (GenericRecord) record.getData(); + return Pair.of( + (String) (data).get(HoodieMetadataPayload.KEY_FIELD_NAME), + composeRecord(data, partitionName)); + }) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); } @@ -409,21 +415,21 @@ private Map, List> getPartitionFileSliceToKeysMa * @param slice - The file slice to open readers for * @return File reader and the record scanner pair for the requested file slice */ - private Pair getOrCreateReaders(String partitionName, FileSlice slice) { + private Pair, HoodieMetadataMergedLogRecordReader> getOrCreateReaders(String partitionName, FileSlice slice) { if (reuse) { - return partitionReaders.computeIfAbsent(Pair.of(partitionName, slice.getFileId()), k -> { - return openReaders(partitionName, slice); }); + Pair key = Pair.of(partitionName, slice.getFileId()); + return partitionReaders.computeIfAbsent(key, ignored -> openReaders(partitionName, slice)); } else { return openReaders(partitionName, slice); } } - private Pair openReaders(String partitionName, FileSlice slice) { + private Pair, HoodieMetadataMergedLogRecordReader> openReaders(String partitionName, FileSlice slice) { try { HoodieTimer timer = HoodieTimer.start(); // Open base file reader - Pair baseFileReaderOpenTimePair = getBaseFileReader(slice, timer); - HoodieFileReader baseFileReader = baseFileReaderOpenTimePair.getKey(); + Pair, Long> baseFileReaderOpenTimePair = getBaseFileReader(slice, timer); + HoodieSeekingFileReader baseFileReader = baseFileReaderOpenTimePair.getKey(); final long baseFileOpenMs = baseFileReaderOpenTimePair.getValue(); // Open the log record scanner using the log files from the latest file slice @@ -441,18 +447,20 @@ private Pair openReaders( } } - private Pair getBaseFileReader(FileSlice slice, HoodieTimer timer) throws IOException { - HoodieFileReader baseFileReader = null; + private Pair, Long> getBaseFileReader(FileSlice slice, HoodieTimer timer) throws IOException { + HoodieSeekingFileReader baseFileReader; Long baseFileOpenMs; // If the base file is present then create a reader Option basefile = slice.getBaseFile(); if (basefile.isPresent()) { - String basefilePath = basefile.get().getPath(); - baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath)); + String baseFilePath = basefile.get().getPath(); + baseFileReader = (HoodieSeekingFileReader) HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO) + .getFileReader(hadoopConf.get(), new Path(baseFilePath)); baseFileOpenMs = timer.endTimer(); - LOG.info(String.format("Opened metadata base file from %s at instant %s in %d ms", basefilePath, + LOG.info(String.format("Opened metadata base file from %s at instant %s in %d ms", baseFilePath, basefile.get().getCommitTime(), baseFileOpenMs)); } else { + baseFileReader = null; baseFileOpenMs = 0L; timer.endTimer(); } @@ -580,7 +588,7 @@ public void close() { * @param partitionFileSlicePair - Partition and FileSlice */ private synchronized void close(Pair partitionFileSlicePair) { - Pair readers = + Pair, HoodieMetadataMergedLogRecordReader> readers = partitionReaders.remove(partitionFileSlicePair); closeReader(readers); } @@ -595,7 +603,7 @@ private void closePartitionReaders() { partitionReaders.clear(); } - private void closeReader(Pair readers) { + private void closeReader(Pair, HoodieMetadataMergedLogRecordReader> readers) { if (readers != null) { try { if (readers.getKey() != null) { diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java index eb7edcd2812d1..aec9877f07d4c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java @@ -18,22 +18,19 @@ package org.apache.hudi.metadata; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.log.InstantRange; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.internal.schema.InternalSchema; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -63,24 +60,7 @@ private HoodieMetadataMergedLogRecordReader(FileSystem fs, String basePath, Stri Option instantRange, boolean allowFullScan, boolean useScanV2) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, true, false, bufferSize, spillableMapBasePath, instantRange, diskMapType, isBitCaskDiskMapCompressionEnabled, false, allowFullScan, - Option.of(partitionName), InternalSchema.getEmptyInternalSchema(), useScanV2); - } - - @Override - protected HoodieAvroRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, - final String payloadClassFQN, final String preCombineField, - final boolean withOperationField, - final Option> simpleKeyGenFields, - final Option partitionName) { - if (hoodieTableConfig.populateMetaFields()) { - return super.createHoodieRecord(rec, hoodieTableConfig, payloadClassFQN, preCombineField, withOperationField, - simpleKeyGenFields, partitionName); - } - - // When meta fields are not available, create the record using the - // preset key field and the known partition name - return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN, - preCombineField, simpleKeyGenFields.get(), withOperationField, partitionName); + Option.of(partitionName), InternalSchema.getEmptyInternalSchema(), useScanV2, HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())); } /** @@ -130,6 +110,11 @@ public synchronized List } } + @Override + protected boolean getPopulateMetaFields() { + return this.hoodieTableMetaClient.getTableConfig().populateMetaFields() && super.getPopulateMetaFields(); + } + @Override protected String getKeyField() { return HoodieMetadataPayload.KEY_FIELD_NAME; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 08128539bc9e8..5c9bc862f9253 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -44,7 +44,7 @@ import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.hadoop.CachingPath; -import org.apache.hudi.io.storage.HoodieHFileReader; +import org.apache.hudi.io.storage.HoodieAvroHFileReader; import org.apache.hudi.util.Lazy; import org.apache.avro.Conversions; @@ -121,7 +121,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload convertMetadataToBloomFilterRecords( } final Path writeFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition); - try (HoodieFileReader fileReader = - HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), writeFilePath)) { + try (HoodieFileReader fileReader = + HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), writeFilePath)) { try { final BloomFilter fileBloomFilter = fileReader.readBloomFilter(); if (fileBloomFilter == null) { @@ -877,8 +878,8 @@ public static HoodieData convertFilesToBloomFilterRecords(HoodieEn } final String pathWithPartition = partitionName + "/" + appendedFile; final Path appendedFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition); - try (HoodieFileReader fileReader = - HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), appendedFilePath)) { + try (HoodieFileReader fileReader = + HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), appendedFilePath)) { final BloomFilter fileBloomFilter = fileReader.readBloomFilter(); if (fileBloomFilter == null) { LOG.error("Failed to read bloom filter for " + appendedFilePath); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java index 4865661cacc00..1d9c5f582403f 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java @@ -23,8 +23,12 @@ import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode; import org.apache.hudi.common.table.cdc.HoodieCDCUtils; @@ -51,6 +55,7 @@ import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.exception.CorruptedLogFileException; @@ -657,6 +662,7 @@ public void testBasicAppendAndScanMultipleFiles(ExternalSpillableMap.DiskMapType .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); List scannedRecords = new ArrayList<>(); @@ -892,6 +898,7 @@ public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMa .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); @@ -977,6 +984,7 @@ public void testAvroLogRecordReaderWithRollbackTombstone(ExternalSpillableMap.Di .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(200, scanner.getTotalLogRecords(), "We read 200 records from 2 write batches"); Set readKeys = new HashSet<>(200); @@ -1067,6 +1075,7 @@ public void testAvroLogRecordReaderWithFailedPartialBlock(ExternalSpillableMap.D .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(200, scanner.getTotalLogRecords(), "We would read 200 records"); Set readKeys = new HashSet<>(200); @@ -1148,6 +1157,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.Di .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(200, scanner.getTotalLogRecords(), "We still would read 200 records"); @@ -1156,7 +1166,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.Di scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); scanner.forEach(s -> { try { - if (!s.getData().getInsertValue(schema).isPresent()) { + if (!((HoodieRecordPayload)s.getData()).getInsertValue(schema).isPresent()) { emptyPayloads.add(true); } } catch (IOException io) { @@ -1195,13 +1205,14 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.Di .withSpillableMapBasePath(spillableBasePath) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals(100, readKeys.size(), "Stream collect should return all 200 records after rollback of delete"); final List newEmptyPayloads = new ArrayList<>(); scanner.forEach(s -> { try { - if (!s.getData().getInsertValue(schema).isPresent()) { + if (!((HoodieRecordPayload) s.getData()).getInsertValue(schema).isPresent()) { newEmptyPayloads.add(true); } } catch (IOException io) { @@ -1310,6 +1321,7 @@ public void testAvroLogRecordReaderWithDisorderDelete(ExternalSpillableMap.DiskM .withSpillableMapBasePath(spillableBasePath) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(200, scanner.getTotalLogRecords(), "We still would read 200 records"); @@ -1318,7 +1330,7 @@ public void testAvroLogRecordReaderWithDisorderDelete(ExternalSpillableMap.DiskM scanner.forEach(s -> readKeys.add(s.getRecordKey())); scanner.forEach(s -> { try { - if (!s.getData().getInsertValue(schema).isPresent()) { + if (!((HoodieRecordPayload)s.getData()).getInsertValue(schema).isPresent()) { emptyPayloadKeys.add(s.getRecordKey()); } } catch (IOException io) { @@ -1417,6 +1429,7 @@ public void testAvroLogRecordReaderWithFailedRollbacks(ExternalSpillableMap.Disk .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would have scanned 0 records because of rollback"); @@ -1489,6 +1502,7 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(ExternalSpillable .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); @@ -1544,6 +1558,7 @@ public void testAvroLogRecordReaderWithInvalidRollback(ExternalSpillableMap.Disk .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(100, scanner.getTotalLogRecords(), "We still would read 100 records"); final List readKeys = new ArrayList<>(100); @@ -1618,6 +1633,7 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(ExternalSpillabl .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); writer.close(); @@ -1728,6 +1744,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(ExternalS .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); @@ -1908,6 +1925,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsRollbackAndMergedLogB .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(true) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(600, scanner.getTotalLogRecords(), "We would read 600 records from scanner"); final List readKeys = new ArrayList<>(); @@ -1996,6 +2014,7 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) .withUseScanV2(useScanv2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(Math.max(numRecordsInLog1, numRecordsInLog2), scanner.getNumMergedRecordsInLog(), @@ -2256,7 +2275,7 @@ public void testV0Format() throws IOException, URISyntaxException { List recordsCopy = new ArrayList<>(records); assertEquals(100, records.size()); assertEquals(100, recordsCopy.size()); - HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, schema); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records.stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()), schema); byte[] content = dataBlock.getBytes(schema); assertTrue(content.length > 0); @@ -2343,10 +2362,10 @@ public void testDataBlockFormatAppendAndReadWithProjectedSchema( private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List records, Map header) { - return getDataBlock(dataBlockType, records, header, new Path("dummy_path")); + return getDataBlock(dataBlockType, records.stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()), header, new Path("dummy_path")); } - private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List records, + private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List records, Map header, Path pathForReader) { switch (dataBlockType) { case CDC_DATA_BLOCK: @@ -2402,10 +2421,10 @@ private static Stream testArgumentsWithoutScanV2Arg() { * Utility to convert the given iterator to a List. */ private static List getRecords(HoodieDataBlock dataBlock) { - ClosableIterator itr = dataBlock.getRecordIterator(); + ClosableIterator> itr = dataBlock.getRecordIterator(HoodieRecordType.AVRO); List elements = new ArrayList<>(); - itr.forEachRemaining(elements::add); + itr.forEachRemaining(r -> elements.add(r.getData())); return elements; } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java index e03d131665c13..d28dfe5e01e67 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.functional; import org.apache.hudi.common.model.HoodieArchivedLogFile; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; @@ -27,7 +28,6 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.testutils.SchemaTestUtil; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; @@ -51,6 +51,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSimpleSchema; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -95,7 +96,7 @@ public void testFailedToGetAppendStreamFromHDFSNameNode() fs.mkdirs(testPath); // Some data & append. - List records = SchemaTestUtil.generateTestRecords(0, 10); + List records = SchemaTestUtil.generateTestRecords(0, 10).stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()); Map header = new HashMap<>(2); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java b/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java index e17c3c4d792f6..2e24526905e23 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java @@ -72,7 +72,7 @@ public void tearDown() throws Exception { @Test public void testCreate() throws IOException { assertTrue(fs.exists(new Path(metaPath, HoodieTableConfig.HOODIE_PROPERTIES_FILE))); - HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null); assertEquals(6, config.getProps().size()); } @@ -85,7 +85,7 @@ public void testUpdate() throws IOException { assertTrue(fs.exists(cfgPath)); assertFalse(fs.exists(backupCfgPath)); - HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null); assertEquals(7, config.getProps().size()); assertEquals("test-table2", config.getTableName()); assertEquals("new_field", config.getPreCombineField()); @@ -98,7 +98,7 @@ public void testDelete() throws IOException { assertTrue(fs.exists(cfgPath)); assertFalse(fs.exists(backupCfgPath)); - HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null); assertEquals(5, config.getProps().size()); assertNull(config.getProps().getProperty("hoodie.invalid.config")); assertFalse(config.getProps().contains(HoodieTableConfig.ARCHIVELOG_FOLDER.key())); @@ -108,13 +108,13 @@ public void testDelete() throws IOException { public void testReadsWhenPropsFileDoesNotExist() throws IOException { fs.delete(cfgPath, false); assertThrows(HoodieIOException.class, () -> { - new HoodieTableConfig(fs, metaPath.toString(), null); + new HoodieTableConfig(fs, metaPath.toString(), null, null); }); } @Test public void testReadsWithUpdateFailures() throws IOException { - HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null); fs.delete(cfgPath, false); try (FSDataOutputStream out = fs.create(backupCfgPath)) { config.getProps().store(out, ""); @@ -122,14 +122,14 @@ public void testReadsWithUpdateFailures() throws IOException { assertFalse(fs.exists(cfgPath)); assertTrue(fs.exists(backupCfgPath)); - config = new HoodieTableConfig(fs, metaPath.toString(), null); + config = new HoodieTableConfig(fs, metaPath.toString(), null, null); assertEquals(6, config.getProps().size()); } @ParameterizedTest @ValueSource(booleans = {true, false}) public void testUpdateRecovery(boolean shouldPropsFileExist) throws IOException { - HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null); if (!shouldPropsFileExist) { fs.delete(cfgPath, false); } @@ -140,7 +140,7 @@ public void testUpdateRecovery(boolean shouldPropsFileExist) throws IOException HoodieTableConfig.recoverIfNeeded(fs, cfgPath, backupCfgPath); assertTrue(fs.exists(cfgPath)); assertFalse(fs.exists(backupCfgPath)); - config = new HoodieTableConfig(fs, metaPath.toString(), null); + config = new HoodieTableConfig(fs, metaPath.toString(), null, null); assertEquals(6, config.getProps().size()); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/HoodieRecordUtilsTest.java b/hudi-common/src/test/java/org/apache/hudi/common/util/HoodieRecordUtilsTest.java new file mode 100644 index 0000000000000..4dcabecccb7b9 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/HoodieRecordUtilsTest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.model.DefaultHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +class HoodieRecordUtilsTest { + + @Test + void loadHoodieMerge() { + String mergeClassName = HoodieAvroRecordMerger.class.getName(); + HoodieRecordMerger recordMerger1 = HoodieRecordUtils.loadRecordMerger(mergeClassName); + HoodieRecordMerger recordMerger2 = HoodieRecordUtils.loadRecordMerger(mergeClassName); + assertEquals(recordMerger1.getClass().getName(), mergeClassName); + assertEquals(recordMerger1, recordMerger2); + } + + @Test + void loadPayload() { + String payloadClassName = DefaultHoodieRecordPayload.class.getName(); + HoodieRecordPayload payload = HoodieRecordUtils.loadPayload(payloadClassName, new Object[]{null, 0}, GenericRecord.class, Comparable.class); + assertEquals(payload.getClass().getName(), payloadClassName); + } +} \ No newline at end of file diff --git a/hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieFileReaderFactory.java b/hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileReaderFactory.java similarity index 75% rename from hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieFileReaderFactory.java rename to hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileReaderFactory.java index ec334bde1e437..c306bab384b07 100644 --- a/hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieFileReaderFactory.java +++ b/hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileReaderFactory.java @@ -18,7 +18,8 @@ package org.apache.hudi.io.storage; -import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.Test; @@ -32,7 +33,7 @@ /** * Tests for {@link HoodieFileReaderFactory}. */ -public class TestHoodieFileReaderFactory { +public class TestHoodieAvroFileReaderFactory { @TempDir public java.nio.file.Path tempDir; @@ -41,19 +42,19 @@ public void testGetFileReader() throws IOException { // parquet file format. final Configuration hadoopConf = new Configuration(); final Path parquetPath = new Path("/partition/path/f1_1-0-1_000.parquet"); - HoodieFileReader parquetReader = HoodieFileReaderFactory.getFileReader(hadoopConf, parquetPath); - assertTrue(parquetReader instanceof HoodieParquetReader); + HoodieFileReader parquetReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(hadoopConf, parquetPath); + assertTrue(parquetReader instanceof HoodieAvroParquetReader); // log file format. final Path logPath = new Path("/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1"); final Throwable thrown = assertThrows(UnsupportedOperationException.class, () -> { - HoodieFileReader logWriter = HoodieFileReaderFactory.getFileReader(hadoopConf, logPath); + HoodieFileReader logWriter = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(hadoopConf, logPath); }, "should fail since log storage reader is not supported yet."); assertTrue(thrown.getMessage().contains("format not supported yet.")); // Orc file format. final Path orcPath = new Path("/partition/path/f1_1-0-1_000.orc"); - HoodieFileReader orcReader = HoodieFileReaderFactory.getFileReader(hadoopConf, orcPath); - assertTrue(orcReader instanceof HoodieOrcReader); + HoodieFileReader orcReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(hadoopConf, orcPath); + assertTrue(orcReader instanceof HoodieAvroOrcReader); } } diff --git a/hudi-examples/hudi-examples-flink/src/test/java/org/apache/hudi/examples/quickstart/TestQuickstartData.java b/hudi-examples/hudi-examples-flink/src/test/java/org/apache/hudi/examples/quickstart/TestQuickstartData.java index 67691a3ec7bd1..ab92990718685 100644 --- a/hudi-examples/hudi-examples-flink/src/test/java/org/apache/hudi/examples/quickstart/TestQuickstartData.java +++ b/hudi-examples/hudi-examples-flink/src/test/java/org/apache/hudi/examples/quickstart/TestQuickstartData.java @@ -37,7 +37,10 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.examples.quickstart.utils.QuickstartConfigurations; import org.apache.parquet.Strings; import org.apache.parquet.avro.AvroParquetReader; @@ -322,7 +325,7 @@ public static void checkWrittenDataMOR( .map(hoodieRecord -> { try { // in case it is a delete - GenericRecord record = (GenericRecord) hoodieRecord.getData() + GenericRecord record = (GenericRecord) ((HoodieAvroRecord)hoodieRecord).getData() .getInsertValue(schema, new Properties()) .orElse(null); return record == null ? (String) null : filterOutVariables(record); @@ -359,6 +362,7 @@ private static HoodieMergedLogRecordScanner getScanner( .withSpillableMapBasePath("/tmp/") .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); } diff --git a/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java b/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java index 6e20ee1190661..810fb33c4922f 100644 --- a/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java +++ b/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java @@ -43,7 +43,6 @@ import java.util.List; import java.util.stream.Collectors; - /** * Simple examples of #{@link HoodieJavaWriteClient}. * diff --git a/hudi-examples/hudi-examples-spark/src/test/java/org/apache/hudi/examples/quickstart/TestHoodieSparkQuickstart.java b/hudi-examples/hudi-examples-spark/src/test/java/org/apache/hudi/examples/quickstart/TestHoodieSparkQuickstart.java index 823d8ccd13685..4c7a02cc8419d 100644 --- a/hudi-examples/hudi-examples-spark/src/test/java/org/apache/hudi/examples/quickstart/TestHoodieSparkQuickstart.java +++ b/hudi-examples/hudi-examples-spark/src/test/java/org/apache/hudi/examples/quickstart/TestHoodieSparkQuickstart.java @@ -19,10 +19,10 @@ package org.apache.hudi.examples.quickstart; import org.apache.hudi.client.SparkRDDReadClient; -import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.testutils.providers.SparkProvider; +import org.apache.spark.HoodieSparkKryoProvider$; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; @@ -84,7 +84,7 @@ public synchronized void runBeforeEach() { initialized = spark != null; if (!initialized) { SparkConf sparkConf = conf(); - SparkRDDWriteClient.registerClasses(sparkConf); + HoodieSparkKryoProvider$.MODULE$.register(sparkConf); SparkRDDReadClient.addHoodieSupport(sparkConf); spark = SparkSession.builder().config(sparkConf).getOrCreate(); sqlContext = spark.sqlContext(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index 9250d6d5b35f0..959b0b5a45690 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -23,8 +23,10 @@ import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.model.EventTimeAvroPayload; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieSyncTableStrategy; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; @@ -114,6 +116,20 @@ private FlinkOptions() { .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n" + "This will render any value set for the option in-effective"); + public static final ConfigOption RECORD_MERGER_IMPLS = ConfigOptions + .key("record.merger.impls") + .stringType() + .defaultValue(HoodieAvroRecordMerger.class.getName()) + .withDescription("List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used. " + + "These merger impls will filter by record.merger.strategy. " + + "Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)"); + + public static final ConfigOption RECORD_MERGER_STRATEGY = ConfigOptions + .key("record.merger.strategy") + .stringType() + .defaultValue(HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID) + .withDescription("Id of merger strategy. Hudi will pick HoodieRecordMerger implementations in record.merger.impls which has the same merger strategy id"); + public static final ConfigOption PARTITION_DEFAULT_NAME = ConfigOptions .key("partition.default_name") .stringType() diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaRegistryProvider.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaRegistryProvider.java index c302c1db0d133..162d2a5dd75cd 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaRegistryProvider.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaRegistryProvider.java @@ -45,7 +45,6 @@ public class SchemaRegistryProvider extends SchemaProvider { private final TypedProperties config; - /** * Configs supported. */ diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java index 64178a82fb91b..ed4f1b02e69da 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -50,6 +51,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Random; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -102,6 +104,8 @@ public class StreamWriteFunction extends AbstractStreamWriteFunction { private transient BiFunction, String, List> writeFunction; + private transient HoodieRecordMerger recordMerger; + /** * Total size tracer. */ @@ -121,6 +125,7 @@ public void open(Configuration parameters) throws IOException { this.tracer = new TotalSizeTracer(this.config); initBuffer(); initWriteFunction(); + initMergeClass(); } @Override @@ -195,6 +200,11 @@ private void initWriteFunction() { } } + private void initMergeClass() { + recordMerger = writeClient.getConfig().getRecordMerger(); + LOG.info("init hoodie merge with class [{}]", recordMerger.getClass().getName()); + } + /** * Represents a data item in the buffer, this is needed to reduce the * memory footprint. @@ -421,7 +431,10 @@ private boolean flushBucket(DataBucket bucket) { List records = bucket.writeBuffer(); ValidationUtils.checkState(records.size() > 0, "Data bucket to flush has no buffering records"); if (config.getBoolean(FlinkOptions.PRE_COMBINE)) { - records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1, this.writeClient.getConfig().getSchema()); + Properties props = new Properties(); + config.addAllToProperties(props); + records = (List) FlinkWriteHelper.newInstance() + .deduplicateRecords(records, (HoodieIndex) null, -1, this.writeClient.getConfig().getSchema(), props, recordMerger); } bucket.preWrite(records); final List writeStatus = new ArrayList<>(writeFunction.apply(records, instant)); @@ -456,8 +469,10 @@ private void flushRemaining(boolean endInput) { List records = bucket.writeBuffer(); if (records.size() > 0) { if (config.getBoolean(FlinkOptions.PRE_COMBINE)) { - records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1, - this.writeClient.getConfig().getSchema()); + Properties props = new Properties(); + config.addAllToProperties(props); + records = (List) FlinkWriteHelper.newInstance() + .deduplicateRecords(records, (HoodieIndex) null, -1, this.writeClient.getConfig().getSchema(), props, recordMerger); } bucket.preWrite(records); writeStatus.addAll(writeFunction.apply(records, currentInstant)); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java index e7bde41ca8b0a..43e8a78d64a05 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java @@ -25,20 +25,23 @@ import org.apache.hudi.client.utils.ConcatenatingIterator; import org.apache.hudi.common.model.ClusteringGroupInfo; import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieFileSliceReader; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieClusteringConfig; -import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.IOUtils; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; @@ -82,6 +85,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Properties; import java.util.Spliterator; import java.util.Spliterators; import java.util.stream.Collectors; @@ -267,7 +271,7 @@ private Iterator readRecordsForGroupWithLogs(List try { Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) ? Option.empty() - : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); + : Option.of(HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType()).getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(table.getMetaClient().getFs()) .withBasePath(table.getMetaClient().getBasePath()) @@ -281,18 +285,18 @@ private Iterator readRecordsForGroupWithLogs(List .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath()) .withDiskMapType(writeConfig.getCommonConfig().getSpillableDiskMapType()) .withBitCaskDiskMapCompressionEnabled(writeConfig.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) + .withRecordMerger(writeConfig.getRecordMerger()) .build(); HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); HoodieFileSliceReader hoodieFileSliceReader = HoodieFileSliceReader.getFileSliceReader(baseFileReader, scanner, readerSchema, - tableConfig.getPayloadClass(), - tableConfig.getPreCombineField(), + tableConfig.getProps(), tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp()))); recordIterators.add(StreamSupport.stream(Spliterators.spliteratorUnknownSize(hoodieFileSliceReader, Spliterator.NONNULL), false).map(hoodieRecord -> { try { - return this.transform((IndexedRecord) hoodieRecord.getData().getInsertValue(readerSchema).get()); + return this.transform(hoodieRecord.toIndexedRecord(readerSchema, new Properties()).get().getData()); } catch (IOException e) { throw new HoodieIOException("Failed to read next record", e); } @@ -313,7 +317,10 @@ private Iterator readRecordsForGroupBaseFiles(List List> iteratorsForPartition = clusteringOps.stream().map(clusteringOp -> { Iterable indexedRecords = () -> { try { - return HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())).getRecordIterator(readerSchema); + HoodieFileReaderFactory fileReaderFactory = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType()); + HoodieAvroFileReader fileReader = (HoodieAvroFileReader) fileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())); + + return new MappingIterator<>(fileReader.getRecordIterator(readerSchema), HoodieRecord::getData); } catch (IOException e) { throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + " and " + clusteringOp.getDeltaFilePaths(), e); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java index 676c03f41c97d..af460b95ce4ed 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java @@ -184,8 +184,11 @@ private void processRecord(HoodieRecord record, Collector out) throws Exce // then update the index state using location with new partition path. HoodieRecord deleteRecord = new HoodieAvroRecord<>(new HoodieKey(recordKey, oldLoc.getPartitionPath()), payloadCreation.createDeletePayload((BaseAvroPayload) record.getData())); + + deleteRecord.unseal(); deleteRecord.setCurrentLocation(oldLoc.toLocal("U")); deleteRecord.seal(); + out.collect((O) deleteRecord); } location = getNewRecordLocation(partitionPath); @@ -200,7 +203,11 @@ private void processRecord(HoodieRecord record, Collector out) throws Exce if (isChangingRecords) { updateIndexState(partitionPath, location); } + + record.unseal(); record.setCurrentLocation(location); + record.seal(); + out.collect((O) record); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java index 45a3e0bf83729..9dd604a717f9e 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java @@ -19,6 +19,7 @@ package org.apache.hudi.sink.partitioner.profile; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -79,7 +80,7 @@ public class WriteProfile { /** * Total records to write for each bucket based on - * the config option {@link org.apache.hudi.config.HoodieStorageConfig#PARQUET_MAX_FILE_SIZE}. + * the config option {@link HoodieStorageConfig#PARQUET_MAX_FILE_SIZE}. */ private long recordsPerBucket; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/PayloadCreation.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/PayloadCreation.java index fb850bace7d48..b7756a490bf31 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/PayloadCreation.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/PayloadCreation.java @@ -86,7 +86,7 @@ public HoodieRecordPayload createPayload(GenericRecord record) throws Excepti public HoodieRecordPayload createDeletePayload(BaseAvroPayload payload) throws Exception { if (shouldCombine) { - return (HoodieRecordPayload) constructor.newInstance(null, payload.orderingVal); + return (HoodieRecordPayload) constructor.newInstance(null, payload.getOrderingVal()); } else { return (HoodieRecordPayload) this.constructor.newInstance(Option.empty()); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index 345c9cde6cd35..519fad1159d08 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -19,7 +19,9 @@ package org.apache.hudi.streamer; import org.apache.hudi.client.utils.OperationConverter; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.StringUtils; @@ -118,6 +120,14 @@ public class FlinkStreamerConfig extends Configuration { + "a GenericRecord. Implement your own, if you want to do something other than overwriting existing value.") public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName(); + @Parameter(names = {"--merger-impls"}, description = "List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used. " + + "These merger impls will filter by merger-strategy " + + "Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)") + public String mergerImpls = HoodieAvroRecordMerger.class.getName(); + + @Parameter(names = {"--merger-strategy"}, description = "Id of merger strategy. Hudi will pick HoodieRecordMerger implementations in merger-impls which has the same merger strategy id") + public String mergerStrategy = HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; + @Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input " + "is purely new data/inserts to gain speed).", converter = OperationConverter.class) public WriteOperationType operation = WriteOperationType.UPSERT; @@ -363,6 +373,8 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt conf.setString(FlinkOptions.OPERATION, config.operation.value()); conf.setString(FlinkOptions.PRECOMBINE_FIELD, config.sourceOrderingField); conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, config.payloadClassName); + conf.setString(FlinkOptions.RECORD_MERGER_IMPLS, config.mergerImpls); + conf.setString(FlinkOptions.RECORD_MERGER_STRATEGY, config.mergerStrategy); conf.setBoolean(FlinkOptions.PRE_COMBINE, config.preCombine); conf.setInteger(FlinkOptions.RETRY_TIMES, Integer.parseInt(config.instantRetryTimes)); conf.setLong(FlinkOptions.RETRY_INTERVAL_MS, Long.parseLong(config.instantRetryInterval)); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java index c455cb7616431..f408ae316ebd6 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java @@ -18,13 +18,17 @@ package org.apache.hudi.table.format; +import java.util.stream.Collectors; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Functions; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; @@ -164,6 +168,7 @@ public static HoodieMergedLogRecordScanner logScanner( .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath()) .withInstantRange(split.getInstantRange()) .withOperationField(flinkConf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)) + .withRecordMerger(writeConfig.getRecordMerger()) .build(); } @@ -183,6 +188,12 @@ public BoundedMemoryRecords( InternalSchema internalSchema, Configuration hadoopConf, org.apache.flink.configuration.Configuration flinkConf) { + List mergers = Arrays.stream(flinkConf.getString(FlinkOptions.RECORD_MERGER_IMPLS).split(",")) + .map(String::trim) + .distinct() + .collect(Collectors.toList()); + HoodieRecordMerger merger = HoodieRecordUtils.createRecordMerger( + split.getTablePath(), EngineType.FLINK, mergers, flinkConf.getString(FlinkOptions.RECORD_MERGER_STRATEGY)); HoodieUnMergedLogRecordScanner.Builder scannerBuilder = HoodieUnMergedLogRecordScanner.newBuilder() .withFileSystem(FSUtils.getFs(split.getTablePath(), hadoopConf)) .withBasePath(split.getTablePath()) @@ -198,7 +209,8 @@ public BoundedMemoryRecords( .withBufferSize( flinkConf.getInteger(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) - .withInstantRange(split.getInstantRange()); + .withInstantRange(split.getInstantRange()) + .withRecordMerger(merger); this.executor = new BoundedInMemoryExecutor<>( StreamerUtil.getMaxCompactionMemoryInBytes(flinkConf), @@ -260,6 +272,7 @@ public static HoodieMergedLogRecordScanner logScanner( .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath()) .withDiskMapType(writeConfig.getCommonConfig().getSpillableDiskMapType()) .withBitCaskDiskMapCompressionEnabled(writeConfig.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) + .withRecordMerger(writeConfig.getRecordMerger()) .build(); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index 2e40831d46f0a..094f9f77fc131 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -18,13 +18,19 @@ package org.apache.hudi.table.format.mor; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.log.InstantRange; import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; @@ -62,6 +68,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.stream.Collectors; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; @@ -674,6 +681,8 @@ protected static class MergeIterator implements ClosableIterator { private final InstantRange instantRange; + private final HoodieRecordMerger recordMerger; + // add the flag because the flink ParquetColumnarRowSplitReader is buggy: // method #reachedEnd() returns false after it returns true. // refactor it out once FLINK-22370 is resolved. @@ -681,7 +690,7 @@ protected static class MergeIterator implements ClosableIterator { private final Set keyToSkip = new HashSet<>(); - private final Properties payloadProps; + private final TypedProperties payloadProps; private RowData currentRecord; @@ -730,6 +739,11 @@ public MergeIterator( this.avroToRowDataConverter = AvroToRowDataConverters.createRowConverter(requiredRowType); this.projection = projection; this.instantRange = split.getInstantRange().orElse(null); + List mergers = Arrays.stream(flinkConf.getString(FlinkOptions.RECORD_MERGER_IMPLS).split(",")) + .map(String::trim) + .distinct() + .collect(Collectors.toList()); + this.recordMerger = HoodieRecordUtils.createRecordMerger(split.getTablePath(), EngineType.FLINK, mergers, flinkConf.getString(FlinkOptions.RECORD_MERGER_STRATEGY)); } @Override @@ -746,19 +760,19 @@ public boolean hasNext() { final String curKey = currentRecord.getString(HOODIE_RECORD_KEY_COL_POS).toString(); if (scanner.getRecords().containsKey(curKey)) { keyToSkip.add(curKey); - Option mergedAvroRecord = mergeRowWithLog(currentRecord, curKey); + Option mergedAvroRecord = mergeRowWithLog(currentRecord, curKey); if (!mergedAvroRecord.isPresent()) { // deleted continue; } else { - final RowKind rowKind = FormatUtils.getRowKindSafely(mergedAvroRecord.get(), this.operationPos); + final RowKind rowKind = FormatUtils.getRowKindSafely(mergedAvroRecord.get().getData(), this.operationPos); if (!emitDelete && rowKind == RowKind.DELETE) { // deleted continue; } IndexedRecord avroRecord = avroProjection.isPresent() - ? avroProjection.get().apply(mergedAvroRecord.get()) - : mergedAvroRecord.get(); + ? avroProjection.get().apply(mergedAvroRecord.get().getData()) + : mergedAvroRecord.get().getData(); this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord); this.currentRecord.setRowKind(rowKind); return true; @@ -817,11 +831,13 @@ public void close() { } } - private Option mergeRowWithLog(RowData curRow, String curKey) { + private Option mergeRowWithLog(RowData curRow, String curKey) { final HoodieAvroRecord record = (HoodieAvroRecord) scanner.getRecords().get(curKey); GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow); + HoodieAvroIndexedRecord hoodieAvroIndexedRecord = new HoodieAvroIndexedRecord(historyAvroRecord); try { - return record.getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema, payloadProps); + Option resultRecord = recordMerger.merge(hoodieAvroIndexedRecord, tableSchema, record, tableSchema, payloadProps).map(Pair::getLeft); + return resultRecord.get().toIndexedRecord(tableSchema, new Properties()); } catch (IOException e) { throw new HoodieIOException("Merge base and delta payloads exception", e); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java index ab409079c1944..5d4dac18ae1ab 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java @@ -24,6 +24,7 @@ import org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.WriteConcurrencyMode; @@ -35,7 +36,6 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieMemoryConfig; -import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java index ed5b178bb701a..3e76b5bcf9b6c 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -31,6 +32,7 @@ import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; @@ -806,9 +808,9 @@ public static void checkWrittenDataMOR( if (scanner != null) { for (String curKey : scanner.getRecords().keySet()) { if (!keyToSkip.contains(curKey)) { - Option record = (Option) scanner.getRecords() - .get(curKey).getData() - .getInsertValue(schema, config.getProps()); + Option record = (Option) ((HoodieAvroRecord) scanner.getRecords() + .get(curKey)).getData() + .getInsertValue(schema, config.getProps()); if (record.isPresent()) { readBuffer.add(filterOutVariables(record.get())); } @@ -844,6 +846,7 @@ private static HoodieMergedLogRecordScanner getScanner( .withSpillableMapBasePath("/tmp/") .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java index 6db1751771904..239062fadd591 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java @@ -24,7 +24,6 @@ import java.io.DataOutput; import java.io.IOException; - /** * Sub-type of File Split which encapsulates both skeleton and bootstrap base file splits. */ diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java index 53ccb7413f9b6..a3b4a6c1660c6 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java @@ -18,11 +18,8 @@ package org.apache.hudi.hadoop; -import java.io.IOException; -import java.util.Iterator; - import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -33,21 +30,25 @@ import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; -import org.apache.hudi.io.storage.HoodieHFileReader; +import org.apache.hudi.io.storage.HoodieAvroHFileReader; + +import java.io.IOException; +import java.util.Iterator; public class HoodieHFileRecordReader implements RecordReader { private long count = 0; private ArrayWritable valueObj; - private HoodieHFileReader reader; - private Iterator recordIterator; + private HoodieAvroHFileReader reader; + private Iterator> recordIterator; private Schema schema; public HoodieHFileRecordReader(Configuration conf, InputSplit split, JobConf job) throws IOException { FileSplit fileSplit = (FileSplit) split; Path path = fileSplit.getPath(); - reader = new HoodieHFileReader(conf, path, new CacheConfig(conf)); + reader = new HoodieAvroHFileReader(conf, path, new CacheConfig(conf)); schema = reader.getSchema(); valueObj = new ArrayWritable(Writable.class, new Writable[schema.getFields().size()]); @@ -63,7 +64,7 @@ public boolean next(NullWritable key, ArrayWritable value) throws IOException { return false; } - GenericRecord record = recordIterator.next(); + IndexedRecord record = recordIterator.next().getData(); ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, schema); value.set(aWritable.get()); count++; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index 356107dd82ca6..569230634781c 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -18,6 +18,7 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.io.NullWritable; @@ -26,12 +27,16 @@ import org.apache.hadoop.mapred.RecordReader; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.HoodieCommonConfig; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; import org.apache.hudi.hadoop.utils.HiveAvroSerializer; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; @@ -52,10 +57,11 @@ public class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader private static final Logger LOG = LogManager.getLogger(AbstractRealtimeRecordReader.class); protected final RecordReader parquetReader; - private final Map> deltaRecordMap; + private final Map deltaRecordMap; private final Set deltaRecordKeys; private final HoodieMergedLogRecordScanner mergedLogRecordScanner; + private final HoodieRecordMerger recordMerger; private final int recordKeyIndex; private Iterator deltaItr; @@ -69,6 +75,7 @@ public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job, this.recordKeyIndex = split.getVirtualKeyInfo() .map(HoodieVirtualKeyInfo::getRecordKeyFieldIndex) .orElse(HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS); + this.recordMerger = HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()); } /** @@ -95,14 +102,15 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())) .withUseScanV2(jobConf.getBoolean(HoodieRealtimeConfig.USE_LOG_RECORD_READER_SCAN_V2, false)) .withInternalSchema(schemaEvolutionContext.internalSchemaOption.orElse(InternalSchema.getEmptyInternalSchema())) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); } - private Option buildGenericRecordwithCustomPayload(HoodieRecord record) throws IOException { + private Option buildGenericRecordwithCustomPayload(HoodieRecord record) throws IOException { if (usesCustomPayload) { - return ((HoodieAvroRecord) record).getData().getInsertValue(getWriterSchema(), payloadProps); + return record.toIndexedRecord(getWriterSchema(), payloadProps); } else { - return ((HoodieAvroRecord) record).getData().getInsertValue(getReaderSchema(), payloadProps); + return record.toIndexedRecord(getReaderSchema(), payloadProps); } } @@ -116,7 +124,7 @@ public boolean next(NullWritable aVoid, ArrayWritable arrayWritable) throws IOEx if (deltaRecordMap.containsKey(key)) { // mark the key as handled this.deltaRecordKeys.remove(key); - Option rec = supportPayload ? mergeRecord(deltaRecordMap.get(key), arrayWritable) : buildGenericRecordwithCustomPayload(deltaRecordMap.get(key)); + Option rec = supportPayload ? mergeRecord(deltaRecordMap.get(key), arrayWritable) : buildGenericRecordwithCustomPayload(deltaRecordMap.get(key)); // If the record is not present, this is a delete record using an empty payload so skip this base record // and move to the next record if (!rec.isPresent()) { @@ -133,7 +141,7 @@ public boolean next(NullWritable aVoid, ArrayWritable arrayWritable) throws IOEx } while (this.deltaItr.hasNext()) { final String key = this.deltaItr.next(); - Option rec = buildGenericRecordwithCustomPayload(deltaRecordMap.get(key)); + Option rec = buildGenericRecordwithCustomPayload(deltaRecordMap.get(key)); if (rec.isPresent()) { setUpWritable(rec, arrayWritable, key); return true; @@ -142,12 +150,12 @@ public boolean next(NullWritable aVoid, ArrayWritable arrayWritable) throws IOEx return false; } - private void setUpWritable(Option rec, ArrayWritable arrayWritable, String key) { - GenericRecord recordToReturn = rec.get(); + private void setUpWritable(Option rec, ArrayWritable arrayWritable, String key) { + GenericRecord recordToReturn = (GenericRecord) rec.get().getData(); if (usesCustomPayload) { // If using a custom payload, return only the projection fields. The readerSchema is a schema derived from // the writerSchema with only the projection fields - recordToReturn = HoodieAvroUtils.rewriteRecord(rec.get(), getReaderSchema()); + recordToReturn = HoodieAvroUtils.rewriteRecord((GenericRecord) rec.get().getData(), getReaderSchema()); } // we assume, a later safe record in the log, is newer than what we have in the map & // replace it. Since we want to return an arrayWritable which is the same length as the elements in the latest @@ -175,7 +183,7 @@ private void setUpWritable(Option rec, ArrayWritable arrayWritabl } } - private Option mergeRecord(HoodieRecord newRecord, ArrayWritable writableFromParquet) throws IOException { + private Option mergeRecord(HoodieRecord newRecord, ArrayWritable writableFromParquet) throws IOException { GenericRecord oldRecord = convertArrayWritableToHoodieRecord(writableFromParquet); // presto will not append partition columns to jobConf.get(serdeConstants.LIST_COLUMNS), but hive will do it. This will lead following results // eg: current table: col1: int, col2: int, par: string, and column par is partition columns. @@ -183,9 +191,11 @@ private Option mergeRecord(HoodieRecord combinedValue = newRecord.getData().combineAndGetUpdateValue(HiveAvroSerializer.rewriteRecordIgnoreResultCheck(oldRecord, - getLogScannerReaderSchema()), getLogScannerReaderSchema(), payloadProps); - return combinedValue; + GenericRecord genericRecord = HiveAvroSerializer.rewriteRecordIgnoreResultCheck(oldRecord, getLogScannerReaderSchema()); + HoodieRecord record = new HoodieAvroIndexedRecord(genericRecord); + Option> mergeResult = recordMerger.merge(record, + genericRecord.getSchema(), newRecord, getLogScannerReaderSchema(), new TypedProperties(payloadProps)); + return mergeResult.map(p -> (HoodieAvroIndexedRecord) p.getLeft()); } private GenericRecord convertArrayWritableToHoodieRecord(ArrayWritable arrayWritable) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java index e21e266b7229f..4ac77788dcf71 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java @@ -24,9 +24,11 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Functions; +import org.apache.hudi.common.util.HoodieRecordUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; @@ -82,7 +84,9 @@ public RealtimeUnmergedRecordReader(RealtimeSplit split, JobConf job, .withLatestInstantTime(split.getMaxCommitTime()) .withReadBlocksLazily(Boolean.parseBoolean(this.jobConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED))) .withReverseReader(false) - .withBufferSize(this.jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)); + .withBufferSize(this.jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())); + this.executor = new BoundedInMemoryExecutor<>( HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes(jobConf), getParallelProducers(scannerBuilder), @@ -105,7 +109,7 @@ private List> getParallelProducers( HoodieUnMergedLogRecordScanner scanner = scannerBuilder.withLogRecordScannerCallback(record -> { // convert Hoodie log record to Hadoop AvroWritable and buffer - GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema(), payloadProps).get(); + GenericRecord rec = (GenericRecord) record.toIndexedRecord(getReaderSchema(), payloadProps).get().getData(); ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema()); queue.insertRecord(aWritable); }) diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java index 0be736b2a043d..a9cf806b19de5 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java @@ -19,6 +19,7 @@ package org.apache.hudi.hadoop; import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -159,6 +160,7 @@ static HoodieTableMetaClient initTableType(Configuration hadoopConf, String base properties.setProperty(HoodieTableConfig.NAME.key(), tableName); properties.setProperty(HoodieTableConfig.TYPE.key(), tableType.name()); properties.setProperty(HoodieTableConfig.PAYLOAD_CLASS_NAME.key(), HoodieAvroPayload.class.getName()); + properties.setProperty(HoodieTableConfig.MERGER_STRATEGY.key(), HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID); return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties); } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java index 84a967a0c4a50..68da1f179e3d4 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java @@ -21,6 +21,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; @@ -395,13 +396,14 @@ public static HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString()); HoodieDataBlock dataBlock = null; + List hoodieRecords = records.stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()); if (logBlockType == HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK) { dataBlock = new HoodieHFileDataBlock( - records, header, Compression.Algorithm.GZ, writer.getLogFile().getPath()); + hoodieRecords, header, Compression.Algorithm.GZ, writer.getLogFile().getPath()); } else if (logBlockType == HoodieLogBlock.HoodieLogBlockType.PARQUET_DATA_BLOCK) { - dataBlock = new HoodieParquetDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD, CompressionCodecName.GZIP, 0.1, true); + dataBlock = new HoodieParquetDataBlock(hoodieRecords, header, HoodieRecord.RECORD_KEY_METADATA_FIELD, CompressionCodecName.GZIP, 0.1, true); } else { - dataBlock = new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); + dataBlock = new HoodieAvroDataBlock(hoodieRecords, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); } writer.appendBlock(dataBlock); return writer; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/SparkDataSourceContinuousIngestTool.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/SparkDataSourceContinuousIngestTool.java index d8a4bbe7dac61..8729c8d6a4abb 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/SparkDataSourceContinuousIngestTool.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/SparkDataSourceContinuousIngestTool.java @@ -40,7 +40,6 @@ import java.util.List; import java.util.Map; - /** * Sample command * diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java index 4af926246ce57..8815ccc015cf3 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Set; - /** * A GenericRecordGeneratorIterator for the custom schema of the workload. Implements {@link Iterator} to allow for iteration semantics. */ diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index 6c951868bb9b3..3ed86c32c948c 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java @@ -18,36 +18,42 @@ package org.apache.hudi.integ.testsuite.reader; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; + import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; +import org.apache.hudi.common.util.HoodieRecordUtils; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.TypeUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieMemoryConfig; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Tuple2; import java.io.IOException; import java.io.UncheckedIOException; @@ -62,8 +68,6 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import scala.Tuple2; - import static java.util.Map.Entry.comparingByValue; import static java.util.stream.Collectors.toMap; @@ -265,9 +269,9 @@ private Iterator readColumnarOrLogFiles(FileSlice fileSlice) thro if (fileSlice.getBaseFile().isPresent()) { // Read the base files using the latest writer schema. Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr)); - HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(metaClient.getHadoopConf(), - new Path(fileSlice.getBaseFile().get().getPath())); - return reader.getRecordIterator(schema); + HoodieAvroFileReader reader = TypeUtils.unsafeCast(HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(metaClient.getHadoopConf(), + new Path(fileSlice.getBaseFile().get().getPath()))); + return new MappingIterator<>(reader.getRecordIterator(schema), HoodieRecord::getData); } else { // If there is no data file, fall back to reading log files HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() @@ -287,14 +291,15 @@ private Iterator readColumnarOrLogFiles(FileSlice fileSlice) thro .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue())) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); // readAvro log files - Iterable> iterable = () -> scanner.iterator(); + Iterable iterable = () -> scanner.iterator(); Schema schema = new Schema.Parser().parse(schemaStr); return StreamSupport.stream(iterable.spliterator(), false) .map(e -> { try { - return (IndexedRecord) e.getData().getInsertValue(schema).get(); + return (IndexedRecord) ((HoodieAvroRecord)e).getData().getInsertValue(schema).get(); } catch (IOException io) { throw new UncheckedIOException(io); } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java index 32969f2cf2c11..bf17ad3343615 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java @@ -32,7 +32,6 @@ import scala.collection.JavaConverters; - /** * Helper class to read avro and/or parquet files and generate a RDD of {@link GenericRecord}. */ diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index 5cdf3c3b80462..997787b36ef73 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -37,7 +37,7 @@ import org.apache.hudi.common.util.TablePathUtils; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodiePayloadConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieNotSupportedException; diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieSparkRecordMerger.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieSparkRecordMerger.java new file mode 100644 index 0000000000000..ff51e02702050 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieSparkRecordMerger.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi; + +import org.apache.avro.Schema; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; + +import java.io.IOException; + +public class HoodieSparkRecordMerger implements HoodieRecordMerger { + + @Override + public String getMergingStrategy() { + return HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; + } + + @Override + public Option> merge(HoodieRecord older, Schema oldSchema, HoodieRecord newer, Schema newSchema, TypedProperties props) throws IOException { + ValidationUtils.checkArgument(older.getRecordType() == HoodieRecordType.SPARK); + ValidationUtils.checkArgument(newer.getRecordType() == HoodieRecordType.SPARK); + + if (newer.getData() == null) { + // Delete record + return Option.empty(); + } + if (older.getData() == null) { + // use natural order for delete record + return Option.of(Pair.of(newer, newSchema)); + } + if (older.getOrderingValue(oldSchema, props).compareTo(newer.getOrderingValue(newSchema, props)) > 0) { + return Option.of(Pair.of(older, oldSchema)); + } else { + return Option.of(Pair.of(newer, newSchema)); + } + } + + @Override + public HoodieRecordType getRecordType() { + return HoodieRecordType.SPARK; + } +} \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 8e5bb9aad04b6..32c5866557243 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -340,6 +340,17 @@ object DataSourceWriteOptions { */ val PAYLOAD_CLASS_NAME = HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME + /** + * HoodieMerger will replace the payload to process the merge of data + * and provide the same capabilities as the payload + */ + val MERGER_IMPLS = HoodieWriteConfig.MERGER_IMPLS + + /** + * Id of merger strategy + */ + val MERGER_STRATEGY = HoodieWriteConfig.MERGER_STRATEGY + /** * Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value * will be obtained by invoking .toString() on the field value. Nested fields can be specified using diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index 9c984b96fb256..9f30db6158f88 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -29,20 +29,21 @@ import org.apache.hudi.HoodieBaseRelation._ import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.utils.SparkInternalSchemaConverter -import org.apache.hudi.common.config.{HoodieMetadataConfig, SerializableConfiguration} +import org.apache.hudi.common.config.{ConfigProperty, HoodieMetadataConfig, SerializableConfiguration} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath import org.apache.hudi.common.model.{FileSlice, HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.util.StringUtils +import org.apache.hudi.common.util.{ConfigUtils, StringUtils} import org.apache.hudi.common.util.StringUtils.isNullOrEmpty import org.apache.hudi.common.util.ValidationUtils.checkState +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema} -import org.apache.hudi.io.storage.HoodieHFileReader +import org.apache.hudi.io.storage.HoodieAvroHFileReader import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex import org.apache.spark.internal.Logging @@ -59,7 +60,6 @@ import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan} import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.sql.{HoodieCatalystExpressionUtils, Row, SQLContext, SparkSession} import org.apache.spark.unsafe.types.UTF8String - import java.net.URI import java.util.Locale @@ -77,7 +77,9 @@ case class HoodieTableState(tablePath: String, preCombineFieldOpt: Option[String], usesVirtualKeys: Boolean, recordPayloadClassName: String, - metadataConfig: HoodieMetadataConfig) + metadataConfig: HoodieMetadataConfig, + mergerImpls: List[String], + mergerStrategy: String) /** * Hoodie BaseRelation which extends [[PrunedFilteredScan]]. @@ -458,6 +460,11 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, } protected def getTableState: HoodieTableState = { + val mergerImpls = ConfigUtils.split2List(getConfigValue(HoodieWriteConfig.MERGER_IMPLS)).asScala.toList + + val mergerStrategy = getConfigValue(HoodieWriteConfig.MERGER_STRATEGY, + Option(metaClient.getTableConfig.getMergerStrategy)) + // Subset of the state of table's configuration as of at the time of the query HoodieTableState( tablePath = basePath, @@ -466,7 +473,9 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, preCombineFieldOpt = preCombineFieldOpt, usesVirtualKeys = !tableConfig.populateMetaFields(), recordPayloadClassName = tableConfig.getPayloadClass, - metadataConfig = fileIndex.metadataConfig + metadataConfig = fileIndex.metadataConfig, + mergerImpls = mergerImpls, + mergerStrategy = mergerStrategy ) } @@ -639,6 +648,12 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, private def prunePartitionColumns(dataStructSchema: StructType): StructType = StructType(dataStructSchema.filterNot(f => partitionColumns.contains(f.name))) + + private def getConfigValue(config: ConfigProperty[String], + defaultValueOption: Option[String]=Option.empty): String = { + optParams.getOrElse(config.key(), + sqlContext.getConf(config.key(), defaultValueOption.getOrElse(config.defaultValue()))) + } } object HoodieBaseRelation extends SparkAdapterSupport { @@ -725,7 +740,7 @@ object HoodieBaseRelation extends SparkAdapterSupport { partitionedFile => { val hadoopConf = hadoopConfBroadcast.value.get() - val reader = new HoodieHFileReader[GenericRecord](hadoopConf, new Path(partitionedFile.filePath), + val reader = new HoodieAvroHFileReader(hadoopConf, new Path(partitionedFile.filePath), new CacheConfig(hadoopConf)) val requiredRowSchema = requiredDataSchema.structTypeSchema @@ -736,7 +751,7 @@ object HoodieBaseRelation extends SparkAdapterSupport { reader.getRecordIterator(requiredAvroSchema).asScala .map(record => { - avroToRowConverter.apply(record).get + avroToRowConverter.apply(record.getData.asInstanceOf[GenericRecord]).get }) } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala index f122c9f9d673d..ad587ab4445b6 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala @@ -30,7 +30,6 @@ import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompac import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.{Partition, SerializableWritable, SparkContext, TaskContext} - import java.io.Closeable case class HoodieMergeOnReadPartition(index: Int, split: HoodieMergeOnReadFileSplit) extends Partition diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 93c55b7b67601..5dda7c9df78c3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -32,7 +32,9 @@ import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient} import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieConfig, HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.engine.HoodieEngineContext import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import org.apache.hudi.common.model._ +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.{CommitUtils, StringUtils} @@ -46,14 +48,14 @@ import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter import org.apache.hudi.internal.schema.utils.{AvroSchemaEvolutionUtils, SerDeHelper} import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory -import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} +import org.apache.hudi.keygen.{SparkKeyGeneratorInterface, TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} import org.apache.hudi.metrics.Metrics import org.apache.hudi.sync.common.HoodieSyncConfig import org.apache.hudi.sync.common.util.SyncUtilHelpers import org.apache.hudi.table.BulkInsertPartitioner import org.apache.hudi.util.SparkKeyGenUtils import org.apache.log4j.LogManager -import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier @@ -299,10 +301,6 @@ object HoodieSparkSqlWriter { (writeStatuses, client) case _ => // Here all other (than DELETE, DELETE_PARTITION) write operations are handled - // - // Convert to RDD[HoodieRecord] - val avroRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, avroRecordName, avroRecordNamespace, - Some(writerSchema)) // Check whether partition columns should be persisted w/in the data-files, or should // be instead omitted from them and simply encoded into the partition path (which is Spark's @@ -324,41 +322,26 @@ object HoodieSparkSqlWriter { // passing onto the Executor val dataFileSchemaStr = dataFileSchema.toString - val shouldCombine = parameters(INSERT_DROP_DUPS.key()).toBoolean || - operation.equals(WriteOperationType.UPSERT) || - parameters.getOrElse(HoodieWriteConfig.COMBINE_BEFORE_INSERT.key(), - HoodieWriteConfig.COMBINE_BEFORE_INSERT.defaultValue()).toBoolean - - val hoodieRecords = avroRecords.mapPartitions(it => { - val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr) - val consistentLogicalTimestampEnabled = parameters.getOrElse( - DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), - DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean - - it.map { avroRecord => - val processedRecord = if (shouldDropPartitionColumns) { - HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema) - } else { - avroRecord - } - val hoodieRecord = if (shouldCombine) { - val orderingVal = HoodieAvroUtils.getNestedFieldVal(avroRecord, hoodieConfig.getString(PRECOMBINE_FIELD), - false, consistentLogicalTimestampEnabled).asInstanceOf[Comparable[_]] - DataSourceUtils.createHoodieRecord(processedRecord, orderingVal, keyGenerator.getKey(avroRecord), - hoodieConfig.getString(PAYLOAD_CLASS_NAME)) - } else { - DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(avroRecord), - hoodieConfig.getString(PAYLOAD_CLASS_NAME)) - } - hoodieRecord - } - }).toJavaRDD() - + // Create a HoodieWriteClient & issue the write. val client = hoodieWriteClient.getOrElse { val finalOpts = addSchemaEvolutionParameters(parameters, internalSchemaOpt, Some(writerSchema)) - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key // TODO(HUDI-4772) proper writer-schema has to be specified here DataSourceUtils.createHoodieClient(jsc, dataFileSchemaStr, path, tblName, mapAsJavaMap(finalOpts)) }.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] + val writeConfig = client.getConfig + if (writeConfig.getRecordMerger.getRecordType == HoodieRecordType.SPARK && tableType == HoodieTableType.MERGE_ON_READ && writeConfig.getLogDataBlockFormat.orElse(HoodieLogBlockType.AVRO_DATA_BLOCK) != HoodieLogBlockType.PARQUET_DATA_BLOCK) { + throw new UnsupportedOperationException(s"${writeConfig.getRecordMerger.getClass.getName} only support parquet log.") + } + // Convert to RDD[HoodieRecord] + val hoodieRecords = createHoodieRecordRdd( + df, + writeConfig, + parameters, + avroRecordName, + avroRecordNamespace, + writerSchema, + dataFileSchemaStr, + operation) if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) { asyncCompactionTriggerFn.get.apply(client) @@ -510,14 +493,43 @@ object HoodieSparkSqlWriter { fullPartitions.distinct } - def generateSchemaWithoutPartitionColumns(partitionParam: String, schema: Schema): Schema = { - val partitionColumns = partitionParam.split(",") + def getPartitionColumns(partitionParam: String): Seq[String] = { + partitionParam.split(",") .map(partitionField => partitionField.trim) .filter(_.nonEmpty) .toSeq + } + + def generateSchemaWithoutPartitionColumns(partitionParam: String, schema: Schema): Schema = { + val partitionColumns = getPartitionColumns(partitionParam) HoodieAvroUtils.removeFields(schema, partitionColumns.toSet.asJava) } + def generateSparkSchemaWithoutPartitionColumns(partitionParam: String, schema: StructType): StructType = { + val partitionColumns = getPartitionColumns(partitionParam) + HoodieInternalRowUtils.removeFields(schema, partitionColumns) + } + + def getAvroProcessedRecord(partitionParam: String, record: GenericRecord, + dropPartitionColumns: Boolean): GenericRecord = { + var processedRecord = record + if (dropPartitionColumns) { + val writeSchema = generateSchemaWithoutPartitionColumns(partitionParam, record.getSchema) + processedRecord = HoodieAvroUtils.rewriteRecord(record, writeSchema) + } + processedRecord + } + + def getProcessedRecord(partitionParam: String, record: GenericRecord, + dropPartitionColumns: Boolean): GenericRecord = { + var processedRecord = record + if (dropPartitionColumns) { + val writeSchema = generateSchemaWithoutPartitionColumns(partitionParam, record.getSchema) + processedRecord = HoodieAvroUtils.rewriteRecord(record, writeSchema) + } + processedRecord + } + def addSchemaEvolutionParameters(parameters: Map[String, String], internalSchemaOpt: Option[InternalSchema], writeSchemaOpt: Option[Schema] = None): Map[String, String] = { val schemaEvolutionEnable = if (internalSchemaOpt.isDefined) "true" else "false" @@ -707,10 +719,10 @@ object HoodieSparkSqlWriter { } def validateSchemaForHoodieIsDeleted(schema: Schema): Unit = { - if (schema.getField(HoodieRecord.HOODIE_IS_DELETED) != null && - AvroConversionUtils.resolveAvroTypeNullability(schema.getField(HoodieRecord.HOODIE_IS_DELETED).schema())._2.getType != Schema.Type.BOOLEAN) { - throw new HoodieException(HoodieRecord.HOODIE_IS_DELETED + " has to be BOOLEAN type. Passed in dataframe's schema has type " - + schema.getField(HoodieRecord.HOODIE_IS_DELETED).schema().getType) + if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) != null && + AvroConversionUtils.resolveAvroTypeNullability(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).schema())._2.getType != Schema.Type.BOOLEAN) { + throw new HoodieException(HoodieRecord.HOODIE_IS_DELETED_FIELD + " has to be BOOLEAN type. Passed in dataframe's schema has type " + + schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).schema().getType) } } @@ -1017,4 +1029,79 @@ object HoodieSparkSqlWriter { Map.empty } } + + private def createHoodieRecordRdd(df: DataFrame, + config: HoodieWriteConfig, + parameters: Map[String, String], + recordName: String, + recordNameSpace: String, + writerSchema: Schema, + dataFileSchemaStr: String, + operation: WriteOperationType) = { + val shouldDropPartitionColumns = config.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS) + val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps)) + val recordType = config.getRecordMerger.getRecordType + + val shouldCombine = parameters(INSERT_DROP_DUPS.key()).toBoolean || + operation.equals(WriteOperationType.UPSERT) || + parameters.getOrElse(HoodieWriteConfig.COMBINE_BEFORE_INSERT.key(), + HoodieWriteConfig.COMBINE_BEFORE_INSERT.defaultValue()).toBoolean + + log.debug(s"Use $recordType") + recordType match { + case HoodieRecord.HoodieRecordType.AVRO => + val avroRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, recordName, recordNameSpace, + Some(writerSchema)) + + avroRecords.mapPartitions(it => { + val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr) + val consistentLogicalTimestampEnabled = parameters.getOrElse( + DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), + DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean + + it.map { avroRecord => + val processedRecord = if (shouldDropPartitionColumns) { + HoodieAvroUtils.rewriteRecord(avroRecord, dataFileSchema) + } else { + avroRecord + } + val hoodieRecord = if (shouldCombine) { + val orderingVal = HoodieAvroUtils.getNestedFieldVal(avroRecord, config.getString(PRECOMBINE_FIELD), + false, consistentLogicalTimestampEnabled).asInstanceOf[Comparable[_]] + DataSourceUtils.createHoodieRecord(processedRecord, orderingVal, keyGenerator.getKey(avroRecord), + config.getString(PAYLOAD_CLASS_NAME)) + } else { + DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(avroRecord), + config.getString(PAYLOAD_CLASS_NAME)) + } + hoodieRecord + } + }).toJavaRDD() + case HoodieRecord.HoodieRecordType.SPARK => + // ut will use AvroKeyGenerator, so we need to cast it in spark record + val sparkKeyGenerator = keyGenerator.asInstanceOf[SparkKeyGeneratorInterface] + val dataFileSchema = new Schema.Parser().parse(dataFileSchemaStr) + val dataFileStructType = HoodieInternalRowUtils.getCachedSchema(dataFileSchema) + val writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema) + val sourceStructType = df.schema + df.queryExecution.toRdd.mapPartitions { iter => + + iter.map { internalRow => + val recordKey = sparkKeyGenerator.getRecordKey(internalRow, sourceStructType) + val partitionPath = sparkKeyGenerator.getPartitionPath(internalRow, sourceStructType) + val key = new HoodieKey(recordKey.toString, partitionPath.toString) + + if (shouldDropPartitionColumns) { + val row = HoodieInternalRowUtils.getCachedUnsafeProjection(dataFileStructType, dataFileStructType) + .apply(HoodieInternalRowUtils.rewriteRecord(internalRow, sourceStructType, dataFileStructType)) + new HoodieSparkRecord(key, row, dataFileStructType, false) + } else { + val row = HoodieInternalRowUtils.getCachedUnsafeProjection(writerStructType, writerStructType) + .apply(HoodieInternalRowUtils.rewriteRecord(internalRow, sourceStructType, writerStructType)) + new HoodieSparkRecord(key, row, writerStructType, false) + } + } + }.toJavaRDD().asInstanceOf[JavaRDD[HoodieRecord[_]]] + } + } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 0f97fe3f0490f..d0e270ed136be 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -18,7 +18,7 @@ package org.apache.hudi import org.apache.hudi.DataSourceOptionsHelper.allAlternatives -import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.DataSourceWriteOptions.{MERGER_IMPLS, _} import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieCommonConfig, HoodieConfig} import org.apache.hudi.common.table.HoodieTableConfig @@ -29,7 +29,6 @@ import org.apache.hudi.sync.common.HoodieSyncConfig import org.apache.hudi.util.SparkKeyGenUtils import org.apache.spark.sql.SparkSession import org.apache.spark.sql.hudi.command.SqlKeyGenerator - import java.util.Properties import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters._ @@ -224,7 +223,8 @@ object HoodieWriterUtils { PRECOMBINE_FIELD -> HoodieTableConfig.PRECOMBINE_FIELD, PARTITIONPATH_FIELD -> HoodieTableConfig.PARTITION_FIELDS, RECORDKEY_FIELD -> HoodieTableConfig.RECORDKEY_FIELDS, - PAYLOAD_CLASS_NAME -> HoodieTableConfig.PAYLOAD_CLASS_NAME + PAYLOAD_CLASS_NAME -> HoodieTableConfig.PAYLOAD_CLASS_NAME, + MERGER_STRATEGY -> HoodieTableConfig.MERGER_STRATEGY ) def mappingSparkDatasourceConfigsToTableConfigs(options: Map[String, String]): Map[String, String] = { val includingTableConfigs = scala.collection.mutable.Map() ++ options diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala index 68b25fafe038d..7e18f7a9746e6 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala @@ -18,33 +18,36 @@ package org.apache.hudi +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.JobConf import org.apache.hudi.HoodieBaseRelation.{BaseFileReader, generateUnsafeProjection} import org.apache.hudi.HoodieConversionUtils.{toJavaOption, toScalaOption} import org.apache.hudi.HoodieDataSourceHelper.AvroDeserializerSupport -import org.apache.hudi.common.model.{HoodieLogFile, HoodieRecord, HoodieRecordPayload} -import org.apache.hudi.config.HoodiePayloadConfig -import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes import org.apache.hudi.LogFileIterator._ -import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMetadataConfig} -import org.apache.hudi.common.engine.HoodieLocalEngineContext +import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMetadataConfig, TypedProperties} +import org.apache.hudi.common.engine.{EngineType, HoodieLocalEngineContext} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType +import org.apache.hudi.common.model.{HoodieAvroIndexedRecord, HoodieEmptyRecord, HoodieLogFile, HoodieRecord, HoodieSparkRecord} import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner +import org.apache.hudi.common.util.HoodieRecordUtils +import org.apache.hudi.config.HoodiePayloadConfig import org.apache.hudi.hadoop.config.HoodieRealtimeConfig +import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes import org.apache.hudi.internal.schema.InternalSchema -import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata} import org.apache.hudi.metadata.HoodieTableMetadata.getDataTableBasePathFromMetadataTable -import org.apache.avro.Schema -import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred.JobConf +import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata} import org.apache.hudi.util.CachingIterator +import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieInternalRowUtils} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection import org.apache.spark.sql.types.StructType import java.io.Closeable -import java.util.Properties import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.util.Try @@ -62,20 +65,23 @@ class LogFileIterator(split: HoodieMergeOnReadFileSplit, protected val maxCompactionMemoryInBytes: Long = getMaxCompactionMemoryInBytes(new JobConf(config)) - protected val payloadProps = tableState.preCombineFieldOpt + protected val payloadProps: TypedProperties = tableState.preCombineFieldOpt .map { preCombineField => HoodiePayloadConfig.newBuilder .withPayloadOrderingField(preCombineField) .build .getProps - }.getOrElse(new Properties()) + } + .getOrElse(new TypedProperties()) protected override val avroSchema: Schema = new Schema.Parser().parse(requiredSchema.avroSchemaStr) protected override val structTypeSchema: StructType = requiredSchema.structTypeSchema protected val logFileReaderAvroSchema: Schema = new Schema.Parser().parse(tableSchema.avroSchemaStr) + protected val logFileReaderStructType: StructType = tableSchema.structTypeSchema - private val requiredSchemaSafeAvroProjection = SafeAvroProjection.create(logFileReaderAvroSchema, avroSchema) + protected val requiredSchemaSafeAvroProjection: SafeAvroProjection = SafeAvroProjection.create(logFileReaderAvroSchema, avroSchema) + protected val requiredSchemaUnsafeRowProjection: UnsafeProjection = HoodieCatalystExpressionUtils.generateUnsafeProjection(logFileReaderStructType, structTypeSchema) // TODO: now logScanner with internalSchema support column project, we may no need projectAvroUnsafe private var logScanner = { @@ -86,21 +92,20 @@ class LogFileIterator(split: HoodieMergeOnReadFileSplit, private val logRecords = logScanner.getRecords.asScala - def logRecordsIterator(): Iterator[(String, HoodieRecord[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]]])] = { - logRecords.iterator.asInstanceOf[Iterator[(String, HoodieRecord[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]]])]] + def logRecordsPairIterator(): Iterator[(String, HoodieRecord[_])] = { + logRecords.iterator } // NOTE: This have to stay lazy to make sure it's initialized only at the point where it's // going to be used, since we modify `logRecords` before that and therefore can't do it any earlier - protected lazy val genericRecordsIterator: Iterator[Option[GenericRecord]] = - logRecords.iterator.map { - case (_, record) => - toScalaOption(record.getData.getInsertValue(logFileReaderAvroSchema, payloadProps)) - .map(_.asInstanceOf[GenericRecord]) + protected lazy val logRecordsIterator: Iterator[Option[HoodieRecord[_]]] = logRecords.iterator.map { + case (_, record: HoodieSparkRecord) => Option(record) + case (_, _: HoodieEmptyRecord[_]) => Option.empty + case (_, record) => + toScalaOption(record.toIndexedRecord(logFileReaderAvroSchema, payloadProps)) } - protected def removeLogRecord(key: String): Option[HoodieRecord[_ <: HoodieRecordPayload[_]]] = - logRecords.remove(key) + protected def removeLogRecord(key: String): Option[HoodieRecord[_]] = logRecords.remove(key) protected def doHasNext: Boolean = hasNextInternal @@ -108,15 +113,16 @@ class LogFileIterator(split: HoodieMergeOnReadFileSplit, // that recursion is unfolded into a loop to avoid stack overflows while // handling records @tailrec private def hasNextInternal: Boolean = { - genericRecordsIterator.hasNext && { - val avroRecordOpt = genericRecordsIterator.next() - if (avroRecordOpt.isEmpty) { - // Record has been deleted, skipping - this.hasNextInternal - } else { - val projectedAvroRecord = requiredSchemaSafeAvroProjection(avroRecordOpt.get) - nextRecord = deserialize(projectedAvroRecord) - true + logRecordsIterator.hasNext && { + logRecordsIterator.next() match { + case Some(r: HoodieAvroIndexedRecord) => + val projectedAvroRecord = requiredSchemaSafeAvroProjection(r.getData.asInstanceOf[GenericRecord]) + nextRecord = deserialize(projectedAvroRecord) + true + case Some(r: HoodieSparkRecord) => + nextRecord = requiredSchemaUnsafeRowProjection(r.getData) + true + case None => this.hasNextInternal } } } @@ -189,6 +195,8 @@ class RecordMergingFileIterator(split: HoodieMergeOnReadFileSplit, private val baseFileIterator = baseFileReader(split.dataFile.get) + private val recordMerger = HoodieRecordUtils.createRecordMerger(tableState.tablePath, EngineType.SPARK, tableState.mergerImpls.asJava, tableState.mergerStrategy) + override def doHasNext: Boolean = hasNextInternal // NOTE: It's crucial for this method to be annotated w/ [[@tailrec]] to make sure @@ -204,14 +212,12 @@ class RecordMergingFileIterator(split: HoodieMergeOnReadFileSplit, nextRecord = requiredSchemaUnsafeProjection(curRow) true } else { - val mergedAvroRecordOpt = merge(serialize(curRow), updatedRecordOpt.get) - if (mergedAvroRecordOpt.isEmpty) { + val mergedRecordOpt = merge(curRow, updatedRecordOpt.get) + if (mergedRecordOpt.isEmpty) { // Record has been deleted, skipping this.hasNextInternal } else { - val projectedAvroRecord = projectAvroUnsafe(mergedAvroRecordOpt.get.asInstanceOf[GenericRecord], - avroSchema, reusableRecordBuilder) - nextRecord = deserialize(projectedAvroRecord) + nextRecord = mergedRecordOpt.get true } } @@ -223,10 +229,25 @@ class RecordMergingFileIterator(split: HoodieMergeOnReadFileSplit, private def serialize(curRowRecord: InternalRow): GenericRecord = serializer.serialize(curRowRecord).asInstanceOf[GenericRecord] - private def merge(curAvroRecord: GenericRecord, newRecord: HoodieRecord[_ <: HoodieRecordPayload[_]]): Option[IndexedRecord] = { + private def merge(curRow: InternalRow, newRecord: HoodieRecord[_]): Option[InternalRow] = { // NOTE: We have to pass in Avro Schema used to read from Delta Log file since we invoke combining API // on the record from the Delta Log - toScalaOption(newRecord.getData.combineAndGetUpdateValue(curAvroRecord, logFileReaderAvroSchema, payloadProps)) + recordMerger.getRecordType match { + case HoodieRecordType.SPARK => + val curRecord = new HoodieSparkRecord(curRow, baseFileReader.schema) + val result = recordMerger.merge(curRecord, baseFileReaderAvroSchema, newRecord, logFileReaderAvroSchema, payloadProps) + toScalaOption(result) + .map(r => { + val schema = HoodieInternalRowUtils.getCachedSchema(r.getRight) + val projection = HoodieInternalRowUtils.getCachedUnsafeProjection(schema, structTypeSchema) + projection.apply(r.getLeft.getData.asInstanceOf[InternalRow]) + }) + case _ => + val curRecord = new HoodieAvroIndexedRecord(serialize(curRow)) + val result = recordMerger.merge(curRecord, baseFileReaderAvroSchema, newRecord, logFileReaderAvroSchema, payloadProps) + toScalaOption(result) + .map(r => deserialize(projectAvroUnsafe(r.getLeft.toIndexedRecord(r.getRight, payloadProps).get.getData.asInstanceOf[GenericRecord], avroSchema, reusableRecordBuilder))) + } } } @@ -295,6 +316,9 @@ object LogFileIterator { getRelativePartitionPath(new Path(tableState.tablePath), logFiles.head.getPath.getParent)) } + val recordMerger = HoodieRecordUtils.createRecordMerger(tableState.tablePath, EngineType.SPARK, tableState.mergerImpls.asJava, tableState.mergerStrategy) + logRecordScannerBuilder.withRecordMerger(recordMerger) + logRecordScannerBuilder.build() } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala index e484e5905b312..092bf8a191971 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala @@ -24,7 +24,7 @@ import org.apache.hudi.HoodieConversionUtils._ import org.apache.hudi.HoodieDataSourceHelper.AvroDeserializerSupport import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} -import org.apache.hudi.common.model.{FileSlice, HoodieLogFile, HoodieRecord, HoodieRecordPayload} +import org.apache.hudi.common.model.{FileSlice, HoodieAvroRecordMerger, HoodieLogFile, HoodieRecord, HoodieRecordMerger, HoodieRecordPayload} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.cdc.HoodieCDCInferCase._ import org.apache.hudi.common.table.cdc.HoodieCDCOperation._ @@ -151,7 +151,7 @@ class HoodieCDCRDD( private lazy val tableState = { val metadataConfig = HoodieMetadataConfig.newBuilder() .fromProperties(props) - .build(); + .build() HoodieTableState( pathToString(basePath), split.changes.last.getInstant, @@ -159,7 +159,10 @@ class HoodieCDCRDD( preCombineFieldOpt, usesVirtualKeys = false, metaClient.getTableConfig.getPayloadClass, - metadataConfig + metadataConfig, + // TODO support CDC with spark record + mergerImpls = List(classOf[HoodieAvroRecordMerger].getName), + mergerStrategy = HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID ) } @@ -217,7 +220,7 @@ class HoodieCDCRDD( * Only one case where it will be used is that extract the change data from log files for mor table. * At the time, 'logRecordIter' will work with [[beforeImageRecords]] that keep all the records of the previous file slice. */ - private var logRecordIter: Iterator[(String, HoodieRecord[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]]])] = Iterator.empty + private var logRecordIter: Iterator[(String, HoodieRecord[_])] = Iterator.empty /** * Only one case where it will be used is that extract the change data from cdc log files. @@ -435,7 +438,7 @@ class HoodieCDCRDD( val absLogPath = new Path(basePath, currentChangeFile.getCdcFiles.get(0)) val morSplit = HoodieMergeOnReadFileSplit(None, List(new HoodieLogFile(fs.getFileStatus(absLogPath)))) val logFileIterator = new LogFileIterator(morSplit, originTableSchema, originTableSchema, tableState, conf) - logRecordIter = logFileIterator.logRecordsIterator() + logRecordIter = logFileIterator.logRecordsPairIterator case AS_IS => assert(currentChangeFile.getCdcFiles != null && !currentChangeFile.getCdcFiles.isEmpty) // load beforeFileSlice to beforeImageRecords @@ -604,9 +607,9 @@ class HoodieCDCRDD( } private def getInsertValue( - record: HoodieRecord[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]]]) + record: HoodieRecord[_]) : Option[IndexedRecord] = { - toScalaOption(record.getData.getInsertValue(avroSchema, payloadProps)) + toScalaOption(record.toIndexedRecord(avroSchema, payloadProps)).map(_.getData) } private def projectAvroUnsafe(record: IndexedRecord): GenericRecord = { @@ -614,8 +617,8 @@ class HoodieCDCRDD( avroSchema, reusableRecordBuilder) } - private def merge(curAvroRecord: GenericRecord, newRecord: HoodieRecord[_ <: HoodieRecordPayload[_]]): IndexedRecord = { - newRecord.getData.combineAndGetUpdateValue(curAvroRecord, avroSchema, payloadProps).get() + private def merge(curAvroRecord: GenericRecord, newRecord: HoodieRecord[_]): IndexedRecord = { + newRecord.getData.asInstanceOf[HoodieRecordPayload[_]].combineAndGetUpdateValue(curAvroRecord, avroSchema, payloadProps).get() } private def generateUnsafeProjection(from: StructType, to: StructType): UnsafeProjection = diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala index 4a3cf38895f0a..7cb4a3c542843 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala @@ -341,7 +341,7 @@ object DataSkippingUtils extends Logging { } } -private object ColumnStatsExpressionUtils { +object ColumnStatsExpressionUtils { @inline def genColMinValueExpr(colName: String): Expression = col(getMinColumnNameFor(colName)).expr @inline def genColMaxValueExpr(colName: String): Expression = col(getMaxColumnNameFor(colName)).expr diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala index 7efb60ae0b6a5..df75b60f5494e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.avro.HoodieAvroUtils.getRootLevelFieldName +import org.apache.hudi.common.model.{HoodieAvroRecordMerger, HoodieRecordMerger} import org.apache.hudi.common.table.HoodieTableConfig -import org.apache.hudi.common.util.ValidationUtils +import org.apache.hudi.common.util.{StringUtils, ValidationUtils} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.types.StructType @@ -68,6 +69,13 @@ object HoodieOptionConfig { .defaultValue(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue()) .build() + val SQL_MERGER_STRATEGY: HoodieSQLOption[String] = buildConf() + .withSqlKey("mergerStrategy") + .withHoodieKey(DataSourceWriteOptions.MERGER_STRATEGY.key) + .withTableConfigKey(HoodieTableConfig.MERGER_STRATEGY.key) + .defaultValue(HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID) + .build() + /** * The mapping of the sql short name key to the hoodie's config key. */ @@ -184,7 +192,7 @@ object HoodieOptionConfig { // extract primaryKey, preCombineField, type options def extractSqlOptions(options: Map[String, String]): Map[String, String] = { val sqlOptions = mappingTableConfigToSqlOption(options) - val targetOptions = keyMapping.keySet -- Set(SQL_PAYLOAD_CLASS.sqlKeyName) + val targetOptions = keyMapping.keySet -- Set(SQL_PAYLOAD_CLASS.sqlKeyName) -- Set(SQL_MERGER_STRATEGY.sqlKeyName) sqlOptions.filterKeys(targetOptions.contains) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index 61acdf866102b..fcba6e310dc97 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -66,6 +66,7 @@ trait ProvidesHoodieConfig extends Logging { RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), TBL_NAME.key -> hoodieCatalogTable.tableName, PRECOMBINE_FIELD.key -> preCombineField, + MERGER_IMPLS.key -> hoodieProps.getString(HoodieWriteConfig.MERGER_IMPLS.key, HoodieWriteConfig.MERGER_IMPLS.defaultValue), HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable, URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitioning, KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, @@ -164,6 +165,7 @@ trait ProvidesHoodieConfig extends Logging { tableType == COW_TABLE_TYPE_OPT_VAL && insertMode == InsertMode.STRICT) { // Validate duplicate key for COW, for MOR it will do the merge with the DefaultHoodieRecordPayload // on reading. + // TODO use HoodieSparkValidateDuplicateKeyRecordMerger when SparkRecordMerger is default classOf[ValidateDuplicateKeyPayload].getCanonicalName } else if (operation == INSERT_OPERATION_OPT_VAL && tableType == COW_TABLE_TYPE_OPT_VAL && insertMode == InsertMode.STRICT){ @@ -190,6 +192,7 @@ trait ProvidesHoodieConfig extends Logging { PRECOMBINE_FIELD.key -> preCombineField, PARTITIONPATH_FIELD.key -> partitionFieldsStr, PAYLOAD_CLASS_NAME.key -> payloadClassName, + MERGER_IMPLS.key -> hoodieProps.getString(HoodieWriteConfig.MERGER_IMPLS.key, HoodieWriteConfig.MERGER_IMPLS.defaultValue), ENABLE_ROW_WRITER.key -> enableBulkInsert.toString, HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn), HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFieldsStr, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/HoodieSparkValidateDuplicateKeyRecordMerger.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/HoodieSparkValidateDuplicateKeyRecordMerger.scala new file mode 100644 index 0000000000000..f59b62d7bec94 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/HoodieSparkValidateDuplicateKeyRecordMerger.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command + +import org.apache.avro.Schema +import org.apache.hudi.HoodieSparkRecordMerger +import org.apache.hudi.common.config.TypedProperties +import org.apache.hudi.common.model.HoodieAvroRecordMerger.Config +import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.util.{collection, Option => HOption} +import org.apache.hudi.exception.HoodieDuplicateKeyException + +/** + * Validate the duplicate key for insert statement without enable the INSERT_DROP_DUPS_OPT + * config. + * @see org.apache.spark.sql.hudi.command.ValidateDuplicateKeyPayload + */ +class HoodieSparkValidateDuplicateKeyRecordMerger extends HoodieSparkRecordMerger { + + override def merge(older: HoodieRecord[_], oldSchema: Schema, newer: HoodieRecord[_], newSchema: Schema, props: TypedProperties): HOption[collection.Pair[HoodieRecord[_], Schema]] = { + val legacyOperatingMode = Config.LegacyOperationMode.valueOf(props.getString(Config.LEGACY_OPERATING_MODE.key, Config.LEGACY_OPERATING_MODE.defaultValue)) + legacyOperatingMode match { + case Config.LegacyOperationMode.PRE_COMBINING => + super.merge(older, oldSchema, newer, newSchema, props) + case Config.LegacyOperationMode.COMBINING => + val key = older.getRecordKey(oldSchema, HoodieRecord.RECORD_KEY_METADATA_FIELD) + throw new HoodieDuplicateKeyException(key) + case _ => + throw new UnsupportedOperationException(String.format("Unsupported legacy operating mode (%s)", legacyOperatingMode)) + } + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala index 01c995fed437c..083c2eb8e84cc 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala @@ -30,11 +30,11 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.{StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String import org.joda.time.format.DateTimeFormat - import java.sql.Timestamp import java.util import java.util.Collections import java.util.concurrent.TimeUnit.{MICROSECONDS, MILLISECONDS} +import org.apache.spark.sql.catalyst.InternalRow /** * Custom Spark-specific [[KeyGenerator]] overriding behavior handling [[TimestampType]] partition values diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java index b9f77bccfd56d..f94540d3b0c68 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java +++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java @@ -21,6 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.client.HoodieInternalWriteStatus; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -155,13 +156,12 @@ protected void assertOutput(Dataset expectedRows, Dataset actualRows, if (populateMetaColumns) { // verify 3 meta fields that are filled in within create handle actualRows.collectAsList().forEach(entry -> { - assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))); + assertEquals(entry.get(HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.ordinal()).toString(), instantTime); + assertFalse(entry.isNullAt(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal())); if (fileNames.isPresent()) { - assertTrue(fileNames.get().contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS - .get(HoodieRecord.FILENAME_METADATA_FIELD)))); + assertTrue(fileNames.get().contains(entry.get(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal()))); } - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))); + assertFalse(entry.isNullAt(HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.ordinal())); }); // after trimming 2 of the meta fields, rest of the fields should match diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java index 453cbb4e748ac..59674b928fdfa 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java @@ -18,7 +18,9 @@ package org.apache.hudi; -import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -26,10 +28,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import java.io.IOException; @@ -239,8 +237,8 @@ public void close() { private static Option convertToString(HoodieRecord record) { try { - String str = HoodieAvroUtils - .bytesToAvro(((OverwriteWithLatestAvroPayload) record.getData()).recordBytes, DataGenerator.avroSchema) + String str = ((OverwriteWithLatestAvroPayload) record.getData()) + .getInsertValue(DataGenerator.avroSchema) .toString(); str = "{" + str.substring(str.indexOf("\"ts\":")); return Option.of(str.replaceAll("}", ", \"partitionpath\": \"" + record.getPartitionPath() + "\"}")); diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkFullBootstrapDataProviderBase.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkFullBootstrapDataProviderBase.java index bc732a1401a4f..ffda92c634f47 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkFullBootstrapDataProviderBase.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkFullBootstrapDataProviderBase.java @@ -18,27 +18,32 @@ package org.apache.hudi.bootstrap; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.DataSourceUtils; import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieSparkRecord; import org.apache.hudi.common.bootstrap.FileStatusUtils; 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.model.HoodieRecord.HoodieRecordType; 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.keygen.KeyGenerator; +import org.apache.hudi.keygen.SparkKeyGeneratorInterface; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; - -import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.rdd.RDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; import java.io.IOException; import java.util.List; @@ -55,32 +60,48 @@ public SparkFullBootstrapDataProviderBase(TypedProperties props, @Override public JavaRDD generateInputRecords(String tableName, String sourceBasePath, - List>> partitionPathsWithFiles) { + List>> partitionPathsWithFiles, HoodieWriteConfig config) { String[] filePaths = partitionPathsWithFiles.stream().map(Pair::getValue) .flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toString())) .toArray(String[]::new); // NOTE: "basePath" option is required for spark to discover the partition column // More details at https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + HoodieRecordType recordType = config.getRecordMerger().getRecordType(); Dataset inputDataset = sparkSession.read().format(getFormat()).option("basePath", sourceBasePath).load(filePaths); try { KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); + String precombineKey = props.getString("hoodie.datasource.write.precombine.field"); String structName = tableName + "_record"; String namespace = "hoodie." + tableName; - RDD genericRecords = HoodieSparkUtils.createRdd(inputDataset, structName, namespace, false, - Option.empty()); - return genericRecords.toJavaRDD().map(gr -> { - String orderingVal = HoodieAvroUtils.getNestedFieldValAsString( - gr, props.getString("hoodie.datasource.write.precombine.field"), false, props.getBoolean( - KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), - Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()))); - try { - return DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr), - props.getString("hoodie.datasource.write.payload.class")); - } catch (IOException ioe) { - throw new HoodieIOException(ioe.getMessage(), ioe); - } - }); + if (recordType == HoodieRecordType.AVRO) { + RDD genericRecords = HoodieSparkUtils.createRdd(inputDataset, structName, namespace, false, + Option.empty()); + return genericRecords.toJavaRDD().map(gr -> { + String orderingVal = HoodieAvroUtils.getNestedFieldValAsString( + gr, precombineKey, false, props.getBoolean( + KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), + Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()))); + try { + return DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr), + props.getString("hoodie.datasource.write.payload.class")); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }); + } else if (recordType == HoodieRecordType.SPARK) { + SparkKeyGeneratorInterface sparkKeyGenerator = (SparkKeyGeneratorInterface) keyGenerator; + StructType structType = inputDataset.schema(); + return inputDataset.queryExecution().toRdd().toJavaRDD().map(internalRow -> { + String recordKey = sparkKeyGenerator.getRecordKey(internalRow, structType).toString(); + String partitionPath = sparkKeyGenerator.getPartitionPath(internalRow, structType).toString(); + HoodieKey key = new HoodieKey(recordKey, partitionPath); + return new HoodieSparkRecord(key, internalRow, structType, false); + }); + } else { + throw new UnsupportedOperationException(recordType.name()); + } + } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala index 01f13b30f647b..d74cdfc2e63f7 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala @@ -18,15 +18,15 @@ package org.apache.spark.sql.hudi import org.apache.avro.Schema -import org.apache.avro.generic.IndexedRecord import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hudi.avro.HoodieAvroWriteSupport import org.apache.hudi.client.SparkTaskContextSupplier import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory} +import org.apache.hudi.common.config.HoodieStorageConfig import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.util.BaseFileUtils -import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig} +import org.apache.hudi.config.HoodieIndexConfig import org.apache.hudi.io.storage.{HoodieAvroParquetWriter, HoodieParquetConfig} import org.apache.parquet.avro.AvroSchemaConverter import org.apache.parquet.hadoop.metadata.CompressionCodecName @@ -57,7 +57,7 @@ object SparkHelpers { // Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'. parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader) - val writer = new HoodieAvroParquetWriter[IndexedRecord](destinationFile, parquetConfig, instantTime, new SparkTaskContextSupplier(), true) + val writer = new HoodieAvroParquetWriter(destinationFile, parquetConfig, instantTime, 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-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index b90807858f3b8..d4cba17f4d498 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -21,6 +21,7 @@ import org.apache.avro.Schema import org.apache.hudi.AvroConversionUtils.convertStructTypeToAvroSchema import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.util.StringUtils +import org.apache.hudi.common.model.HoodieAvroRecordMerger import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.{AVRO_SCHEMA_VALIDATE_ENABLE, TBL_NAME} import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME @@ -45,7 +46,6 @@ import org.apache.spark.sql.types.{BooleanType, StructType} import java.util.Base64 - /** * The Command for hoodie MergeIntoTable. * The match on condition must contain the row key fields currently, so that we can use Hoodie @@ -188,7 +188,9 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie // Create the write parameters val parameters = buildMergeIntoConfig(hoodieCatalogTable) - executeUpsert(sourceDF, parameters) + // TODO Remove it when we implement ExpressionPayload for SparkRecord + val parametersWithAvroRecordMerger = parameters ++ Map(HoodieWriteConfig.MERGER_IMPLS.key -> classOf[HoodieAvroRecordMerger].getName) + executeUpsert(sourceDF, parametersWithAvroRecordMerger) sparkSession.catalog.refreshTable(targetTableIdentify.unquotedString) Seq.empty[Row] diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala index e4e85c2c1a20a..d874f7bec3af2 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala @@ -26,6 +26,7 @@ import org.apache.hudi.SparkAdapterSupport.sparkAdapter import org.apache.hudi.avro.AvroSchemaUtils.isNullable import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro +import org.apache.hudi.common.model.BaseAvroPayload.isDeleteRecord import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodiePayloadProps, HoodieRecord} import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.common.util.{ValidationUtils, Option => HOption} @@ -86,11 +87,14 @@ class ExpressionPayload(@transient record: GenericRecord, processMatchedRecord(ConvertibleRecord(joinedRecord), Some(targetRecord), properties) } + override def canProduceSentinel: Boolean = true + /** * Process the matched record. Firstly test if the record matched any of the update-conditions, * if matched, return the update assignments result. Secondly, test if the record matched * delete-condition, if matched then return a delete record. Finally if no condition matched, - * return a {@link HoodieWriteHandle.IGNORE_RECORD} which will be ignored by HoodieWriteHandle. + * return a [[HoodieRecord.SENTINEL]] which will be ignored by HoodieWriteHandle. + * * @param inputRecord The input record to process. * @param targetRecord The origin exist record. * @param properties The properties. @@ -153,7 +157,7 @@ class ExpressionPayload(@transient record: GenericRecord, if (resultRecordOpt == null) { // If there is no condition matched, just filter this record. // here we return a IGNORE_RECORD, HoodieMergeHandle will not handle it. - HOption.of(HoodieWriteHandle.IGNORE_RECORD) + HOption.of(HoodieRecord.SENTINEL) } else { resultRecordOpt } @@ -180,7 +184,7 @@ class ExpressionPayload(@transient record: GenericRecord, /** * Process the not-matched record. Test if the record matched any of insert-conditions, * if matched then return the result of insert-assignment. Or else return a - * {@link HoodieWriteHandle.IGNORE_RECORD} which will be ignored by HoodieWriteHandle. + * [[HoodieRecord.SENTINEL]] which will be ignored by HoodieWriteHandle. * * @param inputRecord The input record to process. * @param properties The properties. @@ -215,10 +219,20 @@ class ExpressionPayload(@transient record: GenericRecord, } else { // If there is no condition matched, just filter this record. // Here we return a IGNORE_RECORD, HoodieCreateHandle will not handle it. - HOption.of(HoodieWriteHandle.IGNORE_RECORD) + HOption.of(HoodieRecord.SENTINEL) } } + override def isDeleted(schema: Schema, props: Properties): Boolean = { + val deleteConditionText = props.get(ExpressionPayload.PAYLOAD_DELETE_CONDITION) + val isUpdateRecord = props.getProperty(HoodiePayloadProps.PAYLOAD_IS_UPDATE_RECORD_FOR_MOR, "false").toBoolean + val isDeleteOnCondition= if (isUpdateRecord && deleteConditionText != null) { + !getInsertValue(schema, props).isPresent + } else false + + isDeletedRecord || isDeleteOnCondition + } + override def getInsertValue(schema: Schema, properties: Properties): HOption[IndexedRecord] = { val recordSchema = getRecordSchema(properties) val incomingRecord = ConvertibleRecord(bytesToAvro(recordBytes, recordSchema)) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala index 114f4c4ee130c..55a282e773086 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala @@ -35,11 +35,11 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} - import java.io.File import java.util import java.util.Collections import java.util.function.Supplier +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import scala.collection.JavaConverters._ import scala.util.control.Breaks.break @@ -124,7 +124,7 @@ class ExportInstantsProcedure extends BaseProcedure with ProcedureBuilder with L }) { val blk = reader.next.asInstanceOf[HoodieAvroDataBlock] try { - val recordItr = blk.getRecordIterator + val recordItr = blk.getRecordIterator(HoodieRecordType.AVRO) try while ( { recordItr.hasNext }) { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileMetadataProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileMetadataProcedure.scala index 3a26823dedb9a..3553caef515fc 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileMetadataProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileMetadataProcedure.scala @@ -28,10 +28,10 @@ import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.parquet.avro.AvroSchemaConverter import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} - import java.util.Objects import java.util.concurrent.atomic.AtomicInteger import java.util.function.Supplier +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import scala.collection.JavaConverters.{asScalaBufferConverter, asScalaIteratorConverter, mapAsScalaMapConverter} class ShowHoodieLogFileMetadataProcedure extends BaseProcedure with ProcedureBuilder { @@ -93,7 +93,7 @@ class ShowHoodieLogFileMetadataProcedure extends BaseProcedure with ProcedureBui } block match { case dataBlock: HoodieDataBlock => - val recordItr = dataBlock.getRecordIterator + val recordItr = dataBlock.getRecordIterator(HoodieRecordType.AVRO) recordItr.asScala.foreach(_ => recordCount.incrementAndGet()) recordItr.close() } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileRecordsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileRecordsProcedure.scala index 2806138a8954b..9c3e2e469b0fd 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileRecordsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileRecordsProcedure.scala @@ -21,18 +21,18 @@ import org.apache.avro.generic.IndexedRecord import org.apache.hadoop.fs.Path import org.apache.hudi.common.config.HoodieCommonConfig import org.apache.hudi.common.fs.FSUtils -import org.apache.hudi.common.model.HoodieLogFile +import org.apache.hudi.common.model.{HoodieAvroRecordMerger, HoodieLogFile, HoodieRecordPayload} import org.apache.hudi.common.table.log.block.HoodieDataBlock import org.apache.hudi.common.table.log.{HoodieLogFormat, HoodieMergedLogRecordScanner} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.util.ValidationUtils +import org.apache.hudi.common.util.{HoodieRecordUtils, ValidationUtils} import org.apache.hudi.config.{HoodieCompactionConfig, HoodieMemoryConfig} import org.apache.parquet.avro.AvroSchemaConverter import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} - import java.util.Objects import java.util.function.Supplier +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import scala.collection.JavaConverters._ class ShowHoodieLogFileRecordsProcedure extends BaseProcedure with ProcedureBuilder { @@ -76,9 +76,10 @@ class ShowHoodieLogFileRecordsProcedure extends BaseProcedure with ProcedureBuil .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue) .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(classOf[HoodieAvroRecordMerger].getName)) .build scanner.asScala.foreach(hoodieRecord => { - val record = hoodieRecord.getData.getInsertValue(schema).get() + val record = hoodieRecord.getData.asInstanceOf[HoodieRecordPayload[_]].getInsertValue(schema).get() if (allRecords.size() < limit) { allRecords.add(record) } @@ -92,10 +93,10 @@ class ShowHoodieLogFileRecordsProcedure extends BaseProcedure with ProcedureBuil val block = reader.next() block match { case dataBlock: HoodieDataBlock => - val recordItr = dataBlock.getRecordIterator + val recordItr = dataBlock.getRecordIterator(HoodieRecordType.AVRO) recordItr.asScala.foreach(record => { if (allRecords.size() < limit) { - allRecords.add(record) + allRecords.add(record.getData.asInstanceOf[IndexedRecord]) } }) recordItr.close() diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java index 11f0fc97856e9..26786e08bbc6c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -27,7 +28,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.config.HoodieClusteringConfig; -import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java index ba0f4befe3cb6..543e01702d35c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java @@ -53,7 +53,7 @@ import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; import org.apache.hudi.index.HoodieIndex.IndexType; -import org.apache.hudi.io.storage.HoodieParquetReader; +import org.apache.hudi.io.storage.HoodieAvroParquetReader; import org.apache.hudi.keygen.NonpartitionedKeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.table.action.bootstrap.BootstrapUtils; @@ -173,7 +173,7 @@ public Schema generateNewDataSetAndReturnSchema(long timestamp, int numRecords, String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), srcPath, context).stream().findAny().map(p -> p.getValue().stream().findAny()) .orElse(null).get().getPath()).toString(); - HoodieParquetReader parquetReader = new HoodieParquetReader<>(metaClient.getHadoopConf(), new Path(filePath)); + HoodieAvroParquetReader parquetReader = new HoodieAvroParquetReader(metaClient.getHadoopConf(), new Path(filePath)); return parquetReader.getSchema(); } @@ -489,7 +489,7 @@ public TestFullBootstrapDataProvider(TypedProperties props, HoodieSparkEngineCon @Override public JavaRDD generateInputRecords(String tableName, String sourceBasePath, - List>> partitionPaths) { + List>> partitionPaths, HoodieWriteConfig config) { String filePath = FileStatusUtils.toPath(partitionPaths.stream().flatMap(p -> p.getValue().stream()) .findAny().get().getPath()).toString(); ParquetFileReader reader = null; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java index a343ab3c5ccee..3a84755248e2f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java @@ -33,7 +33,7 @@ import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieSparkTable; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java index d0c74d109c2b7..a236755d0725e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java @@ -385,7 +385,7 @@ public TestFullBootstrapDataProvider(TypedProperties props, HoodieSparkEngineCon @Override public JavaRDD generateInputRecords(String tableName, String sourceBasePath, - List>> partitionPaths) { + List>> partitionPaths, HoodieWriteConfig config) { String[] filePaths = partitionPaths.stream().map(Pair::getValue) .flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toString())) .toArray(String[]::new); 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 ddf7aa351c7a7..1bb81f7f92eea 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 @@ -209,6 +209,7 @@ class TestDataSourceDefaults extends ScalaAssertionSupport { val STRUCT_NAME: String = "hoodieRowTopLevelField" val NAMESPACE: String = "hoodieRow" var converterFn: Function1[Row, GenericRecord] = _ + var internalConverterFn: Function1[InternalRow, GenericRecord] = _ override def getKey(record: GenericRecord): HoodieKey = { new HoodieKey(HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true, false), diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index e30ba0d4f5330..375abcebfeee5 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -23,7 +23,7 @@ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieFileIndex.DataSkippingFailureMode import org.apache.hudi.client.HoodieJavaWriteClient import org.apache.hudi.client.common.HoodieJavaEngineContext -import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig} import org.apache.hudi.common.engine.EngineType import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType} import org.apache.hudi.common.table.view.HoodieTableFileSystemView @@ -33,7 +33,7 @@ import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestUtils} import org.apache.hudi.common.util.PartitionPathEncodeUtils import org.apache.hudi.common.util.StringUtils.isNullOrEmpty -import org.apache.hudi.config.{HoodieStorageConfig, HoodieWriteConfig} +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieException import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.TimestampType @@ -48,8 +48,8 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, CsvSource, MethodSource, ValueSource} - import java.util.Properties + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.util.Random diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieInternalRowUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieInternalRowUtils.scala new file mode 100644 index 0000000000000..26a343d2ff613 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieInternalRowUtils.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi + +import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.testutils.HoodieClientTestUtils + +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{HoodieInternalRowUtils, Row, SparkSession} +import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} + +class TestHoodieInternalRowUtils extends FunSuite with Matchers with BeforeAndAfterAll { + + private var sparkSession: SparkSession = _ + + private val schema1 = StructType( + Array( + StructField("name", StringType), + StructField("age", IntegerType) + ) + ) + private val schema2 = StructType( + Array( + StructField("name1", StringType), + StructField("age1", IntegerType) + ) + ) + private val schemaMerge = StructType(schema1.fields ++ schema2.fields) + private val schema1WithMetaData = StructType(Array( + StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, StringType), + StructField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, StringType), + StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, StringType), + StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, StringType), + StructField(HoodieRecord.FILENAME_METADATA_FIELD, StringType), + StructField(HoodieRecord.OPERATION_METADATA_FIELD, StringType), + StructField(HoodieRecord.HOODIE_IS_DELETED_FIELD, BooleanType) + ) ++ schema1.fields) + + override protected def beforeAll(): Unit = { + // Initialize a local spark env + val jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(classOf[TestHoodieInternalRowUtils].getName)) + jsc.setLogLevel("ERROR") + sparkSession = SparkSession.builder.config(jsc.getConf).getOrCreate + } + + override protected def afterAll(): Unit = { + sparkSession.close() + } + + test("test rewrite") { + val data = sparkSession.sparkContext.parallelize(Seq(Row("like", 18, "like1", 181))) + val oldRow = sparkSession.createDataFrame(data, schemaMerge).queryExecution.toRdd.first() + val newRow1 = HoodieInternalRowUtils.rewriteRecord(oldRow, schemaMerge, schema1) + val newRow2 = HoodieInternalRowUtils.rewriteRecord(oldRow, schemaMerge, schema2) + assert(newRow1.get(0, StringType).toString.equals("like")) + assert(newRow1.get(1, IntegerType) == 18) + assert(newRow2.get(0, StringType).toString.equals("like1")) + assert(newRow2.get(1, IntegerType) == 181) + } + + test("test rewrite with nullable value") { + val data = sparkSession.sparkContext.parallelize(Seq(Row("like", 18))) + val oldRow = sparkSession.createDataFrame(data, schema1).queryExecution.toRdd.first() + val newRow = HoodieInternalRowUtils.rewriteRecord(oldRow, schema1, schemaMerge) + assert(newRow.get(0, StringType).toString.equals("like")) + assert(newRow.get(1, IntegerType) == 18) + assert(newRow.get(2, StringType) == null) + assert(newRow.get(3, IntegerType) == null) + } + + +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestStructTypeSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestStructTypeSchemaEvolutionUtils.scala new file mode 100644 index 0000000000000..31d36adbee5a3 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestStructTypeSchemaEvolutionUtils.scala @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi + +import java.nio.ByteBuffer +import java.util.{ArrayList, HashMap, Objects} + +import org.apache.avro.generic.GenericData +import org.apache.avro.{LogicalTypes, Schema} +import org.apache.hudi.avro.HoodieAvroUtils +import org.apache.hudi.internal.schema.Types +import org.apache.hudi.internal.schema.action.TableChanges +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter +import org.apache.hudi.internal.schema.utils.SchemaChangeUtils +import org.apache.hudi.testutils.HoodieClientTestUtils + +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.{HoodieInternalRowUtils, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} +import org.apache.spark.sql.types._ +import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} + +class TestStructTypeSchemaEvolutionUtils extends FunSuite with Matchers with BeforeAndAfterAll { + private var sparkSession: SparkSession = _ + + override protected def beforeAll(): Unit = { + // Initialize a local spark env + val jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(classOf[TestHoodieInternalRowUtils].getName)) + jsc.setLogLevel("ERROR") + sparkSession = SparkSession.builder.config(jsc.getConf).getOrCreate + } + + override protected def afterAll(): Unit = { + sparkSession.close() + } + + /** + * test record data type changes. + * int => long/float/double/string + * long => float/double/string + * float => double/String + * double => String/Decimal + * Decimal => Decimal/String + * String => date/decimal + * date => String + */ + test("test rewrite record with type changed") { + val avroSchema = new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"h0_record\",\"namespace\":\"hoodie.h0\",\"fields\"" + + ":[{\"name\":\"id\",\"type\":[\"null\",\"int\"],\"default\":null}," + + "{\"name\":\"comb\",\"type\":[\"null\",\"int\"],\"default\":null}," + + "{\"name\":\"com1\",\"type\":[\"null\",\"int\"],\"default\":null}," + + "{\"name\":\"col0\",\"type\":[\"null\",\"int\"],\"default\":null}," + + "{\"name\":\"col1\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"col11\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"col12\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"col2\",\"type\":[\"null\",\"float\"],\"default\":null}," + + "{\"name\":\"col21\",\"type\":[\"null\",\"float\"],\"default\":null}," + + "{\"name\":\"col3\",\"type\":[\"null\",\"double\"],\"default\":null}," + + "{\"name\":\"col31\",\"type\":[\"null\",\"double\"],\"default\":null}," + + "{\"name\":\"col4\",\"type\":[\"null\",{\"type\":\"fixed\",\"name\":\"fixed\",\"namespace\":\"hoodie.h0.h0_record.col4\"," + + "\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":4}],\"default\":null}," + + "{\"name\":\"col41\",\"type\":[\"null\",{\"type\":\"fixed\",\"name\":\"fixed\",\"namespace\":\"hoodie.h0.h0_record.col41\"," + + "\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":4}],\"default\":null}," + + "{\"name\":\"col5\",\"type\":[\"null\",\"string\"],\"default\":null}," + + "{\"name\":\"col51\",\"type\":[\"null\",\"string\"],\"default\":null}," + + "{\"name\":\"col6\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}," + + "{\"name\":\"col7\",\"type\":[\"null\",{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}],\"default\":null}," + + "{\"name\":\"col8\",\"type\":[\"null\",\"boolean\"],\"default\":null}," + + "{\"name\":\"col9\",\"type\":[\"null\",\"bytes\"],\"default\":null},{\"name\":\"par\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}]}") + // create a test record with avroSchema + val avroRecord = new GenericData.Record(avroSchema) + avroRecord.put("id", 1) + avroRecord.put("comb", 100) + avroRecord.put("com1", -100) + avroRecord.put("col0", 256) + avroRecord.put("col1", 1000L) + avroRecord.put("col11", -100L) + avroRecord.put("col12", 2000L) + avroRecord.put("col2", -5.001f) + avroRecord.put("col21", 5.001f) + avroRecord.put("col3", 12.999d) + avroRecord.put("col31", 9999.999d) + val currentDecimalType = avroSchema.getField("col4").schema.getTypes.get(1) + val bd = new java.math.BigDecimal("123.456").setScale(currentDecimalType.getLogicalType.asInstanceOf[LogicalTypes.Decimal].getScale) + avroRecord.put("col4", HoodieAvroUtils.DECIMAL_CONVERSION.toFixed(bd, currentDecimalType, currentDecimalType.getLogicalType)) + val currentDecimalType1 = avroSchema.getField("col41").schema.getTypes.get(1) + val bd1 = new java.math.BigDecimal("7890.456").setScale(currentDecimalType1.getLogicalType.asInstanceOf[LogicalTypes.Decimal].getScale) + avroRecord.put("col41", HoodieAvroUtils.DECIMAL_CONVERSION.toFixed(bd1, currentDecimalType1, currentDecimalType1.getLogicalType)) + avroRecord.put("col5", "2011-01-01") + avroRecord.put("col51", "199.342") + avroRecord.put("col6", 18987) + avroRecord.put("col7", 1640491505000000L) + avroRecord.put("col8", false) + val bb = ByteBuffer.wrap(Array[Byte](97, 48, 53)) + avroRecord.put("col9", bb) + assert(GenericData.get.validate(avroSchema, avroRecord)) + val internalSchema = AvroInternalSchemaConverter.convert(avroSchema) + // do change type operation + val updateChange = TableChanges.ColumnUpdateChange.get(internalSchema) + updateChange.updateColumnType("id", Types.LongType.get).updateColumnType("comb", Types.FloatType.get).updateColumnType("com1", Types.DoubleType.get).updateColumnType("col0", Types.StringType.get).updateColumnType("col1", Types.FloatType.get).updateColumnType("col11", Types.DoubleType.get).updateColumnType("col12", Types.StringType.get).updateColumnType("col2", Types.DoubleType.get).updateColumnType("col21", Types.StringType.get).updateColumnType("col3", Types.StringType.get).updateColumnType("col31", Types.DecimalType.get(18, 9)).updateColumnType("col4", Types.DecimalType.get(18, 9)).updateColumnType("col41", Types.StringType.get).updateColumnType("col5", Types.DateType.get).updateColumnType("col51", Types.DecimalType.get(18, 9)).updateColumnType("col6", Types.StringType.get) + val newSchema = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange) + val newAvroSchema = AvroInternalSchemaConverter.convert(newSchema, avroSchema.getName) + val newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, new HashMap[String, String]) + assert(GenericData.get.validate(newAvroSchema, newRecord)) + // Convert avro to internalRow + val structTypeSchema = HoodieInternalRowUtils.getCachedSchema(avroSchema) + val newStructTypeSchema = HoodieInternalRowUtils.getCachedSchema(newAvroSchema) + val row = AvroConversionUtils.createAvroToInternalRowConverter(avroSchema, structTypeSchema).apply(avroRecord).get + val newRowExpected = AvroConversionUtils.createAvroToInternalRowConverter(newAvroSchema, newStructTypeSchema) + .apply(newRecord).get + val newRowActual = HoodieInternalRowUtils.rewriteRecordWithNewSchema(row, structTypeSchema, newStructTypeSchema, new HashMap[String, String]) + internalRowCompare(newRowExpected, newRowActual, newStructTypeSchema) + } + + test("test rewrite nest record") { + val record = Types.RecordType.get(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.Field.get(1, true, "data", Types.StringType.get()), + Types.Field.get(2, true, "preferences", + Types.RecordType.get(Types.Field.get(5, false, "feature1", + Types.BooleanType.get()), Types.Field.get(6, true, "feature2", Types.BooleanType.get()))), + Types.Field.get(3, false, "doubles", Types.ArrayType.get(7, false, Types.DoubleType.get())), + Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get(), + Types.RecordType.get(Types.Field.get(10, false, "lat", Types.FloatType.get()), Types.Field.get(11, false, "long", Types.FloatType.get())), false)) + ) + val schema = AvroInternalSchemaConverter.convert(record, "test1") + val avroRecord = new GenericData.Record(schema) + GenericData.get.validate(schema, avroRecord) + avroRecord.put("id", 2) + avroRecord.put("data", "xs") + // fill record type + val preferencesRecord = new GenericData.Record(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1_preferences")) + preferencesRecord.put("feature1", false) + preferencesRecord.put("feature2", true) + assert(GenericData.get.validate(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1_preferences"), preferencesRecord)) + avroRecord.put("preferences", preferencesRecord) + // fill mapType + val locations = new HashMap[String, GenericData.Record] + val mapSchema = AvroInternalSchemaConverter.convert(record.field("locations").`type`.asInstanceOf[Types.MapType].valueType, "test1_locations") + val locationsValue: GenericData.Record = new GenericData.Record(mapSchema) + locationsValue.put("lat", 1.2f) + locationsValue.put("long", 1.4f) + val locationsValue1: GenericData.Record = new GenericData.Record(mapSchema) + locationsValue1.put("lat", 2.2f) + locationsValue1.put("long", 2.4f) + locations.put("key1", locationsValue) + locations.put("key2", locationsValue1) + avroRecord.put("locations", locations) + val doubles = new ArrayList[Double] + doubles.add(2.0d) + doubles.add(3.0d) + avroRecord.put("doubles", doubles) + // create newSchema + val newRecord = Types.RecordType.get(Types.Field.get(0, false, "id", Types.IntType.get), Types.Field.get(1, true, "data", Types.StringType.get), Types.Field.get(2, true, "preferences", Types.RecordType.get(Types.Field.get(5, false, "feature1", Types.BooleanType.get), Types.Field.get(5, true, "featurex", Types.BooleanType.get), Types.Field.get(6, true, "feature2", Types.BooleanType.get))), Types.Field.get(3, false, "doubles", Types.ArrayType.get(7, false, Types.DoubleType.get)), Types.Field.get(4, false, "locations", Types.MapType.get(8, 9, Types.StringType.get, Types.RecordType.get(Types.Field.get(10, true, "laty", Types.FloatType.get), Types.Field.get(11, false, "long", Types.FloatType.get)), false))) + val newAvroSchema = AvroInternalSchemaConverter.convert(newRecord, schema.getName) + val newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, new HashMap[String, String]) + // test the correctly of rewrite + assert(GenericData.get.validate(newAvroSchema, newAvroRecord)) + // Convert avro to internalRow + val structTypeSchema = HoodieInternalRowUtils.getCachedSchema(schema) + val newStructTypeSchema = HoodieInternalRowUtils.getCachedSchema(newAvroSchema) + val row = AvroConversionUtils.createAvroToInternalRowConverter(schema, structTypeSchema).apply(avroRecord).get + val newRowExpected = AvroConversionUtils.createAvroToInternalRowConverter(newAvroSchema, newStructTypeSchema).apply(newAvroRecord).get + val newRowActual = HoodieInternalRowUtils.rewriteRecordWithNewSchema(row, structTypeSchema, newStructTypeSchema, new HashMap[String, String]) + internalRowCompare(newRowExpected, newRowActual, newStructTypeSchema) + } + + private def internalRowCompare(expected: Any, actual: Any, schema: DataType): Unit = { + schema match { + case StructType(fields) => + val expectedRow = expected.asInstanceOf[InternalRow] + val actualRow = actual.asInstanceOf[InternalRow] + fields.zipWithIndex.foreach { case (field, i) => internalRowCompare(expectedRow.get(i, field.dataType), actualRow.get(i, field.dataType), field.dataType) } + case ArrayType(elementType, _) => + val expectedArray = expected.asInstanceOf[ArrayData].toSeq[Any](elementType) + val actualArray = actual.asInstanceOf[ArrayData].toSeq[Any](elementType) + if (expectedArray.size != actualArray.size) { + throw new AssertionError() + } else { + expectedArray.zip(actualArray).foreach { case (e1, e2) => internalRowCompare(e1, e2, elementType) } + } + case MapType(keyType, valueType, _) => + val expectedKeyArray = expected.asInstanceOf[MapData].keyArray() + val expectedValueArray = expected.asInstanceOf[MapData].valueArray() + val actualKeyArray = actual.asInstanceOf[MapData].keyArray() + val actualValueArray = actual.asInstanceOf[MapData].valueArray() + internalRowCompare(expectedKeyArray, actualKeyArray, ArrayType(keyType)) + internalRowCompare(expectedValueArray, actualValueArray, ArrayType(valueType)) + case StringType => if (checkNull(expected, actual) || !expected.toString.equals(actual.toString)) { + throw new AssertionError(String.format("%s is not equals %s", expected.toString, actual.toString)) + } + // TODO Verify after 'https://github.com/apache/hudi/pull/5907' merge + case BinaryType => if (checkNull(expected, actual) || !expected.asInstanceOf[Array[Byte]].sameElements(actual.asInstanceOf[Array[Byte]])) { + // throw new AssertionError(String.format("%s is not equals %s", expected.toString, actual.toString)) + } + case _ => if (!Objects.equals(expected, actual)) { + // throw new AssertionError(String.format("%s is not equals %s", expected.toString, actual.toString)) + } + } + } + + private def checkNull(left: Any, right: Any): Boolean = { + (left == null && right != null) || (left == null && right != null) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/model/TestHoodieRecordSerialization.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/model/TestHoodieRecordSerialization.scala new file mode 100644 index 0000000000000..d53d8e3743121 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/model/TestHoodieRecordSerialization.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model + +import org.apache.avro.generic.GenericRecord +import org.apache.hudi.AvroConversionUtils.{convertStructTypeToAvroSchema, createInternalRowToAvroConverter} +import org.apache.hudi.SparkAdapterSupport +import org.apache.hudi.client.model.HoodieInternalRow +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType +import org.apache.hudi.common.model.TestHoodieRecordSerialization.{OverwriteWithLatestAvroPayloadWithEquality, cloneUsingKryo, convertToAvroRecord, toUnsafeRow} +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness +import org.apache.spark.sql.{HoodieInternalRowUtils, Row} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.objects.SerializerSupport +import org.apache.spark.sql.catalyst.expressions.{GenericRowWithSchema, UnsafeRow} +import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.addMetaFields +import org.apache.spark.sql.types.{Decimal, StructType} +import org.apache.spark.unsafe.types.UTF8String +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test + +import java.nio.ByteBuffer +import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate} +import java.util.Objects + +class TestHoodieRecordSerialization extends SparkClientFunctionalTestHarness { + + private val rowSchema = StructType.fromDDL("a INT, b STRING, c DATE, d TIMESTAMP, e STRUCT") + + @Test + def testSparkRecord(): Unit = { + def routine(row: InternalRow, schema: StructType, serializedSize: Int): Unit = { + val record = row match { + case ur: UnsafeRow => new HoodieSparkRecord(ur) + case _ => new HoodieSparkRecord(row, schema) + } + + // Step 1: Serialize/de- original [[HoodieSparkRecord]] + val (cloned, originalBytes) = cloneUsingKryo(record) + + assertEquals(serializedSize, originalBytes.length) + // NOTE: That in case when original row isn't an instance of [[UnsafeRow]] + // it would be + // - Projected into [[UnsafeRow]] (prior to serialization by Kryo) + // - Re-constructed as [[UnsafeRow]] + row match { + case _: UnsafeRow => assertEquals(record, cloned) + case _ => + val convertedRecord = new HoodieSparkRecord(toUnsafeRow(row, schema)) + assertEquals(convertedRecord, cloned) + } + + // Step 2: Serialize the already cloned record, and assert that ser/de loop is lossless + val (_, clonedBytes) = cloneUsingKryo(cloned) + assertEquals(ByteBuffer.wrap(originalBytes), ByteBuffer.wrap(clonedBytes)) + } + + val row = Row(1, "test", Date.valueOf(LocalDate.of(2022, 10, 1)), + Timestamp.from(Instant.parse("2022-10-01T23:59:59.00Z")), Row(Decimal.apply(123, 3, 2))) + + val unsafeRow: UnsafeRow = toUnsafeRow(row, rowSchema) + val hoodieInternalRow = new HoodieInternalRow(new Array[UTF8String](5), unsafeRow, false) + + Seq( + (unsafeRow, rowSchema, 87), + (hoodieInternalRow, addMetaFields(rowSchema), 127) + ) foreach { case (row, schema, expectedSize) => routine(row, schema, expectedSize) } + } + + @Test + def testAvroRecords(): Unit = { + def routine(record: HoodieRecord[_], expectedSize: Int): Unit = { + // Step 1: Serialize/de- original [[HoodieRecord]] + val (cloned, originalBytes) = cloneUsingKryo(record) + + assertEquals(expectedSize, originalBytes.length) + assertEquals(record, cloned) + + // Step 2: Serialize the already cloned record, and assert that ser/de loop is lossless + val (_, clonedBytes) = cloneUsingKryo(cloned) + assertEquals(ByteBuffer.wrap(originalBytes), ByteBuffer.wrap(clonedBytes)) + } + + val row = new GenericRowWithSchema(Array(1, "test", Date.valueOf(LocalDate.of(2022, 10, 1)), + Timestamp.from(Instant.parse("2022-10-01T23:59:59.00Z")), Row(Decimal.apply(123, 3, 2))), rowSchema) + val avroRecord = convertToAvroRecord(row) + + val key = new HoodieKey("rec-key", "part-path") + + val legacyRecord = toLegacyAvroRecord(avroRecord, key) + val avroIndexedRecord = new HoodieAvroIndexedRecord(key, avroRecord) + + Seq( + (legacyRecord, 528), + (avroIndexedRecord, 389) + ) foreach { case (record, expectedSize) => routine(record, expectedSize) } + } + + @Test + def testEmptyRecord(): Unit = { + def routine(record: HoodieRecord[_], expectedSize: Int): Unit = { + // Step 1: Serialize/de- original [[HoodieRecord]] + val (cloned, originalBytes) = cloneUsingKryo(record) + + assertEquals(expectedSize, originalBytes.length) + assertEquals(record, cloned) + + // Step 2: Serialize the already cloned record, and assert that ser/de loop is lossless + val (_, clonedBytes) = cloneUsingKryo(cloned) + assertEquals(ByteBuffer.wrap(originalBytes), ByteBuffer.wrap(clonedBytes)) + } + + val key = new HoodieKey("rec-key", "part-path") + + Seq( + (new HoodieEmptyRecord[GenericRecord](key, HoodieOperation.INSERT, 1, HoodieRecordType.AVRO), 27), + (new HoodieEmptyRecord[GenericRecord](key, HoodieOperation.INSERT, 2, HoodieRecordType.SPARK), 27) + ) foreach { case (record, expectedSize) => routine(record, expectedSize) } + } + + + private def toLegacyAvroRecord(avroRecord: GenericRecord, key: HoodieKey): HoodieAvroRecord[OverwriteWithLatestAvroPayload] = { + val avroRecordPayload = new OverwriteWithLatestAvroPayloadWithEquality(avroRecord, 0) + val legacyRecord = new HoodieAvroRecord[OverwriteWithLatestAvroPayload](key, avroRecordPayload) + + legacyRecord + } +} + +object TestHoodieRecordSerialization { + + private def cloneUsingKryo[T](r: HoodieRecord[T]): (HoodieRecord[T], Array[Byte]) = { + val serializer = SerializerSupport.newSerializer(true) + + val buf = serializer.serialize(r) + val cloned: HoodieRecord[T] = serializer.deserialize(buf) + + val bytes = new Array[Byte](buf.remaining()) + buf.get(bytes) + + (cloned, bytes) + } + + private def toUnsafeRow(row: InternalRow, schema: StructType): UnsafeRow = { + val project = HoodieInternalRowUtils.getCachedUnsafeProjection(schema, schema) + project(row) + } + + private def toUnsafeRow(row: Row, schema: StructType): UnsafeRow = { + val encoder = SparkAdapterSupport.sparkAdapter.createSparkRowSerDe(schema) + val internalRow = encoder.serializeRow(row) + internalRow.asInstanceOf[UnsafeRow] + } + + private def convertToAvroRecord(row: Row): GenericRecord = { + val schema = convertStructTypeToAvroSchema(row.schema, "testRecord", "testNamespace") + + createInternalRowToAvroConverter(row.schema, schema, nullable = false) + .apply(toUnsafeRow(row, row.schema)) + } + + class OverwriteWithLatestAvroPayloadWithEquality(avroRecord: GenericRecord, _orderingVal: Comparable[_]) + extends OverwriteWithLatestAvroPayload(avroRecord, _orderingVal) { + override def equals(obj: Any): Boolean = + obj match { + case p: OverwriteWithLatestAvroPayloadWithEquality => + Objects.equals(ByteBuffer.wrap(this.recordBytes), ByteBuffer.wrap(p.recordBytes)) && + Objects.equals(this.orderingVal, p.orderingVal) + case _ => + false + } + + override def hashCode(): Int = Objects.hash(avroRecord, _orderingVal.asInstanceOf[AnyRef]) + } + +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index aeb2bbea38a28..b5d7f91dc8001 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -20,7 +20,7 @@ package org.apache.hudi.functional import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hudi.HoodieConversionUtils.toJavaOption import org.apache.hudi.QuickstartUtils.{convertToStringList, getQuickstartWriteConfigs} -import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig} import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.table.timeline.HoodieInstant import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} @@ -37,8 +37,8 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config import org.apache.hudi.metrics.Metrics import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.hudi.util.JFunction -import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, QuickstartUtils, ScalaAssertionSupport} -import org.apache.spark.sql._ +import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkRecordMerger, QuickstartUtils, ScalaAssertionSupport} +import org.apache.spark.sql.{HoodieInternalRowUtils, _} import org.apache.spark.sql.functions.{col, concat, lit, udf, when} import org.apache.spark.sql.hudi.HoodieSparkSessionExtension import org.apache.spark.sql.types._ @@ -47,10 +47,12 @@ import org.joda.time.format.DateTimeFormat import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{CsvSource, ValueSource} +import org.junit.jupiter.params.provider.{CsvSource, EnumSource} import java.sql.{Date, Timestamp} import java.util.function.Consumer +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -72,6 +74,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1" ) + val sparkOpts = Map(HoodieWriteConfig.MERGER_IMPLS.key -> classOf[HoodieSparkRecordMerger].getName) val verificationCol: String = "driver" val updatedVerificationVal: String = "driver_update" @@ -98,12 +101,16 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport System.gc() } - @Test def testShortNameStorage() { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testShortNameStorage(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // Insert Operation val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) inputDF.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) @@ -111,7 +118,11 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) } - @Test def testNoPrecombine() { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testNoPrecombine(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // Insert Operation val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) @@ -122,26 +133,30 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" - ) + ) ++ writeOpts inputDF.write.format("hudi") .options(commonOptsNoPreCombine) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) - spark.read.format("org.apache.hudi").load(basePath).count() + spark.read.format("org.apache.hudi").options(readOpts).load(basePath).count() } - @Test def testHoodieIsDeletedNonBooleanField() { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testHoodieIsDeletedNonBooleanField(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // Insert Operation val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) - val df = inputDF.withColumn(HoodieRecord.HOODIE_IS_DELETED, lit("abc")) + val df = inputDF.withColumn(HoodieRecord.HOODIE_IS_DELETED_FIELD, lit("abc")) // Should have failed since _hoodie_is_deleted is not a BOOLEAN data type assertThrows(classOf[HoodieException]) { df.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Overwrite) .save(basePath) } @@ -154,8 +169,11 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport * * For COW table, test the snapshot query mode and incremental query mode. */ - @Test - def testPrunePartitionForTimestampBasedKeyGenerator(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testPrunePartitionForTimestampBasedKeyGenerator(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val options = commonOpts ++ Map( "hoodie.compact.inline" -> "false", DataSourceWriteOptions.TABLE_TYPE.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, @@ -164,7 +182,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyy/MM/dd", Config.TIMESTAMP_TIMEZONE_FORMAT_PROP -> "GMT+8:00", Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP -> "yyyy-MM-dd" - ) + ) ++ writeOpts val dataGen1 = new HoodieTestDataGenerator(Array("2022-01-01")) val records1 = recordsToStrings(dataGen1.generateInserts("001", 20)).toList @@ -189,7 +207,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val commit2Time = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp // snapshot query - val snapshotQueryRes = spark.read.format("hudi").load(basePath) + val snapshotQueryRes = spark.read.format("hudi").options(readOpts).load(basePath) // TODO(HUDI-3204) we have to revert this to pre-existing behavior from 0.10 //assertEquals(snapshotQueryRes.where("partition = '2022-01-01'").count, 20) //assertEquals(snapshotQueryRes.where("partition = '2022-01-02'").count, 30) @@ -198,6 +216,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport // incremental query val incrementalQueryRes = spark.read.format("hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit1Time) .option(DataSourceReadOptions.END_INSTANTTIME.key, commit2Time) @@ -214,15 +233,18 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport * archival should kick in and 2 commits should be archived. If schema is valid, no exception will be thrown. If not, * NPE will be thrown. */ - @Test - def testArchivalWithBulkInsert(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testArchivalWithBulkInsert(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + var structType : StructType = null for (i <- 1 to 4) { val records = recordsToStrings(dataGen.generateInserts("%05d".format(i), 100)).toList val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) structType = inputDF.schema inputDF.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option("hoodie.keep.min.commits", "1") .option("hoodie.keep.max.commits", "2") .option("hoodie.cleaner.commits.retained", "0") @@ -243,19 +265,23 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertEquals(schema, actualSchema) } - @Test - def testCopyOnWriteDeletes(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testCopyOnWriteDeletes(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // Insert Operation val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Overwrite) .save(basePath) assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) val snapshotDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*/*/*") assertEquals(100, snapshotDF1.count()) @@ -263,21 +289,26 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2 , 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) val snapshotDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*/*/*") assertEquals(snapshotDF2.count(), 80) } - @Test def testOverWriteModeUseReplaceAction(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testOverWriteModeUseReplaceAction(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) @@ -285,7 +316,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val records2 = recordsToStrings(dataGen.generateInserts("002", 5)).toList val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) @@ -299,12 +330,15 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertEquals("replacecommit", commits(1)) } - @Test - def testReadPathsOnCopyOnWriteTable(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testReadPathsOnCopyOnWriteTable(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val records1 = dataGen.generateInsertsContainsAllPartitions("001", 20) val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) @@ -322,20 +356,21 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val records2 = dataGen.generateInsertsContainsAllPartitions("002", 20) val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) - val inputDF3 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2)) + val inputDF3 = spark.read.options(readOpts).json(spark.sparkContext.parallelize(recordsToStrings(records2), 2)) inputDF3.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) // Use bulk insert here to make sure the files have different file groups. .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) val hudiReadPathDF = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key(), instantTime) .option(DataSourceReadOptions.READ_PATHS.key, record1FilePaths) .load() @@ -344,11 +379,15 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertEquals(expectedCount, hudiReadPathDF.count()) } - @Test def testOverWriteTableModeUseReplaceAction(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testOverWriteTableModeUseReplaceAction(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) @@ -356,7 +395,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val records2 = recordsToStrings(dataGen.generateInserts("002", 5)).toList val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) @@ -370,12 +409,16 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertEquals("replacecommit", commits(1)) } - @Test def testOverWriteModeUseReplaceActionOnDisJointPartitions(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testOverWriteModeUseReplaceActionOnDisJointPartitions(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // step1: Write 5 records to hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH val records1 = recordsToStrings(dataGen.generateInsertsForPartition("001", 5, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) @@ -384,7 +427,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val records2 = recordsToStrings(dataGen.generateInsertsForPartition("002", 7, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).toList val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) @@ -393,12 +436,12 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val records3 = recordsToStrings(dataGen.generateInsertsForPartition("001", 6, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList val inputDF3 = spark.read.json(spark.sparkContext.parallelize(records3, 2)) inputDF3.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) - val allRecords = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*") + val allRecords = spark.read.format("org.apache.hudi").options(readOpts).load(basePath + "/*/*/*") allRecords.registerTempTable("tmpTable") spark.sql(String.format("select count(*) from tmpTable")).show() @@ -426,12 +469,16 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertEquals("replacecommit", commits(2)) } - @Test def testOverWriteTableModeUseReplaceActionOnDisJointPartitions(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testOverWriteTableModeUseReplaceActionOnDisJointPartitions(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // step1: Write 5 records to hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH val records1 = recordsToStrings(dataGen.generateInsertsForPartition("001", 5, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) @@ -440,12 +487,12 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val records2 = recordsToStrings(dataGen.generateInsertsForPartition("002", 7, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).toList val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) - val allRecords = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*") + val allRecords = spark.read.format("org.apache.hudi").options(readOpts).load(basePath + "/*/*/*") allRecords.registerTempTable("tmpTable") spark.sql(String.format("select count(*) from tmpTable")).show() @@ -476,7 +523,11 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertEquals("replacecommit", commits(1)) } - @Test def testDropInsertDup(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testDropInsertDup(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val insert1Cnt = 10 val insert2DupKeyCnt = 9 val insert2NewKeyCnt = 2 @@ -490,11 +541,12 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val records1 = recordsToStrings(inserts1).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) val hoodieROViewDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*/*/*") assertEquals(insert1Cnt, hoodieROViewDF1.count()) @@ -502,22 +554,28 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val records2 = recordsToStrings(inserts2Dup ++ inserts2New).toList val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.INSERT_DROP_DUPS.key, "true") .mode(SaveMode.Append) .save(basePath) val hoodieROViewDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*/*/*") assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate) val hoodieIncViewDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commitInstantTime1) .load(basePath) assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt) } - @Test def testComplexDataTypeWriteAndReadConsistency(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testComplexDataTypeWriteAndReadConsistency(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val schema = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, true) :: StructField("timeStampValue", TimestampType, true) :: StructField("dateValue", DateType, true) :: StructField("decimalValue", DataTypes.createDecimalType(15, 10), true) :: StructField("timestamp", IntegerType, true) @@ -529,11 +587,12 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val rdd = jsc.parallelize(records) val recordsDF = spark.createDataFrame(rdd, schema) recordsDF.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Overwrite) .save(basePath) val recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*") recordsReadDF.printSchema() recordsReadDF.schema.foreach(f => { @@ -549,11 +608,15 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport }) } - @Test def testWithAutoCommitOn(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testWithAutoCommitOn(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(HoodieWriteConfig.AUTO_COMMIT_ENABLE.key, "true") .mode(SaveMode.Overwrite) @@ -562,51 +625,60 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) } - private def getDataFrameWriter(keyGenerator: String): DataFrameWriter[Row] = { + private def getDataFrameWriter(keyGenerator: String, opts: Map[String, String]): DataFrameWriter[Row] = { val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) inputDF.write.format("hudi") - .options(commonOpts) + .options(opts) .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, keyGenerator) .mode(SaveMode.Overwrite) } - @Test def testSparkPartitionByWithCustomKeyGenerator(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testSparkPartitionByWithCustomKeyGenerator(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // Without fieldType, the default is SIMPLE - var writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName) + var writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, writeOpts) writer.partitionBy("current_ts") .mode(SaveMode.Overwrite) .save(basePath) var recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*") - assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("current_ts").cast("string")).count() == 0) + + assertEquals(0L, recordsReadDF.filter(col("_hoodie_partition_path") =!= col("current_ts").cast("string")).count()) // Specify fieldType as TIMESTAMP - writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName) + writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, writeOpts) writer.partitionBy("current_ts:TIMESTAMP") .option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS") .option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd") .mode(SaveMode.Overwrite) .save(basePath) recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*") val udf_date_format = udf((data: Long) => new DateTime(data).toString(DateTimeFormat.forPattern("yyyyMMdd"))) - assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= udf_date_format(col("current_ts"))).count() == 0) + + assertEquals(0L, recordsReadDF.filter(col("_hoodie_partition_path") =!= udf_date_format(col("current_ts"))).count()) // Mixed fieldType - writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName) + writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, writeOpts) writer.partitionBy("driver", "rider:SIMPLE", "current_ts:TIMESTAMP") .option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS") .option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd") .mode(SaveMode.Overwrite) .save(basePath) recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*/*") assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= concat(col("driver"), lit("/"), col("rider"), lit("/"), udf_date_format(col("current_ts")))).count() == 0) // Test invalid partitionKeyType - writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName) + writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, writeOpts) writer = writer.partitionBy("current_ts:DUMMY") .option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS") .option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd") @@ -619,19 +691,23 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport } } - @Test def testSparkPartitionByWithSimpleKeyGenerator() { + @Test + def testSparkPartitionByWithSimpleKeyGenerator() { + val (writeOpts, readOpts) = getWriterReaderOpts(HoodieRecordType.AVRO) + // Use the `driver` field as the partition key - var writer = getDataFrameWriter(classOf[SimpleKeyGenerator].getName) + var writer = getDataFrameWriter(classOf[SimpleKeyGenerator].getName, writeOpts) writer.partitionBy("driver") .mode(SaveMode.Overwrite) .save(basePath) var recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("driver")).count() == 0) // Use the `driver,rider` field as the partition key, If no such field exists, // the default value [[PartitionPathEncodeUtils#DEFAULT_PARTITION_PATH]] is used - writer = getDataFrameWriter(classOf[SimpleKeyGenerator].getName) + writer = getDataFrameWriter(classOf[SimpleKeyGenerator].getName, writeOpts) val t = assertThrows(classOf[Throwable]) { writer.partitionBy("driver", "rider") .save(basePath) @@ -640,27 +716,37 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertEquals("Single partition-path field is expected; provided (driver,rider)", getRootCause(t).getMessage) } - @Test def testSparkPartitionByWithComplexKeyGenerator() { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testSparkPartitionByWithComplexKeyGenerator(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // Use the `driver` field as the partition key - var writer = getDataFrameWriter(classOf[ComplexKeyGenerator].getName) + var writer = getDataFrameWriter(classOf[ComplexKeyGenerator].getName, writeOpts) writer.partitionBy("driver") .mode(SaveMode.Overwrite) .save(basePath) var recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("driver")).count() == 0) // Use the `driver`,`rider` field as the partition key - writer = getDataFrameWriter(classOf[ComplexKeyGenerator].getName) + writer = getDataFrameWriter(classOf[ComplexKeyGenerator].getName, writeOpts) writer.partitionBy("driver", "rider") .save(basePath) recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= concat(col("driver"), lit("/"), col("rider"))).count() == 0) } - @Test def testSparkPartitionByWithTimestampBasedKeyGenerator() { - val writer = getDataFrameWriter(classOf[TimestampBasedKeyGenerator].getName) + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testSparkPartitionByWithTimestampBasedKeyGenerator(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + + val writer = getDataFrameWriter(classOf[TimestampBasedKeyGenerator].getName, writeOpts) writer.partitionBy("current_ts") .option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS") .option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd") @@ -668,51 +754,68 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport .save(basePath) val recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*") val udf_date_format = udf((data: Long) => new DateTime(data).toString(DateTimeFormat.forPattern("yyyyMMdd"))) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= udf_date_format(col("current_ts"))).count() == 0) } - @Test def testSparkPartitionByWithGlobalDeleteKeyGenerator() { - val writer = getDataFrameWriter(classOf[GlobalDeleteKeyGenerator].getName) + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testSparkPartitionByWithGlobalDeleteKeyGenerator(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + + val writer = getDataFrameWriter(classOf[GlobalDeleteKeyGenerator].getName, writeOpts) writer.partitionBy("driver") .mode(SaveMode.Overwrite) .save(basePath) val recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0) } - @Test def testSparkPartitionByWithNonpartitionedKeyGenerator() { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testSparkPartitionByWithNonpartitionedKeyGenerator(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // Empty string column - var writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName) + var writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName, writeOpts) writer.partitionBy("") .mode(SaveMode.Overwrite) .save(basePath) var recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0) // Non-existent column - writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName) + writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName, writeOpts) writer.partitionBy("abc") .mode(SaveMode.Overwrite) .save(basePath) recordsReadDF = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0) } @ParameterizedTest - @CsvSource(Array("true,false", "true,true", "false,true", "false,false")) - def testQueryCOWWithBasePathAndFileIndex(partitionEncode: Boolean, isMetadataEnabled: Boolean): Unit = { + @CsvSource(Array( + "true,false,AVRO", "true,true,AVRO", "false,true,AVRO", "false,false,AVRO", + "true,false,SPARK", "true,true,SPARK", "false,true,SPARK", "false,false,SPARK" + )) + def testQueryCOWWithBasePathAndFileIndex(partitionEncode: Boolean, isMetadataEnabled: Boolean, recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val N = 20 // Test query with partition prune if URL_ENCODE_PARTITIONING has enable val records1 = dataGen.generateInsertsContainsAllPartitions("000", N) val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2)) inputDF1.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode) .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) @@ -723,6 +826,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15") // query the partition by filter val count1 = spark.read.format("hudi") + .options(readOpts) .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) .load(basePath) .filter("partition = '2016/03/15'") @@ -732,6 +836,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport // query the partition by path val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15" val count2 = spark.read.format("hudi") + .options(readOpts) .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) .load(basePath + s"/$partitionPath") .count() @@ -741,7 +846,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val records2 = dataGen.generateInsertsContainsAllPartitions("000", N + 1) val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2)) inputDF2.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode) .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) @@ -749,6 +854,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport .save(basePath) // Incremental query without "*" in path val hoodieIncViewDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commitInstantTime1) .load(basePath) @@ -774,33 +880,39 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport } @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testCopyOnWriteWithDroppedPartitionColumns(enableDropPartitionColumns: Boolean) { + @CsvSource(Array("true, AVRO", "false, AVRO", "true, SPARK", "false, SPARK")) + def testCopyOnWriteWithDroppedPartitionColumns(enableDropPartitionColumns: Boolean, recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val records1 = recordsToStrings(dataGen.generateInsertsContainsAllPartitions("000", 100)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key, enableDropPartitionColumns) .mode(SaveMode.Overwrite) .save(basePath) - val snapshotDF1 = spark.read.format("org.apache.hudi").load(basePath) + val snapshotDF1 = spark.read.format("org.apache.hudi").options(readOpts).load(basePath) assertEquals(snapshotDF1.count(), 100) assertEquals(3, snapshotDF1.select("partition").distinct().count()) } - @Test - def testHoodieIsDeletedCOW(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testHoodieIsDeletedCOW(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val numRecords = 100 val numRecordsToDelete = 2 val records0 = recordsToStrings(dataGen.generateInserts("000", numRecords)).toList val df0 = spark.read.json(spark.sparkContext.parallelize(records0, 2)) df0.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Overwrite) .save(basePath) val snapshotDF0 = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*/*/*") assertEquals(numRecords, snapshotDF0.count()) @@ -811,21 +923,25 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport "_hoodie_is_deleted" ) df2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Append) .save(basePath) val snapshotDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .load(basePath + "/*/*/*/*") assertEquals(numRecords - numRecordsToDelete, snapshotDF2.count()) } - @Test - def testWriteSmallPrecisionDecimalTable(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testWriteSmallPrecisionDecimalTable(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) .withColumn("shortDecimal", lit(new java.math.BigDecimal(s"2090.0000"))) // create decimalType(8, 4) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) @@ -833,11 +949,11 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport // update the value of shortDecimal val inputDF2 = inputDF1.withColumn("shortDecimal", lit(new java.math.BigDecimal(s"3090.0000"))) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) - val readResult = spark.read.format("hudi").load(basePath) + val readResult = spark.read.format("hudi").options(readOpts).load(basePath) assert(readResult.count() == 5) // compare the test result assertEquals(inputDF2.sort("_row_key").select("shortDecimal").collect().map(_.getDecimal(0).toPlainString).mkString(","), @@ -845,8 +961,10 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport } @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testPartitionColumnsProperHandling(useGlobbing: Boolean): Unit = { + @CsvSource(Array("true, AVRO", "false, AVRO", "true, SPARK", "false, SPARK")) + def testPartitionColumnsProperHandling(useGlobbing: Boolean, recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val _spark = spark import _spark.implicits._ @@ -854,7 +972,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport .toDF("id", "name", "age", "ts", "data_date") df.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option("hoodie.insert.shuffle.parallelism", "4") .option("hoodie.upsert.shuffle.parallelism", "4") .option("hoodie.bulkinsert.shuffle.parallelism", "2") @@ -879,7 +997,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport } // Case #1: Partition columns are read from the data file - val firstDF = spark.read.format("hudi").load(path) + val firstDF = spark.read.format("hudi").options(readOpts).load(path) assert(firstDF.count() == 2) @@ -903,6 +1021,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport // won't be able to infer partitioning properly if (!useGlobbing) { val secondDF = spark.read.format("hudi") + .options(readOpts) .option(DataSourceReadOptions.EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH.key, "true") .load(path) @@ -920,16 +1039,18 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport } } - @Test - def testSaveAsTableInDifferentModes(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testSaveAsTableInDifferentModes(recordType: HoodieRecordType): Unit = { val options = scala.collection.mutable.Map.empty ++ commonOpts ++ Map("path" -> basePath) + val (writeOpts, readOpts) = getWriterReaderOpts(recordType, options.toMap) // first use the Overwrite mode val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.Append) .saveAsTable("hoodie_test") @@ -938,28 +1059,28 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport .setBasePath(basePath) .setConf(spark.sessionState.newHadoopConf) .build() - assertEquals(spark.read.format("hudi").load(basePath).count(), 5) + assertEquals(spark.read.format("hudi").options(readOpts).load(basePath).count(), 5) // use the Append mode val records2 = recordsToStrings(dataGen.generateInserts("002", 6)).toList val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.Append) .saveAsTable("hoodie_test") - assertEquals(spark.read.format("hudi").load(basePath).count(), 11) + assertEquals(spark.read.format("hudi").options(readOpts).load(basePath).count(), 11) // use the Ignore mode val records3 = recordsToStrings(dataGen.generateInserts("003", 7)).toList val inputDF3 = spark.read.json(spark.sparkContext.parallelize(records3, 2)) inputDF3.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.Ignore) .saveAsTable("hoodie_test") // nothing to do for the ignore mode - assertEquals(spark.read.format("hudi").load(basePath).count(), 11) + assertEquals(spark.read.format("hudi").options(readOpts).load(basePath).count(), 11) // use the ErrorIfExists mode val records4 = recordsToStrings(dataGen.generateInserts("004", 8)).toList @@ -967,7 +1088,7 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport try { inputDF4.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.ErrorIfExists) .saveAsTable("hoodie_test") } catch { @@ -979,20 +1100,23 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport val inputDF5 = spark.read.json(spark.sparkContext.parallelize(records5, 2)) inputDF5.write.format("org.apache.hudi") .partitionBy("partition") - .options(options) + .options(writeOpts) .mode(SaveMode.Overwrite) .saveAsTable("hoodie_test") - assertEquals(spark.read.format("hudi").load(basePath).count(), 9) + assertEquals(spark.read.format("hudi").options(readOpts).load(basePath).count(), 9) } - @Test - def testMetricsReporterViaDataSource(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testMetricsReporterViaDataSource(recordType: HoodieRecordType): Unit = { + val (writeOpts, _) = getWriterReaderOpts(recordType, getQuickstartWriteConfigs.asScala.toMap) + val dataGenerator = new QuickstartUtils.DataGenerator() val records = convertToStringList(dataGenerator.generateInserts( 10)) val recordsRDD = spark.sparkContext.parallelize(records, 2) val inputDF = spark.read.json(sparkSession.createDataset(recordsRDD)(Encoders.STRING)) inputDF.write.format("hudi") - .options(getQuickstartWriteConfigs) + .options(writeOpts) .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "uuid") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionpath") .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts") @@ -1006,6 +1130,13 @@ class TestCOWDataSource extends HoodieClientTestBase with ScalaAssertionSupport assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) assertEquals(false, Metrics.isInitialized, "Metrics should be shutdown") } + + def getWriterReaderOpts(recordType: HoodieRecordType, opt: Map[String, String] = commonOpts): (Map[String, String], Map[String, String]) = { + recordType match { + case HoodieRecordType.SPARK => (opt ++ sparkOpts, sparkOpts) + case _ => (opt, Map.empty[String, String]) + } + } } object TestCOWDataSource { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala index 0056be33dcab7..731548693f309 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala @@ -23,14 +23,15 @@ import org.apache.hadoop.fs.{LocatedFileStatus, Path} import org.apache.hudi.ColumnStatsIndexSupport.composeIndexSchema import org.apache.hudi.DataSourceWriteOptions.{PRECOMBINE_FIELD, RECORDKEY_FIELD} import org.apache.hudi.HoodieConversionUtils.toProperties -import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig} import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.util.ParquetUtils -import org.apache.hudi.config.{HoodieStorageConfig, HoodieWriteConfig} +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.functional.TestColumnStatsIndex.ColumnStatsTestCase import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceWriteOptions} + import org.apache.spark.sql._ import org.apache.spark.sql.functions.typedLit import org.apache.spark.sql.types._ @@ -41,6 +42,7 @@ import org.junit.jupiter.params.provider.{Arguments, MethodSource, ValueSource} import java.math.BigInteger import java.sql.{Date, Timestamp} + import scala.collection.JavaConverters._ import scala.util.Random diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index 8c545a9b2d640..35b7792f0c36c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -20,9 +20,10 @@ package org.apache.hudi.functional import org.apache.hadoop.fs.Path import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieConversionUtils.toJavaOption +import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig} +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType} import org.apache.hudi.client.SparkRDDWriteClient -import org.apache.hudi.common.config.HoodieMetadataConfig -import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecordPayload, HoodieTableType, OverwriteWithLatestAvroPayload} +import org.apache.hudi.common.model.{HoodieRecordPayload, OverwriteWithLatestAvroPayload} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings @@ -35,18 +36,19 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config import org.apache.hudi.table.action.compact.CompactionTriggerStrategy import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase} import org.apache.hudi.util.JFunction -import org.apache.hudi.{DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, HoodieDataSourceHelpers, SparkDatasetMixin} +import org.apache.hudi.{DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkRecordMerger, SparkDatasetMixin} import org.apache.log4j.LogManager -import org.apache.spark.sql._ +import org.apache.spark.sql.{HoodieInternalRowUtils, _} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hudi.HoodieSparkSessionExtension import org.apache.spark.sql.types.BooleanType import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.CsvSource - +import org.junit.jupiter.params.provider.{CsvSource, EnumSource} import java.util.function.Consumer +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType +import org.apache.hudi.hadoop.config.HoodieRealtimeConfig import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters._ @@ -65,6 +67,10 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" ) + val sparkOpts = Map( + HoodieWriteConfig.MERGER_IMPLS.key -> classOf[HoodieSparkRecordMerger].getName, + HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> "parquet" + ) val verificationCol: String = "driver" val updatedVerificationVal: String = "driver_update" @@ -90,14 +96,21 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { JFunction.toJavaConsumer((receiver: SparkSessionExtensions) => new HoodieSparkSessionExtension().apply(receiver))) ) - @Test def testCount() { + @ParameterizedTest + @CsvSource(Array("AVRO, AVRO, avro", "AVRO, SPARK, parquet", "SPARK, AVRO, parquet", "SPARK, SPARK, parquet")) + def testCount(readType: HoodieRecordType, writeType: HoodieRecordType, logType: String) { + var (_, readOpts) = getWriterReaderOpts(readType) + var (writeOpts, _) = getWriterReaderOpts(writeType) + readOpts = readOpts ++ Map(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> logType) + writeOpts = writeOpts ++ Map(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> logType) + // First Operation: // Producing parquet files to three default partitions. // SNAPSHOT view on MOR table with parquet files only. val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).asScala val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) @@ -105,6 +118,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .save(basePath) assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) val hudiSnapshotDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated @@ -115,10 +129,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).asScala val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Append) .save(basePath) val hudiSnapshotDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(100, hudiSnapshotDF2.count()) // still 100, since we only updated @@ -131,6 +146,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // incremental view // base file only val hudiIncDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "000") .option(DataSourceReadOptions.END_INSTANTTIME.key, commit1Time) @@ -141,6 +157,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { hudiIncDF1.show(1) // log file only val hudiIncDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit1Time) .option(DataSourceReadOptions.END_INSTANTTIME.key, commit2Time) @@ -152,6 +169,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // base file + log file val hudiIncDF3 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "000") .option(DataSourceReadOptions.END_INSTANTTIME.key, commit2Time) @@ -163,6 +181,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // Test incremental query has no instant in range val emptyIncDF = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "000") .option(DataSourceReadOptions.END_INSTANTTIME.key, "001") @@ -171,6 +190,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // Unmerge val hudiSnapshotSkipMergeDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .option(DataSourceReadOptions.REALTIME_MERGE.key, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) .load(basePath + "/*/*/*/*") @@ -180,6 +200,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // Test Read Optimized Query on MOR table val hudiRODF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(100, hudiRODF2.count()) @@ -190,10 +211,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records3 = recordsToStrings(dataGen.generateUniqueUpdates("003", 50)).asScala val inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2)) inputDF3.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Append) .save(basePath) val hudiSnapshotDF3 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") // still 100, because we only updated the existing records @@ -207,6 +229,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // incremental query from commit2Time val hudiIncDF4 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit2Time) .load(basePath) @@ -215,6 +238,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // skip merge incremental view // including commit 2 and commit 3 val hudiIncDF4SkipMerge = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "000") .option(DataSourceReadOptions.REALTIME_MERGE.key, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) @@ -230,10 +254,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records4 = recordsToStrings(newDataGen.generateInserts("004", 100)).asScala val inputDF4: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records4, 2)) inputDF4.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Append) .save(basePath) val hudiSnapshotDF4 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") // 200, because we insert 100 records to a new partition @@ -243,6 +268,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // Incremental query, 50 from log file, 100 from base file of the new partition. val hudiIncDF5 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit2Time) .load(basePath) @@ -255,11 +281,12 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records5 = recordsToStrings(newDataGen.generateUniqueUpdates("005", 50)).asScala val inputDF5: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records5, 2)) inputDF5.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Append) .save(basePath) val commit5Time = HoodieDataSourceHelpers.latestCommit(fs, basePath) val hudiSnapshotDF5 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(200, hudiSnapshotDF5.count()) @@ -269,16 +296,18 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records6 = recordsToStrings(newDataGen.generateInserts("006", 2)).asScala val inputDF6: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records6, 2)) inputDF6.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option("hoodie.compact.inline", "true") .mode(SaveMode.Append) .save(basePath) val commit6Time = HoodieDataSourceHelpers.latestCommit(fs, basePath) val hudiSnapshotDF6 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/2020/01/10/*") assertEquals(102, hudiSnapshotDF6.count()) val hudiIncDF6 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit5Time) .option(DataSourceReadOptions.END_INSTANTTIME.key, commit6Time) @@ -289,14 +318,48 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { } @Test - def testPayloadDelete() { + def testSpill() { + val (writeOpts, readOpts) = getWriterReaderOpts(HoodieRecordType.SPARK) + + val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).asScala + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("org.apache.hudi") + .options(writeOpts) + .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) + .mode(SaveMode.Overwrite) + .save(basePath) + + val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).asScala + val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + inputDF2.write.format("org.apache.hudi") + .options(writeOpts) + .mode(SaveMode.Append) + .save(basePath) + + // Make force spill + spark.sparkContext.hadoopConfiguration.set(HoodieRealtimeConfig.COMPACTION_MEMORY_FRACTION_PROP, "0.00001") + val hudiSnapshotDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .load(basePath + "/*/*/*/*") + assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated + spark.sparkContext.hadoopConfiguration.set(HoodieRealtimeConfig.COMPACTION_MEMORY_FRACTION_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_MEMORY_FRACTION) + } + + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testPayloadDelete(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // First Operation: // Producing parquet files to three default partitions. // SNAPSHOT view on MOR table with parquet files only. val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).asScala val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) @@ -304,6 +367,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .save(basePath) assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) val hudiSnapshotDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated @@ -314,10 +378,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records2 = recordsToStrings(dataGen.generateUniqueDeleteRecords("002", 50)).asScala val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Append) .save(basePath) val hudiSnapshotDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(50, hudiSnapshotDF2.count()) // 50 records were deleted @@ -329,6 +394,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // unmerge query, skip the delete records val hudiSnapshotDF2Unmerge = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .option(DataSourceReadOptions.REALTIME_MERGE.key, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) .load(basePath + "/*/*/*/*") @@ -336,6 +402,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // incremental query, read 50 delete records from log file and get 0 count. val hudiIncDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit2Time) .load(basePath) @@ -347,23 +414,27 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records3 = recordsToStrings(dataGen.generateUniqueDeleteRecords("003", 50)).asScala val inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2)) inputDF3.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Append) .save(basePath) val hudiSnapshotDF3 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(0, hudiSnapshotDF3.count()) // 100 records were deleted, 0 record to load } - @Test - def testPrunedFiltered() { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testPrunedFiltered(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // First Operation: // Producing parquet files to three default partitions. // SNAPSHOT view on MOR table with parquet files only. // Overriding the partition-path field - val opts = commonOpts + (DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition_path") + val opts = writeOpts + (DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition_path") val hoodieRecords1 = dataGen.generateInserts("001", 100) @@ -377,6 +448,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .mode(SaveMode.Overwrite) .save(basePath) val hudiSnapshotDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString @@ -399,18 +471,22 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .mode(SaveMode.Append) .save(basePath) val hudiSnapshotDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") val hudiIncDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "000") .load(basePath) val hudiIncDF1Skipmerge = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.REALTIME_MERGE.key, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "000") .load(basePath) val hudiIncDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit1Time) .load(basePath) @@ -445,6 +521,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .mode(SaveMode.Append).save(basePath) val hudiSnapshotDF3 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") @@ -454,8 +531,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { assertEquals(0, hudiSnapshotDF3.filter("rider = 'rider-003'").count()) } - @Test - def testVectorizedReader() { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testVectorizedReader(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + spark.conf.set("spark.sql.parquet.enableVectorizedReader", true) assertTrue(spark.conf.get("spark.sql.parquet.enableVectorizedReader").toBoolean) // Vectorized Reader will only be triggered with AtomicType schema, @@ -464,13 +544,14 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records1 = recordsToStrings(dataGen.generateInsertsAsPerSchema("001", 100, schema)).asScala val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) val hudiSnapshotDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(100, hudiSnapshotDF1.count()) @@ -479,10 +560,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .asScala val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Append) .save(basePath) val hudiSnapshotDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(100, hudiSnapshotDF2.count()) @@ -500,7 +582,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { hudiSnapshotDF2.show(1) } - @Test def testNoPrecombine() { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testNoPrecombine(recordType: HoodieRecordType) { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // Insert Operation val records = recordsToStrings(dataGen.generateInserts("000", 100)).asScala val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) @@ -511,7 +597,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" - ) + ) ++ writeOpts inputDF.write.format("hudi") .options(commonOptsNoPreCombine) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) @@ -519,43 +605,46 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .mode(SaveMode.Overwrite) .save(basePath) - spark.read.format("org.apache.hudi").load(basePath).count() + spark.read.format("org.apache.hudi").options(readOpts).load(basePath).count() } - @Test - def testPreCombineFiledForReadMOR(): Unit = { - writeData((1, "a0", 10, 100, false)) - checkAnswer((1, "a0", 10, 100, false)) + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testPreCombineFiledForReadMOR(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + + writeData((1, "a0", 10, 100, false), writeOpts) + checkAnswer((1, "a0", 10, 100, false), readOpts) - writeData((1, "a0", 12, 99, false)) + writeData((1, "a0", 12, 99, false), writeOpts) // The value has not update, because the version 99 < 100 - checkAnswer((1, "a0", 10, 100, false)) + checkAnswer((1, "a0", 10, 100, false), readOpts) - writeData((1, "a0", 12, 101, false)) + writeData((1, "a0", 12, 101, false), writeOpts) // The value has update - checkAnswer((1, "a0", 12, 101, false)) + checkAnswer((1, "a0", 12, 101, false), readOpts) - writeData((1, "a0", 14, 98, false)) + writeData((1, "a0", 14, 98, false), writeOpts) // Latest value should be ignored if preCombine honors ordering - checkAnswer((1, "a0", 12, 101, false)) + checkAnswer((1, "a0", 12, 101, false), readOpts) - writeData((1, "a0", 16, 97, true)) + writeData((1, "a0", 16, 97, true), writeOpts) // Ordering value will be honored, the delete record is considered as obsolete // because it has smaller version number (97 < 101) - checkAnswer((1, "a0", 12, 101, false)) + checkAnswer((1, "a0", 12, 101, false), readOpts) - writeData((1, "a0", 18, 96, false)) + writeData((1, "a0", 18, 96, false), writeOpts) // Ordering value will be honored, the data record is considered as obsolete // because it has smaller version number (96 < 101) - checkAnswer((1, "a0", 12, 101, false)) + checkAnswer((1, "a0", 12, 101, false), readOpts) } - private def writeData(data: (Int, String, Int, Int, Boolean)): Unit = { + private def writeData(data: (Int, String, Int, Int, Boolean), opts: Map[String, String]): Unit = { val _spark = spark import _spark.implicits._ val df = Seq(data).toDF("id", "name", "value", "version", "_hoodie_is_deleted") df.write.format("org.apache.hudi") - .options(commonOpts) + .options(opts) // use DefaultHoodieRecordPayload here .option(PAYLOAD_CLASS_NAME.key, classOf[DefaultHoodieRecordPayload].getCanonicalName) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) @@ -567,8 +656,9 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .save(basePath) } - private def checkAnswer(expect: (Int, String, Int, Int, Boolean)): Unit = { + private def checkAnswer(expect: (Int, String, Int, Int, Boolean), opts: Map[String, String]): Unit = { val readDf = spark.read.format("org.apache.hudi") + .options(opts) .load(basePath + "/*") if (expect._5) { if (!readDf.isEmpty) { @@ -603,14 +693,19 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { } @ParameterizedTest - @CsvSource(Array("true,false", "true,true", "false,true", "false,false")) - def testQueryMORWithBasePathAndFileIndex(partitionEncode: Boolean, isMetadataEnabled: Boolean): Unit = { + @CsvSource(Array( + "true,false,AVRO", "true,true,AVRO", "false,true,AVRO", "false,false,AVRO", + "true,false,SPARK", "true,true,SPARK", "false,true,SPARK", "false,false,SPARK" + )) + def testQueryMORWithBasePathAndFileIndex(partitionEncode: Boolean, isMetadataEnabled: Boolean, recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val N = 20 // Test query with partition prune if URL_ENCODE_PARTITIONING has enable val records1 = dataGen.generateInsertsContainsAllPartitions("000", N) val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1).asScala, 2)) inputDF1.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) .option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode) @@ -622,6 +717,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15") // query the partition by filter val count1 = spark.read.format("hudi") + .options(readOpts) .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) .load(basePath) .filter("partition = '2016/03/15'") @@ -631,6 +727,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // query the partition by path val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15" val count2 = spark.read.format("hudi") + .options(readOpts) .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) .load(basePath + s"/$partitionPath") .count() @@ -640,7 +737,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records2 = dataGen.generateInsertsContainsAllPartitions("000", N + 1) val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2).asScala, 2)) inputDF2.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) .option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode) @@ -649,6 +746,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .save(basePath) // Incremental query without "*" in path val hoodieIncViewDF1 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commitInstantTime1) .load(basePath) @@ -656,8 +754,13 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { } @ParameterizedTest - @CsvSource(Array("true, false", "false, true", "false, false", "true, true")) - def testMORPartitionPrune(partitionEncode: Boolean, hiveStylePartition: Boolean): Unit = { + @CsvSource(Array( + "true, false, AVRO", "false, true, AVRO", "false, false, AVRO", "true, true, AVRO", + "true, false, SPARK", "false, true, SPARK", "false, false, SPARK", "true, true, SPARK" + )) + def testMORPartitionPrune(partitionEncode: Boolean, hiveStylePartition: Boolean, recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val partitions = Array("2021/03/01", "2021/03/02", "2021/03/03", "2021/03/04", "2021/03/05") val newDataGen = new HoodieTestDataGenerator(partitions) val records1 = newDataGen.generateInsertsContainsAllPartitions("000", 100).asScala @@ -666,7 +769,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val partitionCounts = partitions.map(p => p -> records1.count(r => r.getPartitionPath == p)).toMap inputDF1.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) .option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode) @@ -675,42 +778,49 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .save(basePath) val count1 = spark.read.format("hudi") + .options(readOpts) .load(basePath) .filter("partition = '2021/03/01'") .count() assertEquals(partitionCounts("2021/03/01"), count1) val count2 = spark.read.format("hudi") + .options(readOpts) .load(basePath) .filter("partition > '2021/03/01' and partition < '2021/03/03'") .count() assertEquals(partitionCounts("2021/03/02"), count2) val count3 = spark.read.format("hudi") + .options(readOpts) .load(basePath) .filter("partition != '2021/03/01'") .count() assertEquals(records1.size - partitionCounts("2021/03/01"), count3) val count4 = spark.read.format("hudi") + .options(readOpts) .load(basePath) .filter("partition like '2021/03/03%'") .count() assertEquals(partitionCounts("2021/03/03"), count4) val count5 = spark.read.format("hudi") + .options(readOpts) .load(basePath) .filter("partition like '%2021/03/%'") .count() assertEquals(records1.size, count5) val count6 = spark.read.format("hudi") + .options(readOpts) .load(basePath) .filter("partition = '2021/03/01' or partition = '2021/03/05'") .count() assertEquals(partitionCounts("2021/03/01") + partitionCounts("2021/03/05"), count6) val count7 = spark.read.format("hudi") + .options(readOpts) .load(basePath) .filter("substr(partition, 9, 10) = '03'") .count() @@ -718,13 +828,16 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { assertEquals(partitionCounts("2021/03/03"), count7) } - @Test - def testReadPathsForMergeOnReadTable(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testReadPathsForMergeOnReadTable(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + // Paths only baseFiles val records1 = dataGen.generateInserts("001", 100) val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1).asScala, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) @@ -738,10 +851,10 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records2 = dataGen.generateUniqueDeleteRecords("002", 100) val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2).asScala, 2)) inputDF2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .mode(SaveMode.Append) .save(basePath) - val hudiReadPathDF1 = spark.read.format("org.apache.hudi") + val hudiReadPathDF1 = spark.read.options(readOpts).format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .option(DataSourceReadOptions.READ_PATHS.key, baseFilePath) .load() @@ -757,6 +870,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val readPaths = baseFilePath + "," + logFilePath val hudiReadPathDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .option(DataSourceReadOptions.READ_PATHS.key, readPaths) .load() @@ -764,13 +878,16 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { assertEquals(0, hudiReadPathDF2.count()) } - @Test - def testReadPathsForOnlyLogFiles(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testReadPathsForOnlyLogFiles(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + initMetaClient(HoodieTableType.MERGE_ON_READ) val records1 = dataGen.generateInsertsContainsAllPartitions("000", 20) val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1).asScala, 2)) inputDF1.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) // Use InMemoryIndex to generate log only mor table. @@ -788,7 +905,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val records2 = dataGen.generateInsertsContainsAllPartitions("000", 20) val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2).asScala, 2)) inputDF2.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) // Use InMemoryIndex to generate log only mor table. @@ -800,7 +917,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val expectedCount1 = records1.asScala.count(record => record.getPartitionPath == dataGen.getPartitionPaths.head) - val hudiReadPathDF = spark.read.format("org.apache.hudi") + val hudiReadPathDF = spark.read.options(readOpts).format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .option(DataSourceReadOptions.READ_PATHS.key, logFilePath) .load() @@ -808,13 +925,16 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { assertEquals(expectedCount1, hudiReadPathDF.count()) } - @Test - def testReadLogOnlyMergeOnReadTable(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testReadLogOnlyMergeOnReadTable(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + initMetaClient(HoodieTableType.MERGE_ON_READ) val records1 = dataGen.generateInsertsContainsAllPartitions("000", 20) val inputDF = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1).asScala, 2)) inputDF.write.format("hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) // Use InMemoryIndex to generate log only mor table. @@ -826,15 +946,18 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // Test read logs only mor table with glob paths. assertEquals(20, spark.read.format("hudi").load(basePath + "/*/*/*/*").count()) // Test read log only mor table. - assertEquals(20, spark.read.format("hudi").load(basePath).count()) + assertEquals(20, spark.read.format("hudi").options(readOpts).load(basePath).count()) } - @Test - def testTempFilesCleanForClustering(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testTempFilesCleanForClustering(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).asScala val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key(), DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) // option for clustering @@ -847,8 +970,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { assertEquals(true, fs.listStatus(tempPath).isEmpty) } - @Test - def testClusteringOnNullableColumn(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testClusteringOnNullableColumn(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).asScala val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) .withColumn("cluster_id", when(expr("end_lon < 0.2 "), lit(null).cast("string")) @@ -856,7 +982,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .withColumn("struct_cluster_col", when(expr("end_lon < 0.1"), lit(null)) .otherwise(struct(col("cluster_id"), col("_row_key")))) inputDF1.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key(), DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) // option for clustering @@ -869,15 +995,18 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .save(basePath) } - @Test - def testHoodieIsDeletedMOR(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testHoodieIsDeletedMOR(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val numRecords = 100 val numRecordsToDelete = 2 val schema = HoodieTestDataGenerator.SHORT_TRIP_SCHEMA val records0 = recordsToStrings(dataGen.generateInsertsAsPerSchema("000", numRecords, schema)).asScala val inputDF0 = spark.read.json(spark.sparkContext.parallelize(records0, 2)) inputDF0.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option("hoodie.compact.inline", "false") .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) @@ -885,6 +1014,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .save(basePath) val snapshotDF0 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(numRecords, snapshotDF0.count()) @@ -897,12 +1027,13 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { "_hoodie_is_deleted" ) df2.write.format("org.apache.hudi") - .options(commonOpts) + .options(writeOpts) .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) .mode(SaveMode.Append) .save(basePath) val snapshotDF2 = spark.read.format("org.apache.hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(numRecords - numRecordsToDelete, snapshotDF2.count()) @@ -915,8 +1046,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { * * For MOR table, test all the three query modes. */ - @Test - def testPrunePartitionForTimestampBasedKeyGenerator(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testPrunePartitionForTimestampBasedKeyGenerator(recordType: HoodieRecordType): Unit = { + val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + val options = commonOpts ++ Map( "hoodie.compact.inline" -> "false", DataSourceWriteOptions.TABLE_TYPE.key -> DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, @@ -925,7 +1059,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyy/MM/dd", Config.TIMESTAMP_TIMEZONE_FORMAT_PROP -> "GMT+8:00", Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP -> "yyyy-MM-dd" - ) + ) ++ writeOpts val dataGen1 = new HoodieTestDataGenerator(Array("2022-01-01")) val records1 = recordsToStrings(dataGen1.generateInserts("001", 50)).asScala @@ -958,7 +1092,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { val commit3Time = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp // snapshot query - val snapshotQueryRes = spark.read.format("hudi").load(basePath) + val snapshotQueryRes = spark.read.format("hudi").options(readOpts).load(basePath) assertEquals(snapshotQueryRes.where(s"_hoodie_commit_time = '$commit1Time'").count, 50) assertEquals(snapshotQueryRes.where(s"_hoodie_commit_time = '$commit2Time'").count, 40) assertEquals(snapshotQueryRes.where(s"_hoodie_commit_time = '$commit3Time'").count, 20) @@ -968,6 +1102,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // read_optimized query val readOptimizedQueryRes = spark.read.format("hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) .load(basePath) // TODO(HUDI-3204) we have to revert this to pre-existing behavior from 0.10 @@ -978,6 +1113,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { // incremental query val incrementalQueryRes = spark.read.format("hudi") + .options(readOpts) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit2Time) .option(DataSourceReadOptions.END_INSTANTTIME.key, commit3Time) @@ -1006,8 +1142,9 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { * * The read-optimized query should read `fg1_dc1.parquet` only in this case. */ - @Test - def testReadOptimizedQueryAfterInflightCompactionAndCompletedDeltaCommit(): Unit = { + @ParameterizedTest + @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) + def testReadOptimizedQueryAfterInflightCompactionAndCompletedDeltaCommit(recordType: HoodieRecordType): Unit = { val (tableName, tablePath) = ("hoodie_mor_ro_read_test_table", s"${basePath}_mor_test_table") val precombineField = "col3" val recordKeyField = "key" @@ -1024,6 +1161,8 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { "hoodie.insert.shuffle.parallelism" -> "1", "hoodie.upsert.shuffle.parallelism" -> "1") + val (writeOpts, readOpts) = getWriterReaderOpts(recordType, options) + // First batch with all inserts // Deltacommit1 (DC1, completed), writing file group 1 (fg1) // fg1_dc1.parquet written to storage @@ -1033,7 +1172,7 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .withColumn(dataField, expr(recordKeyField + " + 1000")) firstDf.write.format("hudi") - .options(options) + .options(writeOpts) .mode(SaveMode.Overwrite) .save(tablePath) @@ -1046,10 +1185,10 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .withColumn(dataField, expr(recordKeyField + " + 2000")) secondDf.write.format("hudi") - .options(options) + .options(writeOpts) .mode(SaveMode.Append).save(tablePath) - val compactionOptions = options ++ Map( + val compactionOptions = writeOpts ++ Map( HoodieCompactionConfig.INLINE_COMPACT_TRIGGER_STRATEGY.key -> CompactionTriggerStrategy.NUM_COMMITS.name, HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key -> "1", DataSourceWriteOptions.ASYNC_COMPACT_ENABLE.key -> "false", @@ -1079,11 +1218,12 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { .withColumn(dataField, expr(recordKeyField + " + 3000")) thirdDf.write.format("hudi") - .options(options) + .options(writeOpts) .mode(SaveMode.Append).save(tablePath) // Read-optimized query on MOR val roDf = spark.read.format("org.apache.hudi") + .options(readOpts) .option( DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) @@ -1095,4 +1235,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin { 1000L, roDf.where(col(recordKeyField) === 0).select(dataField).collect()(0).getLong(0)) } + + def getWriterReaderOpts(recordType: HoodieRecordType, opt: Map[String, String] = commonOpts): (Map[String, String], Map[String, String]) = { + recordType match { + case HoodieRecordType.SPARK => (opt ++ sparkOpts, sparkOpts) + case _ => (opt, Map.empty[String, String]) + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala index 02e0ee6dfd9bc..9942132aba807 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala @@ -20,6 +20,7 @@ package org.apache.hudi.functional import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.model.HoodieAvroRecordMerger import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig @@ -27,7 +28,6 @@ import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.spark.sql.SaveMode import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{Tag, Test} - import scala.collection.JavaConverters._ @Tag("functional") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala index 00ab7091445db..214b725641764 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala @@ -19,11 +19,11 @@ package org.apache.hudi.functional import org.apache.avro.Schema import org.apache.hudi.HoodieBaseRelation.projectSchema -import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig} import org.apache.hudi.common.model.{HoodieRecord, OverwriteNonDefaultsWithLatestAvroPayload} import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.testutils.{HadoopMapRedUtils, HoodieTestDataGenerator} -import org.apache.hudi.config.{HoodieStorageConfig, HoodieWriteConfig} +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.keygen.NonpartitionedKeyGenerator import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, DefaultSource, HoodieBaseRelation, HoodieSparkUtils, HoodieUnsafeRDD} @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.{Dataset, HoodieUnsafeUtils, Row, SaveMode} import org.junit.jupiter.api.Assertions.{assertEquals, fail} import org.junit.jupiter.api.{Disabled, Tag, Test} - import scala.collection.JavaConverters._ @Tag("functional") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala index d4651f9ab7880..3908801f11c93 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala @@ -25,12 +25,14 @@ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.HoodieTimeline import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestTable} +import org.apache.hudi.common.config.HoodieStorageConfig import org.apache.hudi.common.util.{CollectionUtils, CommitUtils} -import org.apache.hudi.config.{HoodieClusteringConfig, HoodieCompactionConfig, HoodieStorageConfig, HoodieWriteConfig} +import org.apache.hudi.config.{HoodieClusteringConfig, HoodieCompactionConfig, HoodieWriteConfig} import org.apache.hudi.exception.TableNotFoundException import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSinkCheckpoint} import org.apache.log4j.LogManager + import org.apache.spark.sql._ import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.sql.types.StructType @@ -44,6 +46,8 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} +import org.apache.hudi.common.config.HoodieStorageConfig + /** * Basic tests on the spark datasource for structured streaming sink */ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala index 08d0e5b3372b0..7ad7ec1024778 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala @@ -29,17 +29,13 @@ import org.apache.hudi.common.table.timeline.HoodieInstant import org.apache.hudi.common.testutils.RawTripTestPayload import org.apache.hudi.config.{HoodieCleanConfig, HoodieWriteConfig} import org.apache.hudi.testutils.HoodieClientTestBase - import org.apache.avro.Schema import org.apache.avro.generic.{GenericRecord, IndexedRecord} - import org.apache.hadoop.fs.Path - +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import org.apache.spark.sql.{DataFrame, SparkSession} - import org.junit.jupiter.api.{AfterEach, BeforeEach} -import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertNull, assertTrue} - +import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertNull} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -128,11 +124,11 @@ abstract class HoodieCDCTestBase extends HoodieClientTestBase { blocks.toList } - protected def readCDCLogFile(relativeLogFile: String, cdcSchema: Schema): List[IndexedRecord] = { - val records = scala.collection.mutable.ListBuffer.empty[IndexedRecord] + protected def readCDCLogFile(relativeLogFile: String, cdcSchema: Schema): List[HoodieRecord[_]] = { + val records = scala.collection.mutable.ListBuffer.empty[HoodieRecord[_]] val blocks = getCDCBlocks(relativeLogFile, cdcSchema) blocks.foreach { block => - records.addAll(block.getRecordIterator.asScala.toList) + records.addAll(block.getRecordIterator[IndexedRecord](HoodieRecordType.AVRO).asScala.toList) } records.toList } @@ -140,18 +136,18 @@ abstract class HoodieCDCTestBase extends HoodieClientTestBase { protected def checkCDCDataForInsertOrUpdate(cdcSupplementalLoggingMode: String, cdcSchema: Schema, dataSchema: Schema, - cdcRecords: Seq[IndexedRecord], + cdcRecords: Seq[HoodieRecord[_]], newHoodieRecords: java.util.List[HoodieRecord[_]], op: HoodieCDCOperation): Unit = { - val cdcRecord = cdcRecords.head.asInstanceOf[GenericRecord] + val cdcRecord = cdcRecords.head.getData.asInstanceOf[GenericRecord] // check schema assertEquals(cdcRecord.getSchema, cdcSchema) if (cdcSupplementalLoggingMode == "cdc_op_key") { // check record key - assert(cdcRecords.map(_.get(1).toString).sorted == newHoodieRecords.map(_.getKey.getRecordKey).sorted) + assert(cdcRecords.map(_.getData.asInstanceOf[GenericRecord].get(1).toString).sorted == newHoodieRecords.map(_.getKey.getRecordKey).sorted) } else if (cdcSupplementalLoggingMode == "cdc_data_before") { // check record key - assert(cdcRecords.map(_.get(1).toString).sorted == newHoodieRecords.map(_.getKey.getRecordKey).sorted) + assert(cdcRecords.map(_.getData.asInstanceOf[GenericRecord].get(1).toString).sorted == newHoodieRecords.map(_.getKey.getRecordKey).sorted) // check before if (op == HoodieCDCOperation.INSERT) { assertNull(cdcRecord.get("before")) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala index 6237e223646f3..bc90485380613 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala @@ -18,6 +18,7 @@ package org.apache.hudi.functional.cdc +import org.apache.avro.generic.GenericRecord import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.common.table.cdc.{HoodieCDCOperation, HoodieCDCSupplementalLoggingMode, HoodieCDCUtils} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} @@ -278,8 +279,8 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { val cdcDataFromCDCLogFile2 = getCDCLogFile(instant2).flatMap(readCDCLogFile(_, cdcSchema)) assertEquals(cdcDataFromCDCLogFile2.size, 50) // check op - assertEquals(cdcDataFromCDCLogFile2.count(r => r.get(0).toString == "u"), 30) - assertEquals(cdcDataFromCDCLogFile2.count(r => r.get(0).toString == "i"), 20) + assertEquals(cdcDataFromCDCLogFile2.count(r => r.getData.asInstanceOf[GenericRecord].get(0).toString == "u"), 30) + assertEquals(cdcDataFromCDCLogFile2.count(r => r.getData.asInstanceOf[GenericRecord].get(0).toString == "i"), 20) val commitTime2 = instant2.getTimestamp var currentSnapshotData = spark.read.format("hudi").load(basePath) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/ReadAndWriteWithoutAvroBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/ReadAndWriteWithoutAvroBenchmark.scala new file mode 100644 index 0000000000000..3547e42148d9a --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/ReadAndWriteWithoutAvroBenchmark.scala @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.hadoop.fs.Path +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.common.config.HoodieStorageConfig +import org.apache.hudi.common.model.HoodieAvroRecordMerger +import org.apache.hudi.config.HoodieCompactionConfig +import org.apache.hudi.{HoodieSparkRecordMerger, HoodieSparkUtils} + +import org.apache.spark.SparkConf +import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hudi.HoodieSparkSessionExtension +import org.apache.spark.sql.{DataFrame, SparkSession} + +object ReadAndWriteWithoutAvroBenchmark extends HoodieBenchmarkBase { + + protected val spark: SparkSession = getSparkSession + private val avroTable = "avro_merger_table" + private val sparkTable = "spark_merger_table" + + def getSparkSession: SparkSession = SparkSession + .builder() + .master("local[4]") + .appName(this.getClass.getCanonicalName) + .withExtensions(new HoodieSparkSessionExtension) + .config("spark.driver.memory", "4G") + .config("spark.executor.memory", "4G") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("hoodie.insert.shuffle.parallelism", "2") + .config("hoodie.upsert.shuffle.parallelism", "2") + .config("hoodie.delete.shuffle.parallelism", "2") + .config("spark.sql.session.timeZone", "CTT") + .config(sparkConf()) + .getOrCreate() + + def sparkConf(): SparkConf = { + val sparkConf = new SparkConf() + if (HoodieSparkUtils.gteqSpark3_2) { + sparkConf.set("spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.hudi.catalog.HoodieCatalog") + } + sparkConf + } + + private def createComplexDataFrame(rowNum: Long): DataFrame = { + var df = spark.range(0, rowNum).toDF("id") + .withColumn("t1", lit(1)) + .withColumn("d1", lit(12.99d)) + .withColumn("s1", lit("s1")) + .withColumn("s2", lit("s2")) + .withColumn("s3", lit("s3")) + for (i <- 0 to 1) { + df = df.withColumn(s"struct$i", struct(col("s1").as("st1"), col("s2").as("st2"), col("s3").as("st3"))) + .withColumn(s"map$i", map(col("s1"), col("s2"))) + .withColumn(s"array$i", array(col("s1"))) + } + df + } + + private def prepareHoodieTable(tableName: String, path: String, tableType: String, mergerImpl: String, df: DataFrame): Unit = { + df.collect() + df.createOrReplaceTempView("input_df") + if (spark.catalog.tableExists(tableName)) { + spark.sql(s"drop table if exists $tableName") + } + spark.sql(s"set ${HoodieWriteConfig.MERGER_IMPLS.key} = $mergerImpl") + spark.sql( + s""" + |create table $tableName( + |id long, + |t1 int, + |d1 double, + |s1 string, + |s2 string, + |s3 string, + |struct0 struct, + |map0 map, + |array0 array, + |struct1 struct, + |map1 map, + |array1 array + |) using hudi + |tblproperties( + | primaryKey = 'id', + | preCombineField = 's1', + | type = '$tableType', + | ${HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key} = 'parquet', + | ${HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key()} = '10') + |location '$path' + """.stripMargin) + spark.sql(s"insert overwrite table $tableName select * from input_df") + } + + /** + * OpenJDK 64-Bit Server VM 1.8.0_345-b01 on Mac OS X 12.4 + * Apple M1 Pro + * pref insert overwrite: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative + * ----------------------------------------------------------------------------------------------------------------------------------- + * org.apache.hudi.common.model.HoodieAvroRecordMerger 16714 17107 353 0.1 16714.5 1.0X + * org.apache.hudi.HoodieSparkRecordMerger 12654 13924 1100 0.1 12653.8 1.3X + */ + private def overwriteBenchmark(): Unit = { + val df = createComplexDataFrame(1000000) + val benchmark = new HoodieBenchmark("pref insert overwrite", 1000000, 3) + Seq(classOf[HoodieAvroRecordMerger].getName, classOf[HoodieSparkRecordMerger].getName).zip(Seq(avroTable, sparkTable)).foreach { + case (merger, tableName) => benchmark.addCase(merger) { _ => + withTempDir { f => + prepareHoodieTable(tableName, new Path(f.getCanonicalPath, tableName).toUri.toString, "mor", merger, df) + } + } + } + benchmark.run() + } + + /** + * Java HotSpot(TM) 64-Bit Server VM 1.8.0_211-b12 on Mac OS X 10.16 + * Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz + * pref upsert: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative + * ----------------------------------------------------------------------------------------------------------------------------------- + * org.apache.hudi.common.model.HoodieAvroRecordMerger 6108 6383 257 0.0 610785.6 1.0X + * org.apache.hudi.HoodieSparkRecordMerger 4833 5468 614 0.0 483300.0 1.3X + * + * Java HotSpot(TM) 64-Bit Server VM 1.8.0_211-b12 on Mac OS X 10.16 + * Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz + * pref read: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative + * ----------------------------------------------------------------------------------------------------------------------------------- + * org.apache.hudi.common.model.HoodieAvroRecordMerger 813 818 8 0.0 81302.1 1.0X + * org.apache.hudi.HoodieSparkRecordMerger 604 616 18 0.0 60430.1 1.3X + */ + private def upsertThenReadBenchmark(): Unit = { + val avroMergerImpl = classOf[HoodieAvroRecordMerger].getName + val sparkMergerImpl = classOf[HoodieSparkRecordMerger].getName + val df = createComplexDataFrame(10000) + withTempDir { avroPath => + withTempDir { sparkPath => + val upsertBenchmark = new HoodieBenchmark("pref upsert", 10000, 3) + prepareHoodieTable(avroTable, new Path(avroPath.getCanonicalPath, avroTable).toUri.toString, "mor", avroMergerImpl, df) + prepareHoodieTable(sparkTable, new Path(sparkPath.getCanonicalPath, sparkTable).toUri.toString, "mor", sparkMergerImpl, df) + Seq(avroMergerImpl, sparkMergerImpl).zip(Seq(avroTable, sparkTable)).foreach { + case (mergerImpl, tableName) => upsertBenchmark.addCase(mergerImpl) { _ => + spark.sql(s"set ${HoodieWriteConfig.MERGER_IMPLS.key} = $mergerImpl") + spark.sql(s"update $tableName set s1 = 's1_new_1' where id > 0") + } + } + upsertBenchmark.run() + + val readBenchmark = new HoodieBenchmark("pref read", 10000, 3) + Seq(avroMergerImpl, sparkMergerImpl).zip(Seq(avroTable, sparkTable)).foreach { + case (mergerImpl, tableName) => readBenchmark.addCase(mergerImpl) { _ => + spark.sql(s"set ${HoodieWriteConfig.MERGER_IMPLS.key} = $mergerImpl") + spark.sql(s"select * from $tableName").collect() + } + } + readBenchmark.run() + + } + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + overwriteBenchmark() + upsertThenReadBenchmark() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala index 13800e6c8b8c2..a185b3849bd2e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala @@ -18,8 +18,13 @@ package org.apache.spark.sql.hudi import org.apache.hadoop.fs.Path -import org.apache.hudi.HoodieSparkUtils +import org.apache.hudi.{HoodieSparkRecordMerger, HoodieSparkUtils} import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.config.HoodieStorageConfig +import org.apache.hudi.common.model.HoodieAvroRecordMerger +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex import org.apache.log4j.Level import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -170,4 +175,47 @@ class HoodieSparkSqlTestBase extends FunSuite with BeforeAndAfterAll { val fs = FSUtils.getFs(filePath, spark.sparkContext.hadoopConfiguration) fs.exists(path) } + + protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val conf = spark.sessionState.conf + val currentValues = pairs.unzip._1.map { k => + if (conf.contains(k)) { + Some(conf.getConfString(k)) + } else None + } + pairs.foreach { case(k, v) => conf.setConfString(k, v) } + try f finally { + pairs.unzip._1.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } + + protected def withRecordType(recordConfig: Map[HoodieRecordType, Map[String, String]]=Map.empty)(f: => Unit) { + // TODO HUDI-5264 Test parquet log with avro record in spark sql test + Seq(HoodieRecordType.AVRO, HoodieRecordType.SPARK).foreach { recordType => + val (merger, format) = recordType match { + case HoodieRecordType.SPARK => (classOf[HoodieSparkRecordMerger].getName, "parquet") + case _ => (classOf[HoodieAvroRecordMerger].getName, "avro") + } + val config = Map( + HoodieWriteConfig.MERGER_IMPLS.key -> merger, + HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> format) ++ recordConfig.getOrElse(recordType, Map.empty) + withSQLConf(config.toList:_*) { + f + // We need to clear indexed location in memory after each test. + HoodieInMemoryHashIndex.clear() + } + } + } + + protected def getRecordType(): HoodieRecordType = { + val merger = spark.sessionState.conf.getConfString(HoodieWriteConfig.MERGER_IMPLS.key, HoodieWriteConfig.MERGER_IMPLS.defaultValue()) + if (merger.equals(classOf[HoodieSparkRecordMerger].getName)) { + HoodieRecordType.SPARK + } else { + HoodieRecordType.AVRO + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala index 0ef89fc5b9fe3..780d1aad1af2e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hudi class TestCompactionTable extends HoodieSparkSqlTestBase { test("Test compaction table") { - withTempDir {tmp => + withRecordType()(withTempDir {tmp => val tableName = generateTableName spark.sql( s""" @@ -67,11 +67,11 @@ class TestCompactionTable extends HoodieSparkSqlTestBase { Seq(4, "a4", 10.0, 1000) ) assertResult(0)(spark.sql(s"show compaction on $tableName").collect().length) - } + }) } test("Test compaction path") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -124,6 +124,6 @@ class TestCompactionTable extends HoodieSparkSqlTestBase { checkException(s"run compaction on '${tmp.getCanonicalPath}' at 12345")( s"Compaction instant: 12345 is not found in ${tmp.getCanonicalPath}, Available pending compaction instants are: " ) - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteFromTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteFromTable.scala index a972f835e8054..e1c30b1104d35 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteFromTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteFromTable.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hudi class TestDeleteFromTable extends HoodieSparkSqlTestBase { test("Test deleting from table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName spark.sql( @@ -78,6 +78,6 @@ class TestDeleteFromTable extends HoodieSparkSqlTestBase { Seq(3, "a3", 30.0, 3000, "2021-01-07") ) } - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala index 44c23d146c68b..ef84eb2c89c0a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql.hudi -import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload} +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieAvroRecordMerger, HoodieRecordMerger, OverwriteWithLatestAvroPayload} import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.common.util.StringUtils import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.spark.sql.types._ import org.junit.jupiter.api.Assertions.assertTrue @@ -31,15 +32,17 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness { def testWithDefaultSqlOptions(): Unit = { val ops1 = Map("primaryKey" -> "id") val with1 = HoodieOptionConfig.withDefaultSqlOptions(ops1) - assertTrue(with1.size == 3) + assertTrue(with1.size == 4) assertTrue(with1("primaryKey") == "id") assertTrue(with1("type") == "cow") assertTrue(with1("payloadClass") == classOf[OverwriteWithLatestAvroPayload].getName) + assertTrue(with1("mergerStrategy") == HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID) val ops2 = Map("primaryKey" -> "id", "preCombineField" -> "timestamp", "type" -> "mor", - "payloadClass" -> classOf[DefaultHoodieRecordPayload].getName + "payloadClass" -> classOf[DefaultHoodieRecordPayload].getName, + "mergerStrategy" -> HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID ) val with2 = HoodieOptionConfig.withDefaultSqlOptions(ops2) assertTrue(ops2 == with2) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala index 4c2bec48a6847..73aaabc0d8f8e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType +import org.apache.spark.sql.hudi.command.HoodieSparkValidateDuplicateKeyRecordMerger import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieDuplicateKeyException import org.apache.hudi.keygen.ComplexKeyGenerator @@ -30,7 +32,7 @@ import java.io.File class TestInsertTable extends HoodieSparkSqlTestBase { test("Test Insert Into with values") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -61,11 +63,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 20.0, 2000, "2021-01-06"), Seq(3, "a3", 30.0, 3000, "2021-01-07") ) - } + }) } test("Test Insert Into with static partition") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -114,11 +116,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 20.0, 2000, "2021-01-06"), Seq(3, "a3", 30.0, 3000, "2021-01-07") ) - } + }) } test("Test Insert Into with dynamic partition") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -168,11 +170,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 20.0, 2000, "2021-01-06"), Seq(3, "a3", 30.0, 3000, "2021-01-07") ) - } + }) } test("Test Insert Into with multi partition") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -221,80 +223,84 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(3, "a3", 30.0, 3000, "20210103", "03"), Seq(4, "a4", 40.0, 4000, "20210104", "04") ) - } + }) } test("Test Insert Into None Partitioned Table") { - withTempDir { tmp => - val tableName = generateTableName - spark.sql(s"set hoodie.sql.insert.mode=strict") - // Create none partitioned cow table - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | location '${tmp.getCanonicalPath}/$tableName' - | tblproperties ( - | type = 'cow', - | primaryKey = 'id', - | preCombineField = 'ts' - | ) - """.stripMargin) - spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") - checkAnswer(s"select id, name, price, ts from $tableName")( - Seq(1, "a1", 10.0, 1000) - ) - spark.sql(s"insert into $tableName select 2, 'a2', 12, 1000") - checkAnswer(s"select id, name, price, ts from $tableName")( - Seq(1, "a1", 10.0, 1000), - Seq(2, "a2", 12.0, 1000) - ) - - assertThrows[HoodieDuplicateKeyException] { - try { - spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") - } catch { - case e: Exception => - var root: Throwable = e - while (root.getCause != null) { - root = root.getCause - } - throw root - } - } - - // Create table with dropDup is true - val tableName2 = generateTableName - spark.sql("set hoodie.datasource.write.insert.drop.duplicates = true") - spark.sql( - s""" - |create table $tableName2 ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | location '${tmp.getCanonicalPath}/$tableName2' - | tblproperties ( - | type = 'mor', - | primaryKey = 'id', - | preCombineField = 'ts' - | ) - """.stripMargin) - spark.sql(s"insert into $tableName2 select 1, 'a1', 10, 1000") - // This record will be drop when dropDup is true - spark.sql(s"insert into $tableName2 select 1, 'a1', 12, 1000") - checkAnswer(s"select id, name, price, ts from $tableName2")( - Seq(1, "a1", 10.0, 1000) - ) - // disable this config to avoid affect other test in this class. - spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false") - spark.sql(s"set hoodie.sql.insert.mode=upsert") - } + withRecordType(Map(HoodieRecordType.SPARK -> + // SparkMerger should use "HoodieSparkValidateDuplicateKeyRecordMerger" + // with "hoodie.sql.insert.mode=strict" + Map(HoodieWriteConfig.MERGER_IMPLS.key -> + classOf[HoodieSparkValidateDuplicateKeyRecordMerger].getName)))(withTempDir { tmp => + val tableName = generateTableName + spark.sql(s"set hoodie.sql.insert.mode=strict") + // Create none partitioned cow table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'cow', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 10.0, 1000) + ) + spark.sql(s"insert into $tableName select 2, 'a2', 12, 1000") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 12.0, 1000) + ) + + assertThrows[HoodieDuplicateKeyException] { + try { + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + } catch { + case e: Exception => + var root: Throwable = e + while (root.getCause != null) { + root = root.getCause + } + throw root + } + } + + // Create table with dropDup is true + val tableName2 = generateTableName + spark.sql("set hoodie.datasource.write.insert.drop.duplicates = true") + spark.sql( + s""" + |create table $tableName2 ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName2' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + spark.sql(s"insert into $tableName2 select 1, 'a1', 10, 1000") + // This record will be drop when dropDup is true + spark.sql(s"insert into $tableName2 select 1, 'a1', 12, 1000") + checkAnswer(s"select id, name, price, ts from $tableName2")( + Seq(1, "a1", 10.0, 1000) + ) + // disable this config to avoid affect other test in this class. + spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false") + spark.sql(s"set hoodie.sql.insert.mode=upsert") + }) } test("Test Insert Into None Partitioned Table strict mode with no preCombineField") { @@ -344,7 +350,7 @@ class TestInsertTable extends HoodieSparkSqlTestBase { } test("Test Insert Overwrite") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a partitioned table spark.sql( @@ -434,27 +440,27 @@ class TestInsertTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, price, ts from $tblNonPartition")( Seq(2, "a2", 10.0, 1000) ) - } + }) } test("Test Different Type of Partition Column") { - withTempDir { tmp => - val typeAndValue = Seq( - ("string", "'1000'"), - ("int", 1000), - ("bigint", 10000), - ("timestamp", "TIMESTAMP'2021-05-20 00:00:00'"), - ("date", "DATE'2021-05-20'") - ) - typeAndValue.foreach { case (partitionType, partitionValue) => - val tableName = generateTableName - validateDifferentTypesOfPartitionColumn(tmp, partitionType, partitionValue, tableName) - } - } + withRecordType()(withTempDir { tmp => + val typeAndValue = Seq( + ("string", "'1000'"), + ("int", 1000), + ("bigint", 10000), + ("timestamp", "TIMESTAMP'2021-05-20 00:00:00'"), + ("date", "DATE'2021-05-20'") + ) + typeAndValue.foreach { case (partitionType, partitionValue) => + val tableName = generateTableName + validateDifferentTypesOfPartitionColumn(tmp, partitionType, partitionValue, tableName) + } + }) } test("Test TimestampType Partition Column With Consistent Logical Timestamp Enabled") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val typeAndValue = Seq( ("timestamp", "TIMESTAMP'2021-05-20 00:00:00'"), ("date", "DATE'2021-05-20'") @@ -464,33 +470,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { spark.sql(s"set hoodie.datasource.write.keygenerator.consistent.logical.timestamp.enabled=true") validateDifferentTypesOfPartitionColumn(tmp, partitionType, partitionValue, tableName) } - } - } - - private def validateDifferentTypesOfPartitionColumn(tmp: File, partitionType: String, partitionValue: Any, tableName: String) = { - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | dt $partitionType - |) using hudi - | tblproperties (primaryKey = 'id') - | partitioned by (dt) - | location '${tmp.getCanonicalPath}/$tableName' - """.stripMargin) - // NOTE: We have to drop type-literal prefix since Spark doesn't parse type literals appropriately - spark.sql(s"insert into $tableName partition(dt = ${dropTypeLiteralPrefix(partitionValue)}) select 1, 'a1', 10") - spark.sql(s"insert into $tableName select 2, 'a2', 10, $partitionValue") - checkAnswer(s"select id, name, price, cast(dt as string) from $tableName order by id")( - Seq(1, "a1", 10, extractRawValue(partitionValue).toString), - Seq(2, "a2", 10, extractRawValue(partitionValue).toString) - ) + }) } test("Test insert for uppercase table name") { - withTempDir{ tmp => + withRecordType()(withTempDir{ tmp => val tableName = s"H_$generateTableName" spark.sql( @@ -513,84 +497,11 @@ class TestInsertTable extends HoodieSparkSqlTestBase { .setConf(spark.sessionState.newHadoopConf()) .build() assertResult(metaClient.getTableConfig.getTableName)(tableName) - } + }) } test("Test Insert Exception") { - val tableName = generateTableName - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | dt string - |) using hudi - | tblproperties (primaryKey = 'id') - | partitioned by (dt) - """.stripMargin) - checkExceptionContain(s"insert into $tableName partition(dt = '2021-06-20') select 1, 'a1', 10, '2021-06-20'") ( - """ - |too many data columns: - |Table columns: 'id', 'name', 'price' - |Data columns: '1', 'a1', '10', '2021-06-20' - |""".stripMargin - ) - checkExceptionContain(s"insert into $tableName select 1, 'a1', 10")( - """ - |not enough data columns: - |Table columns: 'id', 'name', 'price', 'dt' - |Data columns: '1', 'a1', '10' - |""".stripMargin - ) - spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql("set hoodie.sql.insert.mode = strict") - - val tableName2 = generateTableName - spark.sql( - s""" - |create table $tableName2 ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | tblproperties ( - | primaryKey = 'id', - | preCombineField = 'ts' - | ) - """.stripMargin) - checkException(s"insert into $tableName2 values(1, 'a1', 10, 1000)")( - "Table with primaryKey can not use bulk insert in strict mode." - ) - - spark.sql("set hoodie.sql.insert.mode = non-strict") - val tableName3 = generateTableName - spark.sql( - s""" - |create table $tableName3 ( - | id int, - | name string, - | price double, - | dt string - |) using hudi - | tblproperties (primaryKey = 'id') - | partitioned by (dt) - """.stripMargin) - checkException(s"insert overwrite table $tableName3 values(1, 'a1', 10, '2021-07-18')")( - "Insert Overwrite Partition can not use bulk insert." - ) - spark.sql("set hoodie.sql.bulk.insert.enable = false") - spark.sql("set hoodie.sql.insert.mode = upsert") - } - - - test("Test Insert timestamp when 'spark.sql.datetime.java8API.enabled' enables") { - try { - // enable spark.sql.datetime.java8API.enabled - // and use java.time.Instant to replace java.sql.Timestamp to represent TimestampType. - spark.conf.set("spark.sql.datetime.java8API.enabled", value = true) - + withRecordType() { val tableName = generateTableName spark.sql( s""" @@ -598,209 +509,278 @@ class TestInsertTable extends HoodieSparkSqlTestBase { | id int, | name string, | price double, - | dt timestamp - |) - |using hudi - |partitioned by(dt) - |options(type = 'cow', primaryKey = 'id') - |""".stripMargin + | dt string + |) using hudi + | tblproperties (primaryKey = 'id') + | partitioned by (dt) + """.stripMargin) + checkExceptionContain(s"insert into $tableName partition(dt = '2021-06-20') select 1, 'a1', 10, '2021-06-20'") ( + """ + |too many data columns: + |Table columns: 'id', 'name', 'price' + |Data columns: '1', 'a1', '10', '2021-06-20' + |""".stripMargin ) - - spark.sql(s"insert into $tableName values (1, 'a1', 10, cast('2021-05-07 00:00:00' as timestamp))") - checkAnswer(s"select id, name, price, cast(dt as string) from $tableName")( - Seq(1, "a1", 10, "2021-05-07 00:00:00") + checkExceptionContain(s"insert into $tableName select 1, 'a1', 10")( + """ + |not enough data columns: + |Table columns: 'id', 'name', 'price', 'dt' + |Data columns: '1', 'a1', '10' + |""".stripMargin ) - - } finally { - spark.conf.set("spark.sql.datetime.java8API.enabled", value = false) - } - } - - test("Test bulk insert") { - spark.sql("set hoodie.sql.insert.mode = non-strict") - withTempDir { tmp => - Seq("cow", "mor").foreach {tableType => - // Test bulk insert for single partition - val tableName = generateTableName + withSQLConf("hoodie.sql.bulk.insert.enable" -> "true", "hoodie.sql.insert.mode" -> "strict") { + val tableName2 = generateTableName spark.sql( s""" - |create table $tableName ( + |create table $tableName2 ( | id int, | name string, | price double, - | dt string + | ts long |) using hudi | tblproperties ( - | type = '$tableType', - | primaryKey = 'id' + | primaryKey = 'id', + | preCombineField = 'ts' | ) - | partitioned by (dt) - | location '${tmp.getCanonicalPath}/$tableName' - """.stripMargin) - spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false") - - // Enable the bulk insert - spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')") - - checkAnswer(s"select id, name, price, dt from $tableName")( - Seq(1, "a1", 10.0, "2021-07-18") - ) - // Disable the bulk insert - spark.sql("set hoodie.sql.bulk.insert.enable = false") - spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')") - - checkAnswer(s"select id, name, price, dt from $tableName order by id")( - Seq(1, "a1", 10.0, "2021-07-18"), - Seq(2, "a2", 10.0, "2021-07-18") + """.stripMargin) + checkException(s"insert into $tableName2 values(1, 'a1', 10, 1000)")( + "Table with primaryKey can not use bulk insert in strict mode." ) - // Test bulk insert for multi-level partition - val tableMultiPartition = generateTableName + spark.sql("set hoodie.sql.insert.mode = non-strict") + val tableName3 = generateTableName spark.sql( s""" - |create table $tableMultiPartition ( + |create table $tableName3 ( | id int, | name string, | price double, - | dt string, - | hh string + | dt string |) using hudi - | tblproperties ( - | type = '$tableType', - | primaryKey = 'id' - | ) - | partitioned by (dt, hh) - | location '${tmp.getCanonicalPath}/$tableMultiPartition' + | tblproperties (primaryKey = 'id') + | partitioned by (dt) """.stripMargin) + checkException(s"insert overwrite table $tableName3 values(1, 'a1', 10, '2021-07-18')")( + "Insert Overwrite Partition can not use bulk insert." + ) + } + } + } - // Enable the bulk insert - spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql(s"insert into $tableMultiPartition values(1, 'a1', 10, '2021-07-18', '12')") - checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition")( - Seq(1, "a1", 10.0, "2021-07-18", "12") - ) - // Disable the bulk insert - spark.sql("set hoodie.sql.bulk.insert.enable = false") - spark.sql(s"insert into $tableMultiPartition " + - s"values(2, 'a2', 10, '2021-07-18','12')") - - checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition order by id")( - Seq(1, "a1", 10.0, "2021-07-18", "12"), - Seq(2, "a2", 10.0, "2021-07-18", "12") - ) - // Test bulk insert for non-partitioned table - val nonPartitionedTable = generateTableName + test("Test Insert timestamp when 'spark.sql.datetime.java8API.enabled' enables") { + withRecordType() { + withSQLConf("spark.sql.datetime.java8API.enabled" -> "true") { + val tableName = generateTableName spark.sql( s""" - |create table $nonPartitionedTable ( + |create table $tableName ( | id int, | name string, - | price double - |) using hudi - | tblproperties ( - | type = '$tableType', - | primaryKey = 'id' - | ) - | location '${tmp.getCanonicalPath}/$nonPartitionedTable' - """.stripMargin) - spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql(s"insert into $nonPartitionedTable values(1, 'a1', 10)") - checkAnswer(s"select id, name, price from $nonPartitionedTable")( - Seq(1, "a1", 10.0) - ) - spark.sql(s"insert overwrite table $nonPartitionedTable values(2, 'a2', 10)") - checkAnswer(s"select id, name, price from $nonPartitionedTable")( - Seq(2, "a2", 10.0) + | price double, + | dt timestamp + |) + |using hudi + |partitioned by(dt) + |options(type = 'cow', primaryKey = 'id') + |""".stripMargin ) - spark.sql("set hoodie.sql.bulk.insert.enable = false") - // Test CTAS for bulk insert - val tableName2 = generateTableName - spark.sql( - s""" - |create table $tableName2 - |using hudi - |tblproperties( - | type = '$tableType', - | primaryKey = 'id' - |) - | location '${tmp.getCanonicalPath}/$tableName2' - | as - | select * from $tableName - |""".stripMargin) - checkAnswer(s"select id, name, price, dt from $tableName2 order by id")( - Seq(1, "a1", 10.0, "2021-07-18"), - Seq(2, "a2", 10.0, "2021-07-18") + spark.sql(s"insert into $tableName values (1, 'a1', 10, cast('2021-05-07 00:00:00' as timestamp))") + checkAnswer(s"select id, name, price, cast(dt as string) from $tableName")( + Seq(1, "a1", 10, "2021-05-07 00:00:00") ) } } - spark.sql("set hoodie.sql.insert.mode = upsert") + } + + test("Test bulk insert") { + withSQLConf("hoodie.sql.insert.mode" -> "non-strict") { + withRecordType()(withTempDir { tmp => + Seq("cow", "mor").foreach {tableType => + // Test bulk insert for single partition + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | dt string + |) using hudi + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' + | ) + | partitioned by (dt) + | location '${tmp.getCanonicalPath}/$tableName' + """.stripMargin) + spark.sql("set hoodie.datasource.write.insert.drop.duplicates = false") + + // Enable the bulk insert + spark.sql("set hoodie.sql.bulk.insert.enable = true") + spark.sql(s"insert into $tableName values(1, 'a1', 10, '2021-07-18')") + + checkAnswer(s"select id, name, price, dt from $tableName")( + Seq(1, "a1", 10.0, "2021-07-18") + ) + // Disable the bulk insert + spark.sql("set hoodie.sql.bulk.insert.enable = false") + spark.sql(s"insert into $tableName values(2, 'a2', 10, '2021-07-18')") + + checkAnswer(s"select id, name, price, dt from $tableName order by id")( + Seq(1, "a1", 10.0, "2021-07-18"), + Seq(2, "a2", 10.0, "2021-07-18") + ) + + // Test bulk insert for multi-level partition + val tableMultiPartition = generateTableName + spark.sql( + s""" + |create table $tableMultiPartition ( + | id int, + | name string, + | price double, + | dt string, + | hh string + |) using hudi + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' + | ) + | partitioned by (dt, hh) + | location '${tmp.getCanonicalPath}/$tableMultiPartition' + """.stripMargin) + + // Enable the bulk insert + spark.sql("set hoodie.sql.bulk.insert.enable = true") + spark.sql(s"insert into $tableMultiPartition values(1, 'a1', 10, '2021-07-18', '12')") + + checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition")( + Seq(1, "a1", 10.0, "2021-07-18", "12") + ) + // Disable the bulk insert + spark.sql("set hoodie.sql.bulk.insert.enable = false") + spark.sql(s"insert into $tableMultiPartition " + + s"values(2, 'a2', 10, '2021-07-18','12')") + + checkAnswer(s"select id, name, price, dt, hh from $tableMultiPartition order by id")( + Seq(1, "a1", 10.0, "2021-07-18", "12"), + Seq(2, "a2", 10.0, "2021-07-18", "12") + ) + // Test bulk insert for non-partitioned table + val nonPartitionedTable = generateTableName + spark.sql( + s""" + |create table $nonPartitionedTable ( + | id int, + | name string, + | price double + |) using hudi + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' + | ) + | location '${tmp.getCanonicalPath}/$nonPartitionedTable' + """.stripMargin) + spark.sql("set hoodie.sql.bulk.insert.enable = true") + spark.sql(s"insert into $nonPartitionedTable values(1, 'a1', 10)") + checkAnswer(s"select id, name, price from $nonPartitionedTable")( + Seq(1, "a1", 10.0) + ) + spark.sql(s"insert overwrite table $nonPartitionedTable values(2, 'a2', 10)") + checkAnswer(s"select id, name, price from $nonPartitionedTable")( + Seq(2, "a2", 10.0) + ) + spark.sql("set hoodie.sql.bulk.insert.enable = false") + + // Test CTAS for bulk insert + val tableName2 = generateTableName + spark.sql( + s""" + |create table $tableName2 + |using hudi + |tblproperties( + | type = '$tableType', + | primaryKey = 'id' + |) + | location '${tmp.getCanonicalPath}/$tableName2' + | as + | select * from $tableName + |""".stripMargin) + checkAnswer(s"select id, name, price, dt from $tableName2 order by id")( + Seq(1, "a1", 10.0, "2021-07-18"), + Seq(2, "a2", 10.0, "2021-07-18") + ) + } + }) + } } test("Test combine before insert") { - spark.sql("set hoodie.sql.bulk.insert.enable = false") - withTempDir{tmp => - val tableName = generateTableName - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | location '${tmp.getCanonicalPath}/$tableName' - | tblproperties ( - | primaryKey = 'id', - | preCombineField = 'ts' - | ) + withSQLConf("set hoodie.sql.bulk.insert.enable" -> "false") { + withRecordType()(withTempDir{tmp => + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) """.stripMargin) - spark.sql( - s""" - |insert overwrite table $tableName - |select * from ( - | select 1 as id, 'a1' as name, 10 as price, 1000 as ts - | union all - | select 1 as id, 'a1' as name, 11 as price, 1001 as ts - | ) - |""".stripMargin - ) - checkAnswer(s"select id, name, price, ts from $tableName")( - Seq(1, "a1", 11.0, 1001) - ) + spark.sql( + s""" + |insert overwrite table $tableName + |select * from ( + | select 1 as id, 'a1' as name, 10 as price, 1000 as ts + | union all + | select 1 as id, 'a1' as name, 11 as price, 1001 as ts + | ) + |""".stripMargin + ) + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 11.0, 1001) + ) + }) } } test("Test insert pk-table") { - spark.sql("set hoodie.sql.bulk.insert.enable = false") - withTempDir{tmp => - val tableName = generateTableName - spark.sql( - s""" - |create table $tableName ( - | id int, - | name string, - | price double, - | ts long - |) using hudi - | location '${tmp.getCanonicalPath}/$tableName' - | tblproperties ( - | primaryKey = 'id', - | preCombineField = 'ts' - | ) + withSQLConf("hoodie.sql.bulk.insert.enable" -> "false") { + withRecordType()(withTempDir{tmp => + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) """.stripMargin) - spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") - spark.sql(s"insert into $tableName values(1, 'a1', 11, 1000)") - checkAnswer(s"select id, name, price, ts from $tableName")( - Seq(1, "a1", 11.0, 1000) - ) + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(1, 'a1', 11, 1000)") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 11.0, 1000) + ) + }) } } test("Test For read operation's field") { - withTempDir { tmp => { + withRecordType()(withTempDir { tmp => { val tableName = generateTableName val tablePath = s"${tmp.getCanonicalPath}/$tableName" import spark.implicits._ @@ -838,44 +818,45 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(1, null) ) } - } + }) } test("Test enable hoodie.datasource.write.drop.partition.columns when write") { - spark.sql("set hoodie.sql.bulk.insert.enable = false") - Seq("mor", "cow").foreach { tableType => - withTempDir { tmp => - val tableName = generateTableName - spark.sql( - s""" - | create table $tableName ( - | id int, - | name string, - | price double, - | ts long, - | dt string - | ) using hudi - | partitioned by (dt) - | location '${tmp.getCanonicalPath}/$tableName' - | tblproperties ( - | primaryKey = 'id', - | preCombineField = 'ts', - | type = '$tableType', - | hoodie.datasource.write.drop.partition.columns = 'true' - | ) + withSQLConf("hoodie.sql.bulk.insert.enable" -> "false") { + Seq("mor", "cow").foreach { tableType => + withRecordType()(withTempDir { tmp => + val tableName = generateTableName + spark.sql( + s""" + | create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | dt string + | ) using hudi + | partitioned by (dt) + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts', + | type = '$tableType', + | hoodie.datasource.write.drop.partition.columns = 'true' + | ) """.stripMargin) - spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (1, 'a1', 10, 1000)") - spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (2, 'a2', 20, 1000)") - checkAnswer(s"select id, name, price, ts, dt from $tableName")( - Seq(1, "a1", 10, 1000, "2021-12-25"), - Seq(2, "a2", 20, 1000, "2021-12-25") - ) + spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (2, 'a2', 20, 1000)") + checkAnswer(s"select id, name, price, ts, dt from $tableName")( + Seq(1, "a1", 10, 1000, "2021-12-25"), + Seq(2, "a2", 20, 1000, "2021-12-25") + ) + }) } } } test("Test nested field as primaryKey and preCombineField") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName // create table @@ -904,12 +885,12 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq("name_1", 10.0, 1000, "a", 999) ) } - } + }) } test("Test Insert Into With Catalog Identifier for spark >= 3.2.0") { Seq("hudi", "parquet").foreach { format => - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = s"spark_catalog.default.$generateTableName" // Create a partitioned table if (HoodieSparkUtils.gteqSpark3_2) { @@ -946,10 +927,32 @@ class TestInsertTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 10.0, 1000, "2021-01-05") ) } - } + }) } } + private def validateDifferentTypesOfPartitionColumn(tmp: File, partitionType: String, partitionValue: Any, tableName: String) = { + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | dt $partitionType + |) using hudi + | tblproperties (primaryKey = 'id') + | partitioned by (dt) + | location '${tmp.getCanonicalPath}/$tableName' + """.stripMargin) + // NOTE: We have to drop type-literal prefix since Spark doesn't parse type literals appropriately + spark.sql(s"insert into $tableName partition(dt = ${dropTypeLiteralPrefix(partitionValue)}) select 1, 'a1', 10") + spark.sql(s"insert into $tableName select 2, 'a2', 10, $partitionValue") + checkAnswer(s"select id, name, price, cast(dt as string) from $tableName order by id")( + Seq(1, "a1", 10, extractRawValue(partitionValue).toString), + Seq(2, "a2", 10, extractRawValue(partitionValue).toString) + ) + } + test("Test enable hoodie.merge.allow.duplicate.on.inserts when write") { spark.sql("set hoodie.datasource.write.operation = insert") Seq("mor", "cow").foreach { tableType => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala index 232b6bbb511c5..48ee872d4d95f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala @@ -22,7 +22,7 @@ import org.apache.hudi.testutils.DataSourceTestUtils class TestMergeIntoLogOnlyTable extends HoodieSparkSqlTestBase { test("Test Query Log Only MOR Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => // Create table with INMEMORY index to generate log only mor table. val tableName = generateTableName spark.sql( @@ -86,6 +86,6 @@ class TestMergeIntoLogOnlyTable extends HoodieSparkSqlTestBase { Seq(3, "a3", 10.0, 1000), Seq(4, "a4", 11.0, 1000) ) - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala index 4aa91498b110b..0981f74a102ca 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala @@ -23,7 +23,7 @@ import org.apache.hudi.common.fs.FSUtils class TestMergeIntoTable extends HoodieSparkSqlTestBase { test("Test MergeInto Basic") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -109,11 +109,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { """.stripMargin) val cnt = spark.sql(s"select * from $tableName where id = 1").count() assertResult(0)(cnt) - } + }) } test("Test MergeInto with ignored record") { - withTempDir {tmp => + withRecordType()(withTempDir {tmp => val sourceTable = generateTableName val targetTable = generateTableName // Create source table @@ -187,11 +187,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 20.0, 1001), Seq(3, "a3", 12.0, 1000) ) - } + }) } test("Test MergeInto for MOR table ") { - withTempDir {tmp => + withRecordType()(withTempDir {tmp => val tableName = generateTableName // Create a mor partitioned table. spark.sql( @@ -298,11 +298,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,price,dt from $tableName order by id")( Seq(1, "a1", 12, "2021-03-21") ) - } + }) } test("Test MergeInto with insert only") { - withTempDir {tmp => + withRecordType()(withTempDir {tmp => // Create a partitioned mor table val tableName = generateTableName spark.sql( @@ -352,11 +352,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 10, "2021-03-21"), Seq(2, "a2", 10, "2021-03-20") ) - } + }) } test("Test MergeInto For PreCombineField") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName1 = generateTableName // Create a mor partitioned table. @@ -425,11 +425,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 12, "2021-03-21", 1002) ) } - } + }) } test("Test MergeInto with preCombine field expression") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName1 = generateTableName spark.sql( @@ -485,11 +485,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 24, "2021-03-21", 1002) ) } - } + }) } test("Test MergeInto with primaryKey expression") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName1 = generateTableName spark.sql( s""" @@ -544,11 +544,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,price,v,dt from $tableName1 order by id")( Seq(1, "a1", 10, 1000, "2021-03-21") ) - } + }) } test("Test MergeInto with combination of delete update insert") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val sourceTable = generateTableName val targetTable = generateTableName // Create source table @@ -606,11 +606,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(11, "s11", 110, 2011, "2021-03-21"), Seq(12, "s12", 120, 2012, "2021-03-21") ) - } + }) } test("Merge Hudi to Hudi") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val sourceTable = generateTableName spark.sql( @@ -711,11 +711,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(2, "a2", 10, 1001) ) } - } + }) } test("Test Different Type of PreCombineField") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val typeAndValue = Seq( ("string", "'1000'"), ("int", 1000), @@ -771,11 +771,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 20.0) ) } - } + }) } test("Test MergeInto For MOR With Compaction On") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -821,11 +821,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(3, "a3", 10.0, 1000), Seq(4, "a4", 11.0, 1000) ) - } + }) } test("Test MereInto With Null Fields") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val types = Seq( "string" , "int", @@ -866,11 +866,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", null, 1000) ) } - } + }) } test("Test MergeInto With All Kinds Of DataType") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val dataAndTypes = Seq( ("string", "'a1'"), ("int", "10"), @@ -912,11 +912,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { Seq(1, "a1", extractRawValue(dataValue), 1000) ) } - } + }) } test("Test MergeInto with no-full fields source") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -947,11 +947,11 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, value, ts from $tableName")( Seq(1, "a1", 10, 1001) ) - } + }) } test("Test Merge Into with target matched columns cast-ed") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -1027,6 +1027,6 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, value, ts from $tableName")( Seq(1, "a1", 10, 1004) ) - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala index 7f23c9f7d8aa3..0a8458063cd5a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.Row class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { test("Test MergeInto for MOR table 2") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a mor partitioned table. spark.sql( @@ -137,11 +137,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { | price = s0.price + t0.price, ts = s0.ts, dt = s0.dt """.stripMargin )("assertion failed: Target table's field(price) cannot be the right-value of the update clause for MOR table.") - } + }) } test("Test Merge Into CTAS Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -172,11 +172,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name from $tableName")( Seq(1, "a1_1") ) - } + }) } test("Test Merge With Complex Data Type") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -235,11 +235,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, s_value, a_value, m_value, ts from $tableName")( Seq(1, "a1", Row(1, "12"), Seq("a0", "a1", "a2"), Map("k1" -> "v1"), 1000) ) - } + }) } test("Test column name matching for insert * and update set *") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -318,11 +318,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(3, "a3", 102.0, 1000, "2021-05-05"), Seq(4, "a4", 100.0, 1000, "2021-05-06") ) - } + }) } test("Test MergeInto For Source Table With Column Aliases") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -361,11 +361,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, "a1", 10.0, 1000) ) } - } + }) } test("Test MergeInto When PrimaryKey And PreCombineField Of Source Table And Target Table Differ In Case Only") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -430,11 +430,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, "a1", 11.0, 1001), Seq(2, "a2", 12.0, 1002) ) - } + }) } test("Test ignoring case") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -501,11 +501,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, "a1", 111.0, 1111, "2021-05-05"), Seq(2, "a2", 112.0, 1112, "2021-05-05") ) - } + }) } test("Test ignoring case for MOR table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a mor partitioned table. spark.sql( @@ -541,11 +541,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id, name, price, ts, dt from $tableName")( Seq(1, "a1", 111.0, 1111, "2021-05-05") ) - } + }) } test("Test only insert when source table contains history") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table spark.sql( @@ -586,11 +586,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, "a1", 1.0, 10, "2022-08-18"), Seq(2, "a2", 10.0, 100, "2022-08-18") ) - } + }) } test("Test only insert when source table contains history and target table has multiple keys") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create table with multiple keys spark.sql( @@ -632,11 +632,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { Seq(1, 1, "a1", 1.0, 10, "2022-08-18"), Seq(1, 2, "a2", 10.0, 100, "2022-08-18") ) - } + }) } test("Test Merge Into For Source Table With Different Column Order") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a mor partitioned table. spark.sql( @@ -670,11 +670,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,price,dt from $tableName")( Seq(1, "a1", 10, "2021-03-21") ) - } + }) } - test ("Test Merge into with String cast to Double") { - withTempDir { tmp => + test("Test Merge into with String cast to Double") { + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a cow partitioned table. spark.sql( @@ -710,11 +710,11 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,price,dt from $tableName")( Seq(1, "a1", 10.1, "2021-03-21") ) - } + }) } test("Test Merge into where manually set DefaultHoodieRecordPayload") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName // Create a cow table with default payload class, check whether it will be overwritten by ExpressionPayload. // if not, this ut cannot pass since DefaultHoodieRecordPayload can not promotion int to long when insert a ts with Integer value @@ -748,7 +748,7 @@ class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { checkAnswer(s"select id,name,ts from $tableName")( Seq(1, "a2", 1000) ) - } + }) } test("Test only insert for source table in dup key with preCombineField") { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala index a84ff0fa3f857..14d6d9afaee49 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala @@ -20,11 +20,13 @@ package org.apache.spark.sql.hudi import org.apache.hadoop.fs.Path import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD_OPT_KEY, PRECOMBINE_FIELD_OPT_KEY, RECORDKEY_FIELD_OPT_KEY, TABLE_NAME} import org.apache.hudi.QuickstartUtils.{DataGenerator, convertToStringList, getQuickstartWriteConfigs} +import org.apache.hudi.common.config.HoodieStorageConfig import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload import org.apache.hudi.config.HoodieWriteConfig -import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieSparkUtils} +import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieSparkRecordMerger, HoodieSparkUtils} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.functions.{arrays_zip, col, expr, lit} import org.apache.spark.sql.types.StringType @@ -67,7 +69,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } test("Test multi change data type") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -134,11 +136,11 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { spark.sessionState.catalog.refreshTable(TableIdentifier(tableName)) } } - } + }) } test("Test multi change data type2") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -171,7 +173,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } - } + }) } test("Test Enable and Disable Schema on read") { @@ -227,7 +229,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } test("Test Partition Table alter ") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -325,11 +327,11 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 or id = 11 order by id").show(false) } } - } + }) } test("Test Chinese table ") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -375,12 +377,12 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } - } + }) } test("Test Alter Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -437,7 +439,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } - } + }) } test("Test Alter Table multiple times") { @@ -479,8 +481,8 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } test("Test Alter Table complex") { - withTempDir { tmp => - Seq("cow", "mor").foreach { tableType => + withRecordType()(withTempDir { tmp => + Seq("mor").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" if (HoodieSparkUtils.gteqSpark3_1) { @@ -563,11 +565,11 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } - } + }) } test("Test schema auto evolution complex") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("COPY_ON_WRITE", "MERGE_ON_READ").foreach { tableType => val tableName = generateTableName val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" @@ -592,14 +594,15 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY -> "true" ) + val (writeOpt, readOpt) = getWriterReaderOpts(getRecordType(), hudiOptions) orgStringDf.write .format("org.apache.hudi") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) - .options(hudiOptions) + .options(writeOpt) .mode(SaveMode.Overwrite) .save(tablePath) - val oldView = spark.read.format("hudi").load(tablePath) + val oldView = spark.read.format("hudi").options(readOpt).load(tablePath) oldView.show(5, false) val records2 = RawTripTestPayload.recordsToStrings(dataGen.generateUpdatesAsPerSchema("002", 100, schema)).toList @@ -609,12 +612,12 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { updatedStringDf.write .format("org.apache.hudi") - .options(hudiOptions) + .options(writeOpt) .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option("hoodie.datasource.write.reconcile.schema", "true") .mode(SaveMode.Append) .save(tablePath) - spark.read.format("hudi").load(tablePath).registerTempTable("newView") + spark.read.format("hudi").options(readOpt).load(tablePath).registerTempTable("newView") val checkResult = spark.sql(s"select tip_history.amount,city_to_state,distance_in_meters,fare,height from newView where _row_key='$checkRowKey' ") .collect().map(row => (row.isNullAt(0), row.isNullAt(1), row.isNullAt(2), row.isNullAt(3), row.isNullAt(4))) assertResult((false, false, false, true, true))(checkResult(0)) @@ -623,6 +626,18 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { ) } } + }) + } + + val sparkOpts = Map( + HoodieWriteConfig.MERGER_IMPLS.key -> classOf[HoodieSparkRecordMerger].getName, + HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> "parquet" + ) + + def getWriterReaderOpts(recordType: HoodieRecordType, opt: Map[String, String]): (Map[String, String], Map[String, String]) = { + recordType match { + case HoodieRecordType.SPARK => (opt ++ sparkOpts, sparkOpts) + case _ => (opt, Map.empty[String, String]) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala index 90d0734945d7d..72bd71e4bba53 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala @@ -85,7 +85,8 @@ class TestSqlConf extends HoodieSparkSqlTestBase with BeforeAndAfter { assertResult(HoodieTableType.MERGE_ON_READ)(new HoodieTableConfig( new Path(tablePath).getFileSystem(new Configuration), s"$tablePath/" + HoodieTableMetaClient.METAFOLDER_NAME, - HoodieTableConfig.PAYLOAD_CLASS_NAME.defaultValue).getTableType) + HoodieTableConfig.PAYLOAD_CLASS_NAME.defaultValue, + HoodieTableConfig.MERGER_STRATEGY.defaultValue).getTableType) // Manually pass incremental configs to global configs to make sure Hudi query is able to load the // global configs diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala index 30d465f38d3c7..ed67b0caf9add 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala @@ -23,7 +23,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient class TestTimeTravelTable extends HoodieSparkSqlTestBase { test("Test Insert and Update Record with time travel") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName1 = generateTableName spark.sql( s""" @@ -62,13 +62,13 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { s"select id, name, price, ts from $tableName1 TIMESTAMP AS OF '$instant1'")( Seq(1, "a1", 10.0, 1000) ) - } + }) } } test("Test Insert Into Records with time travel To new Table") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => // Create Non-Partitioned table val tableName1 = generateTableName spark.sql( @@ -137,13 +137,13 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 10.0, 1000, "2022-02-14"), Seq(2, "a2", 10.0, 1000, "2022-02-15") ) - } + }) } } test("Test Two Table's Union Join with time travel") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach { tableType => val tableName = generateTableName @@ -235,25 +235,25 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { Seq(4, "a4", 20.0, 1000) ) } - } + }) } } test("Test Unsupported syntax can be parsed") { if (HoodieSparkUtils.gteqSpark3_2) { checkAnswer("select 1 distribute by 1")(Seq(1)) - withTempDir { dir => + withRecordType()(withTempDir { dir => val path = dir.toURI.getPath spark.sql(s"insert overwrite local directory '$path' using parquet select 1") // Requires enable hive support, so didn't test it // spark.sql(s"insert overwrite local directory '$path' stored as orc select 1") - } + }) } } test("Test Select Record with time travel and Repartition") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => val tableName = generateTableName spark.sql( s""" @@ -291,13 +291,13 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { s"select id, name, price, ts from $tableName TIMESTAMP AS OF '$instant' distribute by cast(rand() * 2 as int)")( Seq(1, "a1", 10.0, 1000) ) - } + }) } } test("Test Time Travel With Schema Evolution") { if (HoodieSparkUtils.gteqSpark3_2) { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => spark.sql("set hoodie.schema.on.read.enable=true") val tableName = generateTableName spark.sql( @@ -350,7 +350,7 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 1000, null), Seq(2, "a2", 1100, "hudi") ) - } + }) } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala index 2d8d6ceca7142..8937e8595d389 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala @@ -22,7 +22,7 @@ import org.apache.hudi.HoodieSparkUtils.isSpark2 class TestUpdateTable extends HoodieSparkSqlTestBase { test("Test Update Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach {tableType => val tableName = generateTableName // create table @@ -60,11 +60,11 @@ class TestUpdateTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 40.0, 1000) ) } - } + }) } test("Test Update Table On Non-PK Condition") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach {tableType => /** non-partitioned table */ val tableName = generateTableName @@ -161,11 +161,11 @@ class TestUpdateTable extends HoodieSparkSqlTestBase { Seq(3, "a2", 33.0, 1001, "2022") ) } - } + }) } test("Test ignoring case for Update Table") { - withTempDir { tmp => + withRecordType()(withTempDir { tmp => Seq("cow", "mor").foreach {tableType => val tableName = generateTableName // create table @@ -202,6 +202,6 @@ class TestUpdateTable extends HoodieSparkSqlTestBase { Seq(1, "a1", 40.0, 1000) ) } - } + }) } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index d4ca8bc402015..d537dcef4b424 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql._ +import org.apache.spark.sql.execution.vectorized.MutableColumnarRow import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ @@ -48,6 +49,12 @@ import scala.collection.mutable.ArrayBuffer */ class Spark2Adapter extends SparkAdapter { + override def isColumnarBatchRow(r: InternalRow): Boolean = { + // NOTE: In Spark 2.x there's no [[ColumnarBatchRow]], instead [[MutableColumnarRow]] is leveraged + // for vectorized reads + r.isInstanceOf[MutableColumnarRow] + } + override def getCatalogUtils: HoodieCatalogUtils = { throw new UnsupportedOperationException("Catalog utilities are not supported in Spark 2.x"); } diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala index bed6ff33508ba..0a4bc289b35e7 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, P import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetFileFormat} import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.vectorized.ColumnarUtils import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, HoodieSpark31CatalogUtils, HoodieSpark31CatalystExpressionUtils, HoodieSpark31CatalystPlanUtils, HoodieSpark3CatalogUtils, SparkSession} /** @@ -37,6 +38,8 @@ import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansU */ class Spark3_1Adapter extends BaseSpark3Adapter { + override def isColumnarBatchRow(r: InternalRow): Boolean = ColumnarUtils.isColumnarBatchRow(r) + override def getCatalogUtils: HoodieSpark3CatalogUtils = HoodieSpark31CatalogUtils override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark31CatalystExpressionUtils diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/vectorized/ColumnarUtils.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/vectorized/ColumnarUtils.scala new file mode 100644 index 0000000000000..e6015a65cb0d6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/vectorized/ColumnarUtils.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.vectorized + +import org.apache.spark.sql.catalyst.InternalRow + +object ColumnarUtils { + + /** + * Utility verifying whether provided instance of [[InternalRow]] is actually + * an instance of [[ColumnarBatchRow]] + * + * NOTE: This utility is required, since in Spark <= 3.3 [[ColumnarBatchRow]] is package-private + */ + def isColumnarBatchRow(r: InternalRow): Boolean = r.isInstanceOf[ColumnarBatchRow] + +} diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala index 24d4b5f3473bb..4a37890557e22 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala @@ -30,12 +30,15 @@ import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, P import org.apache.spark.sql.hudi.analysis.TableValuedFunctions import org.apache.spark.sql.parser.HoodieSpark3_2ExtendedSqlParser import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.vectorized.ColumnarUtils /** * Implementation of [[SparkAdapter]] for Spark 3.2.x branch */ class Spark3_2Adapter extends BaseSpark3Adapter { + override def isColumnarBatchRow(r: InternalRow): Boolean = ColumnarUtils.isColumnarBatchRow(r) + override def getCatalogUtils: HoodieSpark3CatalogUtils = HoodieSpark32CatalogUtils override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark32CatalystExpressionUtils diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/vectorized/ColumnarUtils.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/vectorized/ColumnarUtils.scala new file mode 100644 index 0000000000000..e6015a65cb0d6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/vectorized/ColumnarUtils.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.vectorized + +import org.apache.spark.sql.catalyst.InternalRow + +object ColumnarUtils { + + /** + * Utility verifying whether provided instance of [[InternalRow]] is actually + * an instance of [[ColumnarBatchRow]] + * + * NOTE: This utility is required, since in Spark <= 3.3 [[ColumnarBatchRow]] is package-private + */ + def isColumnarBatchRow(r: InternalRow): Boolean = r.isInstanceOf[ColumnarBatchRow] + +} diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala index 7546f73f1abf2..696c044223f2d 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Sp import org.apache.spark.sql.hudi.analysis.TableValuedFunctions import org.apache.spark.sql.parser.HoodieSpark3_3ExtendedSqlParser import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatchRow import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, HoodieSpark33CatalogUtils, HoodieSpark33CatalystExpressionUtils, HoodieSpark33CatalystPlanUtils, HoodieSpark3CatalogUtils, SparkSession} /** @@ -37,6 +38,8 @@ import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansU */ class Spark3_3Adapter extends BaseSpark3Adapter { + override def isColumnarBatchRow(r: InternalRow): Boolean = r.isInstanceOf[ColumnarBatchRow] + override def getCatalogUtils: HoodieSpark3CatalogUtils = HoodieSpark33CatalogUtils override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark33CatalystExpressionUtils 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 a58f835dab29e..131fdab6db23c 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 @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieDeltaWriteStat; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; @@ -84,6 +85,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_BATCH_SYNC_PARTITION_NUM; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; @@ -448,8 +450,8 @@ private static HoodieLogFile generateLogData(Path parquetFilePath, boolean isLog Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(parquetFilePath.getParent()) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(dataFile.getFileId()) .overBaseCommit(dataFile.getCommitTime()).withFs(fileSystem).build(); - List records = (isLogSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100) - : SchemaTestUtil.generateEvolvedTestRecords(100, 100)); + List records = (isLogSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100) + : SchemaTestUtil.generateEvolvedTestRecords(100, 100)).stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()); Map header = new HashMap<>(2); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java index 347078ec71cbc..69e50cbdeda9d 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java @@ -34,6 +34,7 @@ import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieLogFormat; @@ -63,7 +64,6 @@ import com.beust.jcommander.Parameter; import jline.internal.Log; import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -1047,7 +1047,7 @@ private List getAllColumnNames() { private Option readBloomFilterFromFile(String partitionPath, String filename) { Path path = new Path(FSUtils.getPartitionPath(metaClient.getBasePathV2(), partitionPath), filename); BloomFilter bloomFilter; - try (HoodieFileReader fileReader = HoodieFileReaderFactory.getFileReader(metaClient.getHadoopConf(), path)) { + try (HoodieFileReader fileReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(metaClient.getHadoopConf(), path)) { bloomFilter = fileReader.readBloomFilter(); if (bloomFilter == null) { Log.error("Failed to read bloom filter for " + path); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java index 334afd05d37e9..8f5f002d19b8f 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java @@ -64,6 +64,7 @@ import org.apache.hudi.utilities.transform.Transformer; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.spark.HoodieSparkKryoProvider$; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -280,7 +281,8 @@ private static SparkConf buildSparkConf(String appName, String defaultMaster, Ma sparkConf.set("spark.driver.allowMultipleContexts", "true"); additionalConfigs.forEach(sparkConf::set); - return SparkRDDWriteClient.registerClasses(sparkConf); + HoodieSparkKryoProvider$.MODULE$.register(sparkConf); + return sparkConf; } private static SparkConf buildSparkConf(String appName, Map additionalConfigs) { @@ -294,7 +296,8 @@ private static SparkConf buildSparkConf(String appName, Map addi sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK"); additionalConfigs.forEach(sparkConf::set); - return SparkRDDWriteClient.registerClasses(sparkConf); + HoodieSparkKryoProvider$.MODULE$.register(sparkConf); + return sparkConf; } public static JavaSparkContext buildSparkContext(String appName, Map configs) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 9821f393daf7f..e57ac513a8e8d 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -31,27 +31,39 @@ import org.apache.hudi.HoodieConversionUtils; import org.apache.hudi.HoodieSparkSqlWriter; import org.apache.hudi.HoodieSparkUtils; +import org.apache.hudi.SparkAdapterSupport$; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.model.HoodieSparkRecord; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.ConfigUtils; +import org.apache.hudi.common.util.HoodieRecordUtils; +import org.apache.hudi.common.util.IdentityIterator; +import org.apache.hudi.common.util.MappingIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; @@ -68,6 +80,7 @@ import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.keygen.SparkKeyGeneratorInterface; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.metrics.HoodieMetrics; @@ -93,8 +106,12 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.avro.HoodieAvroDeserializer; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; import scala.collection.JavaConversions; import java.io.Closeable; @@ -260,7 +277,12 @@ public DeltaSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, Sche public void refreshTimeline() throws IOException { if (fs.exists(new Path(cfg.targetBasePath))) { try { - HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(cfg.targetBasePath).setPayloadClassName(cfg.payloadClassName).build(); + HoodieTableMetaClient meta = HoodieTableMetaClient.builder() + .setConf(new Configuration(fs.getConf())) + .setBasePath(cfg.targetBasePath) + .setPayloadClassName(cfg.payloadClassName) + .setMergerStrategy(props.getProperty(HoodieWriteConfig.MERGER_STRATEGY.key(), HoodieWriteConfig.MERGER_STRATEGY.defaultValue())) + .build(); switch (meta.getTableType()) { case COPY_ON_WRITE: this.commitTimelineOpt = Option.of(meta.getActiveTimeline().getCommitTimeline().filterCompletedInstants()); @@ -451,6 +473,15 @@ public Pair>> readFromSource( } private Pair>> fetchFromSource(Option resumeCheckpointStr) { + HoodieRecordType recordType = HoodieRecordUtils.createRecordMerger(null, EngineType.SPARK, + ConfigUtils.split2List(props.getProperty(HoodieWriteConfig.MERGER_IMPLS.key(), HoodieWriteConfig.MERGER_IMPLS.defaultValue())), + props.getProperty(HoodieWriteConfig.MERGER_STRATEGY.key(), HoodieWriteConfig.MERGER_STRATEGY.defaultValue())).getRecordType(); + if (recordType == HoodieRecordType.SPARK && HoodieTableType.valueOf(cfg.tableType) == HoodieTableType.MERGE_ON_READ + && HoodieLogBlockType.fromId(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(), "avro")) + != HoodieLogBlockType.PARQUET_DATA_BLOCK) { + throw new UnsupportedOperationException("Spark record only support parquet log."); + } + final Option> avroRDDOptional; final String checkpointStr; SchemaProvider schemaProvider; @@ -519,15 +550,40 @@ private Pair>> fetchFromSourc boolean shouldCombine = cfg.filterDupes || cfg.operation.equals(WriteOperationType.UPSERT); Set partitionColumns = getPartitionColumns(keyGenerator, props); JavaRDD avroRDD = avroRDDOptional.get(); - JavaRDD records = avroRDD.map(record -> { - GenericRecord gr = isDropPartitionColumns() ? HoodieAvroUtils.removeFields(record, partitionColumns) : record; - HoodieRecordPayload payload = shouldCombine ? DataSourceUtils.createPayload(cfg.payloadClassName, gr, - (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false, props.getBoolean( - KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), - Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())))) - : DataSourceUtils.createPayload(cfg.payloadClassName, gr); - return new HoodieAvroRecord<>(keyGenerator.getKey(record), payload); - }); + + JavaRDD records; + SerializableSchema avroSchema = new SerializableSchema(schemaProvider.getTargetSchema()); + SerializableSchema processedAvroSchema = new SerializableSchema(isDropPartitionColumns() ? HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get()); + if (recordType == HoodieRecordType.AVRO) { + records = avroRDD.map(record -> { + GenericRecord gr = isDropPartitionColumns() ? HoodieAvroUtils.removeFields(record, partitionColumns) : record; + HoodieRecordPayload payload = shouldCombine ? DataSourceUtils.createPayload(cfg.payloadClassName, gr, + (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false, props.getBoolean( + KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), + Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())))) + : DataSourceUtils.createPayload(cfg.payloadClassName, gr); + return new HoodieAvroRecord<>(keyGenerator.getKey(record), payload); + }); + } else if (recordType == HoodieRecordType.SPARK) { + // TODO we should remove it if we can read InternalRow from source. + records = avroRDD.mapPartitions(itr -> { + StructType baseStructType = AvroConversionUtils.convertAvroSchemaToStructType(processedAvroSchema.get()); + StructType targetStructType = isDropPartitionColumns() ? AvroConversionUtils + .convertAvroSchemaToStructType(HoodieAvroUtils.removeFields(processedAvroSchema.get(), partitionColumns)) : baseStructType; + HoodieAvroDeserializer deserializer = SparkAdapterSupport$.MODULE$.sparkAdapter().createAvroDeserializer(processedAvroSchema.get(), baseStructType); + + return new MappingIterator<>(new IdentityIterator<>(itr), rec -> { + InternalRow row = (InternalRow) deserializer.deserialize(rec).get(); + SparkKeyGeneratorInterface keyGenerator = (SparkKeyGeneratorInterface) this.keyGenerator; + String recordKey = keyGenerator.getRecordKey(row, baseStructType).toString(); + String partitionPath = keyGenerator.getPartitionPath(row, baseStructType).toString(); + return new HoodieSparkRecord(new HoodieKey(recordKey, partitionPath), + HoodieInternalRowUtils.getCachedUnsafeProjection(baseStructType, targetStructType).apply(row), targetStructType, false); + }); + }); + } else { + throw new UnsupportedOperationException(recordType.name()); + } return Pair.of(schemaProvider, Pair.of(checkpointStr, records)); } @@ -883,7 +939,10 @@ private Schema getSchemaForWriteConfig(Schema targetSchema) { if (SchemaCompatibility.checkReaderWriterCompatibility(targetSchema, InputBatch.NULL_SCHEMA).getType() == SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE && SchemaCompatibility.checkReaderWriterCompatibility(InputBatch.NULL_SCHEMA, targetSchema).getType() == SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE) { // target schema is null. fetch schema from commit metadata and use it - HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(cfg.targetBasePath).setPayloadClassName(cfg.payloadClassName).build(); + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())) + .setBasePath(cfg.targetBasePath) + .setPayloadClassName(cfg.payloadClassName) + .build(); int totalCompleted = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants(); if (totalCompleted > 0) { try { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/postprocessor/DeleteSupportSchemaPostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/postprocessor/DeleteSupportSchemaPostProcessor.java index dd1084d2e1eab..66fff680ad4c0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/postprocessor/DeleteSupportSchemaPostProcessor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/postprocessor/DeleteSupportSchemaPostProcessor.java @@ -45,8 +45,8 @@ public DeleteSupportSchemaPostProcessor(TypedProperties props, JavaSparkContext @Override public Schema processSchema(Schema schema) { - if (schema.getField(HoodieRecord.HOODIE_IS_DELETED) != null) { - LOG.warn(String.format("column %s already exists!", HoodieRecord.HOODIE_IS_DELETED)); + if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) != null) { + LOG.warn(String.format("column %s already exists!", HoodieRecord.HOODIE_IS_DELETED_FIELD)); return schema; } @@ -57,7 +57,7 @@ public Schema processSchema(Schema schema) { targetFields.add(new Schema.Field(sourceField.name(), sourceField.schema(), sourceField.doc(), sourceField.defaultVal())); } // add _hoodie_is_deleted column - targetFields.add(new Schema.Field(HoodieRecord.HOODIE_IS_DELETED, Schema.create(Schema.Type.BOOLEAN), null, false)); + targetFields.add(new Schema.Field(HoodieRecord.HOODIE_IS_DELETED_FIELD, Schema.create(Schema.Type.BOOLEAN), null, false)); return Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false, targetFields); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java index 81f06a0f9f48f..f36263d7ec423 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java @@ -415,7 +415,6 @@ private Option getOffsetsByTimestamp(KafkaConsumer consumer, List process(JavaRDD maxwellJsonRecords) { // insert or update if (INSERT.equals(type) || UPDATE.equals(type)) { // tag this record not delete. - result.put(HoodieRecord.HOODIE_IS_DELETED, false); + result.put(HoodieRecord.HOODIE_IS_DELETED_FIELD, false); return result.toString(); // delete @@ -138,7 +138,7 @@ public JavaRDD process(JavaRDD maxwellJsonRecords) { private String processDelete(JsonNode inputJson, ObjectNode result) { // tag this record as delete. - result.put(HoodieRecord.HOODIE_IS_DELETED, true); + result.put(HoodieRecord.HOODIE_IS_DELETED_FIELD, true); PreCombineFieldType preCombineFieldType = valueOf(this.props.getString(Config.PRECOMBINE_FIELD_TYPE_PROP.key(), diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieRepairTool.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieRepairTool.java index 00cf3ae8832b2..4238439294908 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieRepairTool.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieRepairTool.java @@ -21,7 +21,6 @@ import org.apache.hudi.HoodieTestCommitGenerator; import org.apache.hudi.client.SparkRDDReadClient; -import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; @@ -36,6 +35,7 @@ import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.spark.HoodieSparkKryoProvider$; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; @@ -93,7 +93,7 @@ public void initWithCleanState() throws IOException { boolean initialized = spark != null; if (!initialized) { SparkConf sparkConf = conf(); - SparkRDDWriteClient.registerClasses(sparkConf); + HoodieSparkKryoProvider$.MODULE$.register(sparkConf); SparkRDDReadClient.addHoodieSupport(sparkConf); spark = SparkSession.builder().config(sparkConf).getOrCreate(); sqlContext = spark.sqlContext(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 715f660fffd74..d911e8bacb0ee 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -30,16 +30,19 @@ import org.apache.hudi.DataSourceReadOptions; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.HoodieSparkUtils$; +import org.apache.hudi.HoodieSparkRecordMerger; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; import org.apache.hudi.common.config.DFSPropertiesConfiguration; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; @@ -115,6 +118,8 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; @@ -126,6 +131,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Properties; import java.util.concurrent.CountDownLatch; @@ -161,21 +167,28 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase { private static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamer.class); - protected HoodieDeltaStreamer initialHoodieDeltaStreamer(String tableBasePath, int totalRecords, String asyncCluster) throws IOException { + protected HoodieDeltaStreamer initialHoodieDeltaStreamer(String tableBasePath, int totalRecords, String asyncCluster, HoodieRecordType recordType) throws IOException { HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", asyncCluster, "")); return new HoodieDeltaStreamer(cfg, jsc); } + protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute, HoodieRecordType recordType) { + return initialHoodieClusteringJob(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, null, recordType); + } + protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute) { - return initialHoodieClusteringJob(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, null); + return initialHoodieClusteringJob(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, null, HoodieRecordType.AVRO); } - protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute, Boolean retryLastFailedClusteringJob) { + protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute, + Boolean retryLastFailedClusteringJob, HoodieRecordType recordType) { HoodieClusteringJob.Config scheduleClusteringConfig = buildHoodieClusteringUtilConfig(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, retryLastFailedClusteringJob); + TestHelpers.addRecordMerger(recordType, scheduleClusteringConfig.configs); return new HoodieClusteringJob(jsc, scheduleClusteringConfig); } @@ -283,6 +296,13 @@ static HoodieDeltaStreamer.Config makeConfigForHudiIncrSrc(String srcBasePath, S return cfg; } + static void addRecordMerger(HoodieRecordType type, List hoodieConfig) { + if (type == HoodieRecordType.SPARK) { + hoodieConfig.add(String.format("%s=%s", HoodieWriteConfig.MERGER_IMPLS.key(), HoodieSparkRecordMerger.class.getName())); + hoodieConfig.add(String.format("%s=%s", HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(),"parquet")); + } + } + static void assertRecordCount(long expected, String tablePath, SQLContext sqlContext) { sqlContext.clearCache(); long recordCount = sqlContext.read().format("org.apache.hudi").load(tablePath).count(); @@ -468,14 +488,23 @@ private static HoodieDeltaStreamer.Config getBaseConfig() { */ private static Stream schemaEvolArgs() { return Stream.of( - Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, true), - Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, false), - Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, true), - Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, false), - Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, true), - Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, false), - Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, true), - Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, false)); + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, true, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, false, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, true, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, false, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, true, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, false, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, true, HoodieRecordType.AVRO), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, false, HoodieRecordType.AVRO), + + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, true, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), true, false, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, true, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL(), false, false, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, true, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), true, false, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, true, HoodieRecordType.SPARK), + Arguments.of(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL(), false, false, HoodieRecordType.SPARK)); } private static Stream provideValidCliArgs() { @@ -604,12 +633,14 @@ public void testTableCreation() throws Exception { LOG.debug("Expected error during table creation", e); } - @Test - public void testBulkInsertsAndUpsertsWithBootstrap() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testBulkInsertsAndUpsertsWithBootstrap(HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/test_table"; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); syncAndAssertRecordCount(cfg, 1000, tableBasePath, "00000", 1); // No new data => no commits. @@ -654,6 +685,9 @@ public void testBulkInsertsAndUpsertsWithBootstrap() throws Exception { assertEquals(expectedFieldNames.size(), fields.length); assertTrue(fieldNames.containsAll(HoodieRecord.HOODIE_META_COLUMNS)); assertTrue(fieldNames.containsAll(expectedFieldNames)); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, bootstrapSourcePath); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, newDatasetBasePath); } @Test @@ -696,12 +730,13 @@ private void syncAndAssertRecordCount(HoodieDeltaStreamer.Config cfg, Integer ex // TODO add tests w/ disabled reconciliation @ParameterizedTest @MethodSource("schemaEvolArgs") - public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, boolean useSchemaPostProcessor) throws Exception { + public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, boolean useSchemaPostProcessor, HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/test_table_schema_evolution" + tableType + "_" + useUserProvidedSchema + "_" + useSchemaPostProcessor; defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); // Insert data produced with Schema A, pass Schema A HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), PROPS_FILENAME_TEST_SOURCE, false, true, false, null, tableType); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.configs.add("hoodie.deltastreamer.schemaprovider.source.schema.file=" + basePath + "/source.avsc"); cfg.configs.add("hoodie.deltastreamer.schemaprovider.target.schema.file=" + basePath + "/source.avsc"); cfg.configs.add(DataSourceWriteOptions.RECONCILE_SCHEMA().key() + "=true"); @@ -715,6 +750,7 @@ public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, // Upsert data produced with Schema B, pass Schema B cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TripsWithEvolvedOptionalFieldTransformer.class.getName()), PROPS_FILENAME_TEST_SOURCE, false, true, false, null, tableType); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.configs.add("hoodie.deltastreamer.schemaprovider.source.schema.file=" + basePath + "/source.avsc"); cfg.configs.add("hoodie.deltastreamer.schemaprovider.target.schema.file=" + basePath + "/source_evolved.avsc"); cfg.configs.add(DataSourceWriteOptions.RECONCILE_SCHEMA().key() + "=true"); @@ -739,6 +775,7 @@ public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, } cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), PROPS_FILENAME_TEST_SOURCE, false, true, false, null, tableType); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.configs.add("hoodie.deltastreamer.schemaprovider.source.schema.file=" + basePath + "/source.avsc"); if (useUserProvidedSchema) { cfg.configs.add("hoodie.deltastreamer.schemaprovider.target.schema.file=" + basePath + "/source_evolved.avsc"); @@ -767,20 +804,24 @@ public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, assertEquals(expectedSchema, tableSchema); // clean up and reinit + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); UtilitiesTestBase.Helpers.deleteFileFromDfs(FSUtils.getFs(cfg.targetBasePath, jsc.hadoopConfiguration()), basePath + "/" + PROPS_FILENAME_TEST_SOURCE); writeCommonPropsToFile(fs, basePath); defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); } - @Test - public void testUpsertsCOWContinuousMode() throws Exception { - testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow"); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsCOWContinuousMode(HoodieRecordType recordType) throws Exception { + testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow", recordType); } - @Test - public void testUpsertsCOW_ContinuousModeDisabled() throws Exception { - String tableBasePath = basePath + "/non_continuous_cow"; + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsCOW_ContinuousModeDisabled(HoodieRecordType recordType) throws Exception { + String tableBasePath = basePath + "/non_continuous_cow"; HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.add(String.format("%s=%s", TURN_METRICS_ON.key(), "true")); cfg.configs.add(String.format("%s=%s", METRICS_REPORTER_TYPE_VALUE.key(), "CONSOLE")); @@ -789,22 +830,27 @@ public void testUpsertsCOW_ContinuousModeDisabled() throws Exception { ds.sync(); TestHelpers.assertRecordCount(SQL_SOURCE_NUM_RECORDS, tableBasePath, sqlContext); assertFalse(Metrics.isInitialized(), "Metrics should be shutdown"); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } - @Test - public void testUpsertsCOWContinuousModeShutdownGracefully() throws Exception { - testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow", true); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsCOWContinuousModeShutdownGracefully(HoodieRecordType recordType) throws Exception { + testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow", true, recordType); } - @Test - public void testUpsertsMORContinuousMode() throws Exception { - testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor"); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsMORContinuousMode(HoodieRecordType recordType) throws Exception { + testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor", recordType); } - @Test - public void testUpsertsMOR_ContinuousModeDisabled() throws Exception { - String tableBasePath = basePath + "/non_continuous_mor"; + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testUpsertsMOR_ContinuousModeDisabled(HoodieRecordType recordType) throws Exception { + String tableBasePath = basePath + "/non_continuous_mor"; HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.tableType = HoodieTableType.MERGE_ON_READ.name(); cfg.configs.add(String.format("%s=%s", TURN_METRICS_ON.key(), "true")); cfg.configs.add(String.format("%s=%s", METRICS_REPORTER_TYPE_VALUE.key(), "CONSOLE")); @@ -813,18 +859,20 @@ public void testUpsertsMOR_ContinuousModeDisabled() throws Exception { ds.sync(); TestHelpers.assertRecordCount(SQL_SOURCE_NUM_RECORDS, tableBasePath, sqlContext); assertFalse(Metrics.isInitialized(), "Metrics should be shutdown"); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } - private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception { - testUpsertsContinuousMode(tableType, tempDir, false); + private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir, HoodieRecordType recordType) throws Exception { + testUpsertsContinuousMode(tableType, tempDir, false, recordType); } - private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir, boolean testShutdownGracefully) throws Exception { + private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir, boolean testShutdownGracefully, HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/" + tempDir; // Keep it higher than batch-size to test continuous mode int totalRecords = 3000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; if (testShutdownGracefully) { cfg.postWriteTerminationStrategyClass = NoNewDataTerminationStrategy.class.getName(); @@ -847,6 +895,7 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir } return true; }); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer.Config cfg, Function condition) throws Exception { @@ -899,14 +948,15 @@ static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, Function getAsyncServicesConfigs(int totalRecords, String autoClean, String inlineCluster, String inlineClusterMaxCommit, @@ -1109,10 +1163,11 @@ private HoodieIndexer.Config buildIndexerConfig(String basePath, return config; } - @Test - public void testHoodieIndexer() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testHoodieIndexer(HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/asyncindexer"; - HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 1000, "false"); + HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 1000, "false", recordType); deltaStreamerTestRunner(ds, (r) -> { TestHelpers.assertAtLeastNCommits(2, tableBasePath, fs); @@ -1139,13 +1194,14 @@ public void testHoodieIndexer() throws Exception { } return true; }); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } @ParameterizedTest @ValueSource(booleans = {true, false}) public void testHoodieAsyncClusteringJob(boolean shouldPassInClusteringInstantTime) throws Exception { String tableBasePath = basePath + "/asyncClusteringJob"; - HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false"); + HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false", HoodieRecordType.AVRO); CountDownLatch countDownLatch = new CountDownLatch(1); deltaStreamerTestRunner(ds, (r) -> { @@ -1181,14 +1237,16 @@ public void testHoodieAsyncClusteringJob(boolean shouldPassInClusteringInstantTi } } - @Test - public void testAsyncClusteringService() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testAsyncClusteringService(HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/asyncClustering"; // Keep it higher than batch-size to test continuous mode int totalRecords = 2000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "3")); @@ -1201,6 +1259,7 @@ public void testAsyncClusteringService() throws Exception { TestHelpers.assertAtLeastNCommits(4, tableBasePath, fs); TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, fs); TestHelpers.assertDistinctRecordCount(totalRecords, tableBasePath, sqlContext); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } /** @@ -1209,14 +1268,16 @@ public void testAsyncClusteringService() throws Exception { * * @throws Exception */ - @Test - public void testAsyncClusteringServiceWithConflicts() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testAsyncClusteringServiceWithConflicts(HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/asyncClusteringWithConflicts"; // Keep it higher than batch-size to test continuous mode int totalRecords = 2000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "3")); @@ -1229,16 +1290,19 @@ public void testAsyncClusteringServiceWithConflicts() throws Exception { TestHelpers.assertAtLeastNCommits(4, tableBasePath, fs); TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, fs); TestHelpers.assertDistinctRecordCount(1900, tableBasePath, sqlContext); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } - @Test - public void testAsyncClusteringServiceWithCompaction() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testAsyncClusteringServiceWithCompaction(HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/asyncClusteringCompaction"; // Keep it higher than batch-size to test continuous mode int totalRecords = 2000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = true; cfg.tableType = HoodieTableType.MERGE_ON_READ.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "3")); @@ -1252,16 +1316,18 @@ public void testAsyncClusteringServiceWithCompaction() throws Exception { TestHelpers.assertAtLeastNCommits(4, tableBasePath, fs); TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, fs); TestHelpers.assertDistinctRecordCount(totalRecords, tableBasePath, sqlContext); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob) throws Exception { + @CsvSource(value = {"true, AVRO", "true, SPARK", "false, AVRO", "false, SPARK"}) + public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob, HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/asyncClustering3"; // ingest data int totalRecords = 3000; HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.continuousMode = false; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "false", "0", "false", "0")); @@ -1288,7 +1354,7 @@ public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob // trigger a scheduleAndExecute clustering job // when retryFailedClustering true => will rollback and re-execute failed clustering plan with same instant timestamp. // when retryFailedClustering false => will make and execute a new clustering plan with new instant timestamp. - HoodieClusteringJob scheduleAndExecute = initialHoodieClusteringJob(tableBasePath, null, false, "scheduleAndExecute", retryLastFailedClusteringJob); + HoodieClusteringJob scheduleAndExecute = initialHoodieClusteringJob(tableBasePath, null, false, "scheduleAndExecute", retryLastFailedClusteringJob, recordType); scheduleAndExecute.cluster(0); String completeClusteringTimeStamp = meta.getActiveTimeline().reload().getCompletedReplaceTimeline().lastInstant().get().getTimestamp(); @@ -1298,14 +1364,15 @@ public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob } else { assertFalse(clusteringRequest.getTimestamp().equalsIgnoreCase(completeClusteringTimeStamp)); } + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } @ParameterizedTest - @ValueSource(strings = {"execute", "schedule", "scheduleAndExecute"}) - public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMode) throws Exception { + @CsvSource(value = {"execute, AVRO", "schedule, AVRO", "scheduleAndExecute, AVRO", "execute, SPARK", "schedule, SPARK", "scheduleAndExecute, SPARK"}) + public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMode, HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/asyncClustering2"; - HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false"); - HoodieClusteringJob scheduleClusteringJob = initialHoodieClusteringJob(tableBasePath, null, true, runningMode); + HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false", recordType); + HoodieClusteringJob scheduleClusteringJob = initialHoodieClusteringJob(tableBasePath, null, true, runningMode, recordType); deltaStreamerTestRunner(ds, (r) -> { Exception exception = null; @@ -1345,6 +1412,9 @@ public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMod throw new IllegalStateException("Unexpected value: " + runningMode); } }); + if (runningMode.toLowerCase(Locale.ROOT).equals(SCHEDULE_AND_EXECUTE)) { + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); + } } /** @@ -1352,14 +1422,16 @@ public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMod * step involves using a SQL template to transform a source TEST-DATA-SOURCE ============================> HUDI TABLE * 1 ===============> HUDI TABLE 2 (incr-pull with transform) (incr-pull) Hudi Table 1 is synced with Hive. */ - @Test - public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline(HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/test_table2"; String downstreamTableBasePath = basePath + "/test_downstream_table2"; // Initial bulk insert to ingest to first hudi table HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT, Collections.singletonList(SqlQueryBasedTransformer.class.getName()), PROPS_FILENAME_TEST_SOURCE, true); + TestHelpers.addRecordMerger(recordType, cfg.configs); // NOTE: We should not have need to set below config, 'datestr' should have assumed date partitioning cfg.configs.add("hoodie.datasource.hive_sync.partition_fields=year,month,day"); new HoodieDeltaStreamer(cfg, jsc, fs, hiveServer.getHiveConf()).sync(); @@ -1372,6 +1444,7 @@ public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() t HoodieDeltaStreamer.Config downstreamCfg = TestHelpers.makeConfigForHudiIncrSrc(tableBasePath, downstreamTableBasePath, WriteOperationType.BULK_INSERT, true, null); + TestHelpers.addRecordMerger(recordType, downstreamCfg.configs); new HoodieDeltaStreamer(downstreamCfg, jsc, fs, hiveServer.getHiveConf()).sync(); TestHelpers.assertRecordCount(1000, downstreamTableBasePath, sqlContext); TestHelpers.assertDistanceCount(1000, downstreamTableBasePath, sqlContext); @@ -1411,6 +1484,7 @@ public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() t downstreamCfg = TestHelpers.makeConfigForHudiIncrSrc(tableBasePath, downstreamTableBasePath, WriteOperationType.UPSERT, false, null); + TestHelpers.addRecordMerger(recordType, downstreamCfg.configs); downstreamCfg.sourceLimit = 2000; new HoodieDeltaStreamer(downstreamCfg, jsc).sync(); TestHelpers.assertRecordCount(2000, downstreamTableBasePath, sqlContext); @@ -1433,6 +1507,8 @@ public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() t assertEquals(lastInstantForUpstreamTable, hiveClient.getLastCommitTimeSynced(tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, downstreamTableBasePath); } @Test @@ -1519,12 +1595,14 @@ public void testPayloadClassUpdateWithCOWTable() throws Exception { assertFalse(props.containsKey(HoodieTableConfig.PAYLOAD_CLASS_NAME.key())); } - @Test - public void testFilterDupes() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testFilterDupes(HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/test_dupes_table"; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); new HoodieDeltaStreamer(cfg, jsc).sync(); TestHelpers.assertRecordCount(1000, tableBasePath, sqlContext); TestHelpers.assertCommitMetadata("00000", tableBasePath, fs, 1); @@ -1545,6 +1623,7 @@ public void testFilterDupes() throws Exception { HoodieTableMetaClient mClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build(); HoodieInstant lastFinished = mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get(); HoodieDeltaStreamer.Config cfg2 = TestHelpers.makeDropAllConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg2.configs); cfg2.filterDupes = false; cfg2.sourceLimit = 2000; cfg2.operation = WriteOperationType.UPSERT; @@ -1570,7 +1649,7 @@ public void testFilterDupes() throws Exception { } catch (IllegalArgumentException e) { assertTrue(e.getMessage().contains("'--filter-dupes' needs to be disabled when '--op' is 'UPSERT' to ensure updates are not missed.")); } - + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } @Test @@ -2207,14 +2286,16 @@ private void insertInTable(String tableBasePath, int count, WriteOperationType o } } - @Test - public void testInsertOverwrite() throws Exception { - testDeltaStreamerWithSpecifiedOperation(basePath + "/insert_overwrite", WriteOperationType.INSERT_OVERWRITE); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testInsertOverwrite(HoodieRecordType recordType) throws Exception { + testDeltaStreamerWithSpecifiedOperation(basePath + "/insert_overwrite", WriteOperationType.INSERT_OVERWRITE, recordType); } - @Test - public void testInsertOverwriteTable() throws Exception { - testDeltaStreamerWithSpecifiedOperation(basePath + "/insert_overwrite_table", WriteOperationType.INSERT_OVERWRITE_TABLE); + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testInsertOverwriteTable(HoodieRecordType recordType) throws Exception { + testDeltaStreamerWithSpecifiedOperation(basePath + "/insert_overwrite_table", WriteOperationType.INSERT_OVERWRITE_TABLE, recordType); } @Disabled("Local run passing; flaky in CI environment.") @@ -2243,9 +2324,10 @@ public void testDeletePartitions() throws Exception { TestHelpers.assertNoPartitionMatch(tableBasePath, sqlContext, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); } - void testDeltaStreamerWithSpecifiedOperation(final String tableBasePath, WriteOperationType operationType) throws Exception { + void testDeltaStreamerWithSpecifiedOperation(final String tableBasePath, WriteOperationType operationType, HoodieRecordType recordType) throws Exception { // Initial insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); new HoodieDeltaStreamer(cfg, jsc).sync(); TestHelpers.assertRecordCount(1000, tableBasePath, sqlContext); TestHelpers.assertDistanceCount(1000, tableBasePath, sqlContext); @@ -2265,6 +2347,7 @@ void testDeltaStreamerWithSpecifiedOperation(final String tableBasePath, WriteOp TestHelpers.assertRecordCount(1950, tableBasePath, sqlContext); TestHelpers.assertDistanceCount(1950, tableBasePath, sqlContext); TestHelpers.assertCommitMetadata("00001", tableBasePath, fs, 2); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } @Test @@ -2300,11 +2383,13 @@ public void testFetchingCheckpointFromPreviousCommits() throws IOException { .getCommitsTimeline()).get().getMetadata(CHECKPOINT_KEY), "def"); } - @Test - public void testDropPartitionColumns() throws Exception { + @ParameterizedTest + @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) + public void testDropPartitionColumns(HoodieRecordType recordType) throws Exception { String tableBasePath = basePath + "/test_drop_partition_columns" + testNum++; // ingest data with dropping partition columns enabled HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + TestHelpers.addRecordMerger(recordType, cfg.configs); cfg.configs.add(String.format("%s=%s", HoodieTableConfig.DROP_PARTITION_COLUMNS.key(), "true")); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); ds.sync(); @@ -2320,6 +2405,7 @@ public void testDropPartitionColumns() throws Exception { List tableFields = tableSchema.getFields().stream().map(Schema.Field::name).collect(Collectors.toList()); // now assert that the partition column is not in the target schema assertFalse(tableFields.contains("partition_path")); + UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } @Test diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java index 7ecf12b15ed64..2c1ab1f1c1950 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java @@ -232,7 +232,7 @@ public void testMaxwellJsonKafkaSourcePostProcessor() throws IOException { // database name should be null JsonNode database = record.get("database"); // insert and update records should be tagged as no delete - boolean isDelete = record.get(HoodieRecord.HOODIE_IS_DELETED).booleanValue(); + boolean isDelete = record.get(HoodieRecord.HOODIE_IS_DELETED_FIELD).booleanValue(); assertFalse(isDelete); assertNull(database); @@ -252,7 +252,7 @@ public void testMaxwellJsonKafkaSourcePostProcessor() throws IOException { .process(inputDelete).map(mapper::readTree).foreach(record -> { // delete records should be tagged as delete - boolean isDelete = record.get(HoodieRecord.HOODIE_IS_DELETED).booleanValue(); + boolean isDelete = record.get(HoodieRecord.HOODIE_IS_DELETED_FIELD).booleanValue(); // update_time should equals ts String updateTime = record.get("update_time").textValue(); diff --git a/packaging/hudi-utilities-slim-bundle/pom.xml b/packaging/hudi-utilities-slim-bundle/pom.xml index 74de7f863985c..3f0bbf12e7501 100644 --- a/packaging/hudi-utilities-slim-bundle/pom.xml +++ b/packaging/hudi-utilities-slim-bundle/pom.xml @@ -333,6 +333,11 @@ + + org.apache.hudi + hudi-spark-client + ${project.version} +