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 fcb273f0a73bd..b253e26416055 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 @@ -24,7 +24,9 @@ import org.apache.hudi.cli.HoodiePrintHelper; import org.apache.hudi.cli.TableHeader; import org.apache.hudi.common.fs.FSUtils; +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.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Reader; @@ -81,7 +83,7 @@ public String showArchivedCommits( // read the avro blocks while (reader.hasNext()) { HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); - blk.getRecordIterator().forEachRemaining(readRecords::add); + blk.getRecordIterator(HoodieAvroIndexedRecord::new).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) @@ -155,8 +157,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(HoodieAvroIndexedRecord::new)) { + recordItr.forEachRemaining(r -> readRecords.add((IndexedRecord) 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 fa6e15b7af696..85734f923742b 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 @@ -25,7 +25,9 @@ import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.cli.HoodieCLI; import org.apache.hudi.common.fs.FSUtils; +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.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Reader; @@ -124,9 +126,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(HoodieAvroIndexedRecord::new)) { while (recordItr.hasNext()) { - IndexedRecord ir = recordItr.next(); + IndexedRecord ir = (IndexedRecord) 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() @@ -157,7 +159,7 @@ private int copyArchivedInstants(List statuses, Set actionSe default: throw new HoodieException("Unknown type of action " + action); } - + final String instantTime = archiveEntryRecord.get("commitTime").toString(); final String outPath = localFolder + Path.SEPARATOR + instantTime + "." + action; writeToFile(outPath, HoodieAvroUtils.avroToJson(metadata, true)); 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 49cc25b895730..17d413fb1f8f8 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 @@ -24,9 +24,9 @@ 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.HoodieLogFile; 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.log.HoodieLogFormat; @@ -61,6 +61,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; @@ -124,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(HoodieAvroIndexedRecord::new)) { recordItr.forEachRemaining(r -> recordCount.incrementAndGet()); } } @@ -221,8 +222,8 @@ public String showLogFileRecords( .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.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()); } @@ -238,10 +239,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(HoodieAvroIndexedRecord::new)) { recordItr.forEachRemaining(record -> { if (allRecords.size() < limit) { - allRecords.add(record); + allRecords.add((IndexedRecord) record.getData()); } }); } 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 fbfc1d8ec902e..3462f6db55cc9 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,12 +18,10 @@ 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.HoodieJsonPayload import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory} import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.util.BaseFileUtils @@ -50,7 +48,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 ee7fbda11b783..bc8f395e2c7ab 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 @@ -27,9 +27,10 @@ import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator; 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.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; @@ -104,7 +105,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()); @@ -184,7 +185,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()); @@ -223,12 +224,12 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.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-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 5c485bed0581d..662c3108b7c43 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 @@ -43,7 +43,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; @@ -124,7 +123,7 @@ * @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 c53554d8e04d2..91e2236f2f7fe 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,10 @@ 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.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; @@ -339,7 +341,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(HoodieAvroIndexedRecord::new).forEachRemaining(r -> records.add((IndexedRecord) r.getData())); if (records.size() >= this.config.getCommitArchivalBatchSize()) { writeToFile(wrapperSchema, records); } @@ -646,7 +648,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/utils/MergingIterator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MergingIterator.java index 47dde723e00dd..f84f1c00d93b3 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,18 @@ 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.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 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; @@ -47,6 +45,6 @@ public boolean hasNext() { @Override public T next() { - return mergeFunction.apply(Pair.of(leftIterator.next(), rightIterator.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..52e411108f3d2 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,13 +20,12 @@ 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.hudi.io.storage.HoodieAvroFileReader; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -38,28 +37,28 @@ /** * 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, + Option baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass, String preCombineField, Option> simpleKeyGenFieldsOpt) throws IOException { if (baseFileReader.isPresent()) { Iterator baseIterator = baseFileReader.get().getRecordIterator(schema); while (baseIterator.hasNext()) { GenericRecord record = (GenericRecord) baseIterator.next(); - HoodieRecord hoodieRecord = transform( + HoodieRecord hoodieRecord = transform( record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt); scanner.processNextRecord(hoodieRecord); } return new HoodieFileSliceReader(scanner.iterator()); } else { - Iterable> iterable = () -> scanner.iterator(); + 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(); + GenericRecord record = (GenericRecord) e.toIndexedRecord(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); @@ -68,9 +67,11 @@ public static HoodieFileSliceReader getFileSliceReader( } } - private static HoodieRecord transform( - GenericRecord record, HoodieMergedLogRecordScanner scanner, String payloadClass, - String preCombineField, Option> simpleKeyGenFieldsOpt) { + 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()) 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 5e1f832b7f239..3a8a1ee007acc 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.BoundedInMemoryQueueConsumer; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.execution.HoodieLazyInsertIterable.HoodieInsertValueGenResult; @@ -37,7 +37,7 @@ /** * Consumes stream of hoodie records from in-memory queue and writes to one or more create-handles. */ -public class CopyOnWriteInsertHandler +public class CopyOnWriteInsertHandler extends BoundedInMemoryQueueConsumer, List> { private HoodieWriteConfig config; @@ -68,9 +68,9 @@ public CopyOnWriteInsertHandler(HoodieWriteConfig config, String instantTime, } @Override - public void consumeOneRecord(HoodieInsertValueGenResult payload) { - final HoodieRecord insertPayload = payload.record; - String partitionPath = insertPayload.getPartitionPath(); + public void consumeOneRecord(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 @@ -81,19 +81,19 @@ public void consumeOneRecord(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 b078076b864f5..0ffc23d32e730 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.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,19 +74,19 @@ 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; } } @@ -98,12 +94,12 @@ public HoodieInsertValueGenResult(T record, Schema schema, Properties properties * 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( + static Function, HoodieInsertValueGenResult> getTransformFunction( Schema schema, HoodieWriteConfig config) { return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema, config.getProps()); } - static Function, HoodieInsertValueGenResult> getTransformFunction( + static Function, HoodieInsertValueGenResult> getTransformFunction( Schema schema) { return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema, CollectionUtils.EMPTY_PROPERTIES); } 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 9b3dc8df0098a..e5f8d901c853e 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 @@ -31,7 +31,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIndexException; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.table.HoodieTable; import org.apache.log4j.LogManager; @@ -132,7 +132,7 @@ public static List filterKeysFromFile(Path filePath, List candid // Load all rowKeys from the file, to double-confirm if (!candidateRecordKeys.isEmpty()) { HoodieTimer timer = new HoodieTimer().startTimer(); - HoodieFileReader fileReader = HoodieFileReaderFactory.getFileReader(configuration, filePath); + HoodieAvroFileReader fileReader = HoodieFileReaderFactory.getFileReader(configuration, filePath); Set fileRowKeys = fileReader.filterRowKeys(new TreeSet<>(candidateRecordKeys)); foundRecordKeys.addAll(fileRowKeys); LOG.info(String.format("Checked keys against file %s, in %d ms. #candidates (%d) #found (%d)", filePath, 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 426e20f83b034..599492265d840 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; @@ -67,6 +66,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 +79,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 @@ -87,7 +87,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. @@ -126,7 +126,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()); } @@ -201,8 +201,8 @@ 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(); try { // Pass the isUpdateRecord to the props for HoodieRecordPayload to judge // Whether it is an update or insert record. @@ -210,24 +210,15 @@ 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() : hoodieRecord.getData().getInsertValue(tableSchema, recordProperties); - if (avroRecord.isPresent()) { - if (avroRecord.get().equals(IGNORE_RECORD)) { - return avroRecord; + Option finalRecord = Option.empty(); + if (!nullifyPayload && hoodieRecord.isPresent(tableSchema, recordProperties)) { + if (hoodieRecord.shouldIgnore(tableSchema, recordProperties)) { + return Option.of(hoodieRecord); } // 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); - HoodieAvroUtils.addCommitMetadataToRecord(rewriteRecord, instantTime, seqId); - } - if (config.allowOperationMetadataField()) { - HoodieAvroUtils.addOperationToRecord(rewriteRecord, hoodieRecord.getOperation()); - } + HoodieRecord rewrittenRecord = hoodieRecord.rewriteRecord(tableSchema, recordProperties, schemaOnReadEnabled, writeSchemaWithMetaFields); + HoodieRecord populatedRecord = populateMetadataFields(rewrittenRecord, tableSchema, recordProperties); + finalRecord = Option.of(populatedRecord); if (isUpdateRecord) { updatedRecordsWritten++; } else { @@ -243,7 +234,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); @@ -251,6 +242,24 @@ private Option getIndexedRecord(HoodieRecord hoodieRecord) { return Option.empty(); } + private HoodieRecord populateMetadataFields(HoodieRecord hoodieRecord, Schema schema, Properties prop) throws IOException { + Map metadataValues = new HashMap<>(); + String seqId = + HoodieRecord.generateSequenceId(instantTime, getPartitionId(), RECORD_COUNTER.getAndIncrement()); + if (config.populateMetaFields()) { + metadataValues.put(HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD, fileId); + metadataValues.put(HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD, partitionPath); + metadataValues.put(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD, hoodieRecord.getRecordKey()); + metadataValues.put(HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD, instantTime); + metadataValues.put(HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD, seqId); + } + if (config.allowOperationMetadataField()) { + metadataValues.put(HoodieRecord.HoodieMetadataField.OPERATION_METADATA_FIELD, hoodieRecord.getOperation().getName()); + } + + return hoodieRecord.addMetadataValues(schema, prop, metadataValues); + } + private void initNewStatus() { HoodieDeltaWriteStat prevStat = (HoodieDeltaWriteStat) this.writeStatus.getStat(); // Make a new write status and copy basic fields over. @@ -326,7 +335,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) { @@ -359,8 +368,13 @@ private void processAppendResult(AppendResult result, List record .collect(Collectors.toList()); } + List indexedRecords = new LinkedList<>(); + for (HoodieRecord hoodieRecord : recordList) { + indexedRecords.add((IndexedRecord) hoodieRecord.toIndexedRecord(tableSchema, config.getProps()).get()); + } + Map> columnRangesMetadataMap = - collectColumnRangeMetadata(recordList, fieldsToIndex, stat.getPath()); + collectColumnRangeMetadata(indexedRecords, fieldsToIndex, stat.getPath()); stat.setRecordsStats(columnRangesMetadataMap); } @@ -418,7 +432,7 @@ public boolean canWrite(HoodieRecord record) { } @Override - public void write(HoodieRecord record, Option insertValue) { + protected void doWrite(HoodieRecord record, Schema schema, TypedProperties props) { Option> recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata(); try { init(record); @@ -491,7 +505,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; } @@ -502,12 +516,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(); + 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(tableSchema, 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)); @@ -552,17 +571,17 @@ 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, header, keyField, writeConfig.getParquetCompressionCodec()); + return new HoodieParquetDataBlock(records, header, keyField, writeConfig.getParquetCompressionCodec()); default: throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented"); } 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/HoodieConcatHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java index ca245e0c391ba..c428ab0622a66 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 @@ -18,20 +18,18 @@ package org.apache.hudi.io; +import org.apache.avro.Schema; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.model.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.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) { + String key = oldRecord.getRecordKey(keyGeneratorOpt); + Schema schema = useWriterSchemaForCompaction ? tableSchemaWithMetaFields : tableSchema; 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, schema, config.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 738e2d6b48d13..7e55b84135bd2 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,17 +19,15 @@ 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; @@ -51,11 +49,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; @@ -130,22 +128,20 @@ 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.isPresent(schema, config.getProps())) { + if (record.shouldIgnore(schema, config.getProps())) { return; } // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema if (preserveMetadata) { - fileWriter.writeAvro(record.getRecordKey(), - rewriteRecordWithMetadata((GenericRecord) avroRecord.get(), path.getName())); + fileWriter.write(record.getRecordKey(), record.rewriteRecordWithMetadata( + schema, config.getProps(), schemaOnReadEnabled, writeSchemaWithMetaFields, path.getName()), writeSchemaWithMetaFields); } else { - fileWriter.writeAvroWithMetadata(record.getKey(), rewriteRecord((GenericRecord) avroRecord.get())); + fileWriter.writeWithMetadata(record.getKey(), record.rewriteRecordWithMetadata( + schema, config.getProps(), schemaOnReadEnabled, writeSchemaWithMetaFields, path.getName()), writeSchemaWithMetaFields); } // update the new location of record, so we know where to find it next record.unseal(); @@ -180,18 +176,14 @@ 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(tableSchemaWithMetaFields, config.getProps())); - } else { - write(record, record.getData().getInsertValue(tableSchema, config.getProps())); - } + while (keyIterator.hasNext()) { + final String key = keyIterator.next(); + HoodieRecord record = recordMap.get(key); + if (useWriterSchema) { + write(record, tableSchemaWithMetaFields, config.getProps()); + } else { + write(record, useWriterSchema ? tableSchemaWithMetaFields : tableSchema, config.getProps()); } - } catch (IOException io) { - throw new HoodieInsertException("Failed to insert records for path " + path, io); } } 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 36ee7d96709ab..902a38b93b206 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,18 +18,16 @@ 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; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndexUtils; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.table.HoodieTable; - -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -43,7 +41,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); @@ -69,7 +67,7 @@ && getCompletedMetadataPartitions(hoodieTable.getMetaClient().getTableConfig()) bloomFilter = hoodieTable.getMetadataTable().getBloomFilter(partitionPathFileIDPair.getLeft(), partitionPathFileIDPair.getRight()) .orElseThrow(() -> new HoodieIndexException("BloomFilter missing for " + partitionPathFileIDPair.getRight())); } else { - try (HoodieFileReader reader = createNewFileReader()) { + try (HoodieAvroFileReader reader = createNewFileReader()) { bloomFilter = reader.readBloomFilter(); } } 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 92fa5c28394a5..f16a4dff24e21 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,7 +28,6 @@ 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; @@ -40,16 +40,13 @@ import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieUpsertException; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.keygen.BaseKeyGenerator; -import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; -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; @@ -62,6 +59,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; @SuppressWarnings("Duplicates") @@ -94,13 +92,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; @@ -257,12 +255,12 @@ protected void init(String fileId, Iterator> newRecordsItr) { + ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); } - private boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option indexedRecord) { + private boolean writeUpdateRecord(HoodieRecord hoodieRecord, HoodieRecord oldRecord, Option combineRecordOp) throws IOException { boolean isDelete = false; - if (indexedRecord.isPresent()) { + Schema schema = useWriterSchemaForCompaction ? tableSchemaWithMetaFields : tableSchema; + if (combineRecordOp.isPresent()) { updatedRecordsWritten++; - GenericRecord record = (GenericRecord) indexedRecord.get(); - if (oldRecord != record) { + if (oldRecord.getData() != combineRecordOp.get().getData()) { // the incoming record is chosen isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation()); } else { @@ -270,27 +268,26 @@ private boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord ol return false; } } - return writeRecord(hoodieRecord, indexedRecord, isDelete); + return writeRecord(hoodieRecord, combineRecordOp, schema, config.getProps(), isDelete); } protected void writeInsertRecord(HoodieRecord hoodieRecord) throws IOException { Schema schema = useWriterSchemaForCompaction ? tableSchemaWithMetaFields : tableSchema; - Option insertRecord = hoodieRecord.getData().getInsertValue(schema, config.getProps()); // just skip the ignored record - if (insertRecord.isPresent() && insertRecord.get().equals(IGNORE_RECORD)) { + if (hoodieRecord.shouldIgnore(schema, config.getProps())) { return; } - if (writeRecord(hoodieRecord, insertRecord, HoodieOperation.isDelete(hoodieRecord.getOperation()))) { + if (writeRecord(hoodieRecord, Option.of(hoodieRecord), schema, config.getProps(), HoodieOperation.isDelete(hoodieRecord.getOperation()))) { insertRecordsWritten++; } } - protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord) { - return writeRecord(hoodieRecord, indexedRecord, false); + protected boolean writeRecord(HoodieRecord hoodieRecord, Option combineRecord, Schema schema, Properties prop) throws IOException { + return writeRecord(hoodieRecord, combineRecord, schema, prop, false); } - protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord, boolean isDelete) { - Option recordMetadata = hoodieRecord.getData().getMetadata(); + protected boolean writeRecord(HoodieRecord hoodieRecord, Option combineRecord, Schema schema, Properties prop, boolean isDelete) throws IOException { + Option recordMetadata = hoodieRecord.getMetadata(); if (!partitionPath.equals(hoodieRecord.getPartitionPath())) { HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: " + hoodieRecord.getPartitionPath() + " but trying to insert into partition: " + partitionPath); @@ -298,8 +295,8 @@ protected boolean writeRecord(HoodieRecord hoodieRecord, Option hoodieRecord, Option oldRecord) { + String key = oldRecord.getRecordKey(keyGeneratorOpt); boolean copyOldRecord = true; + Schema schema = useWriterSchemaForCompaction ? tableSchemaWithMetaFields : tableSchema; + 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(); try { - Option combinedAvroRecord = - hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, - useWriterSchemaForCompaction ? tableSchemaWithMetaFields : tableSchema, - config.getPayloadConfig().getProps()); + Option combinedRecord = + hoodieRecord.combineAndGetUpdateValue(oldRecord, + schema, + props); - if (combinedAvroRecord.isPresent() && combinedAvroRecord.get().equals(IGNORE_RECORD)) { + if (combinedRecord.isPresent() && combinedRecord.get().shouldIgnore(schema, 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(hoodieRecord, oldRecord, combinedRecord)) { /* * ONLY WHEN 1) we have an update for this key AND 2) We are able to successfully * write the combined new value @@ -355,7 +354,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, schema, 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)); @@ -366,13 +365,13 @@ 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 { 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(), record.rewriteRecordWithMetadata(schema, prop, schemaOnReadEnabled, writeSchemaWithMetaFields, newFilePath.getName()), writeSchemaWithMetaFields); } else { - fileWriter.writeAvroWithMetadata(key, rewriteRecord(avroRecord)); + fileWriter.writeWithMetadata(key, record.rewriteRecord(schema, prop, schemaOnReadEnabled, writeSchemaWithMetaFields), writeSchemaWithMetaFields); } } @@ -437,7 +436,7 @@ public void performMergeDataValidationCheck(WriteStatus writeStatus) { long oldNumWrites = 0; try { - HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(hoodieTable.getHadoopConf(), oldFilePath); + HoodieAvroFileReader reader = HoodieFileReaderFactory.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/HoodieRangeInfoHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java index abe4a9befef9b..753d1db3010fb 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 @@ -18,10 +18,9 @@ package org.apache.hudi.io; -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; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.table.HoodieTable; import java.io.IOException; @@ -29,7 +28,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) { @@ -37,7 +36,7 @@ public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable h } public String[] getMinMaxKeys() throws IOException { - try (HoodieFileReader reader = createNewFileReader()) { + try (HoodieAvroFileReader reader = createNewFileReader()) { return reader.readMinMaxRecordKeys(); } } 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 fee75b22decd7..b699e9700c1f6 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,11 +21,10 @@ 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; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.table.HoodieTable; @@ -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; @@ -62,7 +61,7 @@ protected HoodieBaseFile getLatestDataFile() { .getLatestBaseFile(partitionPathFileIDPair.getLeft(), partitionPathFileIDPair.getRight()).get(); } - protected HoodieFileReader createNewFileReader() throws IOException { + protected HoodieAvroFileReader createNewFileReader() throws IOException { return HoodieFileReaderFactory.getFileReader(hoodieTable.getHadoopConf(), new Path(getLatestDataFile().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 7dce31a4c349b..60c6a2da7fb1f 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 @@ -22,16 +22,12 @@ 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 +44,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 +71,8 @@ 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) { + String key = oldRecord.getRecordKey(keyGeneratorOpt); // To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than // the oldRecord's key. @@ -94,9 +90,9 @@ public void write(GenericRecord oldRecord) { } try { if (useWriterSchemaForCompaction) { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(tableSchemaWithMetaFields, config.getProps())); + writeRecord(hoodieRecord, Option.of(hoodieRecord), tableSchemaWithMetaFields, config.getProps()); } else { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(tableSchema, config.getProps())); + writeRecord(hoodieRecord, Option.of(hoodieRecord), tableSchema, config.getProps()); } insertRecordsWritten++; writtenRecordKeys.add(keyToPreWrite); @@ -117,9 +113,9 @@ public List close() { HoodieRecord hoodieRecord = keyToNewRecords.get(key); if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) { if (useWriterSchemaForCompaction) { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(tableSchemaWithMetaFields, config.getProps())); + writeRecord(hoodieRecord, Option.of(hoodieRecord), tableSchemaWithMetaFields, config.getProps()); } else { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(tableSchema, config.getProps())); + writeRecord(hoodieRecord, Option.of(hoodieRecord), tableSchema, config.getProps()); } insertRecordsWritten++; } 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 b7fdbecfd56d1..09f8831f8b71d 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,12 +18,15 @@ package org.apache.hudi.io; +import org.apache.avro.Schema; +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.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; @@ -34,48 +37,22 @@ import org.apache.hudi.io.storage.HoodieFileWriterFactory; 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.List; -import java.util.HashMap; import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; /** * 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(); - /** * The specified schema of the table. ("specified" denotes that this is configured by the client, * as opposed to being implicitly fetched out of the commit metadata) @@ -208,35 +185,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(); @@ -269,35 +226,7 @@ protected long getAttemptId() { } protected HoodieFileWriter createNewFileWriter(String instantTime, Path path, HoodieTable hoodieTable, - HoodieWriteConfig config, Schema schema, TaskContextSupplier taskContextSupplier) throws IOException { + HoodieWriteConfig config, Schema schema, TaskContextSupplier taskContextSupplier) throws IOException { return HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, schema, taskContextSupplier); } - - 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; - } - } } 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/HoodieHFileWriter.java b/hudi-client/hudi-client-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-client/hudi-client-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-client/hudi-client-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 - implements HoodieFileWriter, Closeable { +public class HoodieAvroOrcWriter implements HoodieAvroFileWriter, Closeable { private static final AtomicLong RECORD_INDEX = new AtomicLong(1); private final long maxFileSize; @@ -68,8 +66,8 @@ public class HoodieOrcWriter +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/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index ffdff25738ed7..29d4b44d42651 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -23,7 +23,6 @@ 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; @@ -46,7 +45,7 @@ public class HoodieFileWriterFactory { - public static HoodieFileWriter getFileWriter( + 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()); @@ -64,14 +63,14 @@ public static throw new UnsupportedOperationException(extension + " format not supported yet."); } - private static HoodieFileWriter newParquetFileWriter( + private static HoodieAvroFileWriter 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( + private static HoodieAvroFileWriter 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(); @@ -81,34 +80,34 @@ private static HoodieFi config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(), conf, config.getParquetCompressionRatio(), config.parquetDictionaryEnabled()); - return new HoodieAvroParquetWriter<>(path, parquetConfig, instantTime, taskContextSupplier, populateMetaFields); + return new HoodieAvroParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, populateMetaFields); } - static HoodieFileWriter newHFileFileWriter( + static HoodieAvroFileWriter 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, + HoodieAvroHFileReader.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()); + return new HoodieAvroHFileWriter(instantTime, path, hfileConfig, schema, taskContextSupplier, config.populateMetaFields()); } - private static HoodieFileWriter newOrcFileWriter( + private static HoodieAvroFileWriter 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); + return new HoodieAvroOrcWriter(instantTime, path, orcConfig, schema, taskContextSupplier); } private static BloomFilter createBloomFilter(HoodieWriteConfig config) { return BloomFilterFactory.createBloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP(), - config.getDynamicBloomFilterMaxNumEntries(), - config.getBloomFilterType()); + config.getDynamicBloomFilterMaxNumEntries(), + config.getBloomFilterType()); } } 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 eeb287abd543c..8d44c603c6cc2 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 java.io.IOException; import java.util.Iterator; @@ -34,10 +33,10 @@ * * @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); } 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 56526d23db006..e3c40deba07c2 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 @@ -48,7 +48,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.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -111,7 +110,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 f893b4ccd5c4e..18e5e17a6be06 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 8966a5d51c7cb..73ad53e4a6c16 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.BoundedInMemoryQueueConsumer; -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 protected void consumeOneRecord(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.getWriterSchemaWithMetaFields(), 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 30ed27b39b77a..c3d550d475058 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 @@ -28,7 +28,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; @@ -56,7 +55,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 d8e51bcd1643e..41815b02a7190 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 @@ -24,7 +24,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; @@ -45,7 +44,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 79eef43b3c00a..af8b4dcecd60a 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 @@ -29,7 +29,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; -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.timeline.HoodieActiveTimeline; @@ -71,7 +70,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 15ead5efb0080..f0021a0125cb7 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..a4b09d006ee21 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,6 @@ 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.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -34,7 +33,7 @@ /** * 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; 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 a96ff73947cdb..53f71b6ee8c26 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 5d62ef390233f..00d0e3ba37795 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 4e33eb06038cd..e055b4477cc35 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 java.util.Set; @@ -28,7 +27,7 @@ /** * When file groups in clustering, write records to these file group need to check. */ -public abstract class UpdateStrategy { +public abstract class UpdateStrategy { protected final HoodieEngineContext engineContext; protected Set fileGroupsInPendingClustering; 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 5355194ff75bf..5b7118c6d906b 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 31c8bbd6d30d2..198b470fcc1cb 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); 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 5ead348140aa3..393ee9ddb698c 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,13 @@ 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.model.HoodieAvroIndexedRecord; 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.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; @@ -40,6 +41,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import javax.annotation.Nonnull; + import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Iterator; @@ -47,7 +50,7 @@ /** * Helper to read records from previous version of base file and run Merge. */ -public abstract class BaseMergeHelper { +public abstract class BaseMergeHelper { /** * Read records from previous version of base file and merge. @@ -57,7 +60,7 @@ public abstract class BaseMergeHelper { */ public abstract void runMerge(HoodieTable table, HoodieMergeHandle upsertHandle) throws IOException; - protected GenericRecord transformRecordBasedOnNewSchema(GenericDatumReader gReader, GenericDatumWriter gWriter, + protected HoodieRecord transformRecordBasedOnNewSchema(GenericDatumReader gReader, GenericDatumWriter gWriter, ThreadLocal encoderCache, ThreadLocal decoderCache, GenericRecord gRec) { ByteArrayOutputStream inStream = null; @@ -71,7 +74,7 @@ protected GenericRecord transformRecordBasedOnNewSchema(GenericDatumReader getMergingIterator(HoodieTable table, HoodieMergeHandle mergeHandle, - HoodieBaseFile baseFile, HoodieFileReader reader, - Schema readSchema, boolean externalSchemaTransformation) throws IOException { + protected Iterator getMergingIterator(HoodieTable table, + HoodieMergeHandle mergeHandle, + HoodieBaseFile baseFile, + HoodieFileReader reader, + Schema readerSchema, + boolean externalSchemaTransformation) throws IOException { Path externalFilePath = new Path(baseFile.getBootstrapBaseFile().get().getPath()); Configuration bootstrapFileConfig = new Configuration(table.getHadoopConf()); - HoodieFileReader bootstrapReader = HoodieFileReaderFactory.getFileReader(bootstrapFileConfig, externalFilePath); + HoodieFileReader bootstrapReader = HoodieFileReaderFactory.getFileReader(bootstrapFileConfig, externalFilePath); + Schema bootstrapReadSchema; if (externalSchemaTransformation) { bootstrapReadSchema = bootstrapReader.getSchema(); @@ -102,14 +108,25 @@ protected Iterator getMergingIterator(HoodieTable tab bootstrapReadSchema = mergeHandle.getWriterSchema(); } - return new MergingIterator<>(reader.getRecordIterator(readSchema), bootstrapReader.getRecordIterator(bootstrapReadSchema), - (inputRecordPair) -> HoodieAvroUtils.stitchRecords(inputRecordPair.getLeft(), inputRecordPair.getRight(), mergeHandle.getWriterSchemaWithMetaFields())); + return new MergingIterator<>( + reader.getRecordIterator(readerSchema, HoodieAvroIndexedRecord::new), + bootstrapReader.getRecordIterator(bootstrapReadSchema, HoodieAvroIndexedRecord::new), + (oneRecord, otherRecord) -> mergeRecords(oneRecord, otherRecord, readerSchema, mergeHandle.getWriterSchemaWithMetaFields())); + } + + @Nonnull + private static HoodieRecord mergeRecords(HoodieRecord one, HoodieRecord other, Schema readerSchema, Schema writerSchema) { + try { + return one.mergeWith(other, readerSchema, writerSchema); + } catch (IOException e) { + throw new HoodieIOException("Failed to merge records", e); + } } /** * Consumer that dequeues records from queue and sends to Merge Handle. */ - protected static class UpdateHandler extends BoundedInMemoryQueueConsumer { + protected static class UpdateHandler extends BoundedInMemoryQueueConsumer { private final HoodieMergeHandle upsertHandle; @@ -118,12 +135,13 @@ protected UpdateHandler(HoodieMergeHandle upsertHandle) { } @Override - protected void consumeOneRecord(GenericRecord record) { + protected void consumeOneRecord(HoodieRecord record) { upsertHandle.write(record); } @Override - protected void finish() {} + protected void finish() { + } @Override protected Void getResult() { 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 846afec7c1db3..d8682152fa93e 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 @@ -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.common.model.WriteOperationType; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; @@ -30,7 +29,7 @@ import java.time.Duration; import java.time.Instant; -public abstract class BaseWriteHelper { +public abstract class BaseWriteHelper { public HoodieWriteMetadata write(String instantTime, I inputRecords, 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..f54184abb0169 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 @@ -26,7 +26,6 @@ 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.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; @@ -45,7 +44,7 @@ * @param */ @SuppressWarnings("checkstyle:LineLength") -public class HoodieDeleteHelper extends +public class HoodieDeleteHelper extends BaseDeleteHelper>, HoodieData, HoodieData, R> { private HoodieDeleteHelper() { } 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 3e2d8abdd7466..6935e67f2d5b2 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 @@ -19,14 +19,15 @@ package org.apache.hudi.table.action.commit; import org.apache.avro.SchemaCompatibility; -import org.apache.hudi.avro.HoodieAvroUtils; + import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; +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.HoodieRecordPayload; +import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; @@ -49,17 +50,22 @@ import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.BinaryEncoder; import org.apache.hadoop.conf.Configuration; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.HashMap; import java.util.Map; +import java.util.Properties; import java.util.stream.Collectors; -public class HoodieMergeHelper extends +public class HoodieMergeHelper extends BaseMergeHelper>, HoodieData, HoodieData> { + private static final Logger LOG = LogManager.getLogger(HoodieMergeHelper.class); + private HoodieMergeHelper() { } @@ -92,7 +98,7 @@ public void runMerge(HoodieTable>, HoodieData wrapper = null; - HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); Option querySchemaOpt = SerDeHelper.fromJson(table.getConfig().getInternalSchema()); boolean needToReWriteRecord = false; @@ -124,14 +130,14 @@ public void runMerge(HoodieTable>, HoodieData readerIterator; + final Iterator readerIterator; if (baseFile.getBootstrapBaseFile().isPresent()) { readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation); } else { if (needToReWriteRecord) { - readerIterator = HoodieAvroUtils.rewriteRecordWithNewSchema(reader.getRecordIterator(), readSchema, renameCols); + readerIterator = new RewriteIterator(reader.getRecordIterator(HoodieAvroIndexedRecord::new), readSchema, readSchema, table.getConfig().getProps(), renameCols); } else { - readerIterator = reader.getRecordIterator(readSchema); + readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new); } } @@ -142,7 +148,8 @@ public void runMerge(HoodieTable>, HoodieData>, HoodieData { + + private final ClosableIterator iter; + private final Schema newSchema; + private final Schema recordSchema; + private final Properties prop; + private final Map renameCols; + + public RewriteIterator(ClosableIterator iter, Schema newSchema, Schema recordSchema, Properties prop, Map renameCols) { + this.iter = iter; + this.newSchema = newSchema; + this.recordSchema = recordSchema; + this.prop = prop; + this.renameCols = renameCols; + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public HoodieRecord next() { + try { + return iter.next().rewriteRecordWithNewSchema(recordSchema, prop, newSchema, renameCols); + } catch (IOException e) { + LOG.error("Error rewrite record with new schema", e); + throw new HoodieException(e); + } + } + + @Override + public void close() { + iter.close(); + } + } } 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 b56d39b8e3679..e24cd71ab64c4 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 @@ -21,15 +21,13 @@ 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.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.collection.Pair; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; -public class HoodieWriteHelper extends BaseWriteHelper>, +public class HoodieWriteHelper extends BaseWriteHelper>, HoodieData, HoodieData, R> { private HoodieWriteHelper() { @@ -60,10 +58,10 @@ public HoodieData> deduplicateRecords( return Pair.of(key, record); }).reduceByKey((rec1, rec2) -> { @SuppressWarnings("unchecked") - T reducedData = (T) rec2.getData().preCombine(rec1.getData()); - HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey(); + HoodieRecord reducedRec = rec2.preCombine(rec1); + HoodieKey reducedKey = rec1.getData().equals(reducedRec) ? rec1.getKey() : rec2.getKey(); - return new HoodieAvroRecord<>(reducedKey, reducedData); + return (HoodieRecord) reducedRec.newInstance(reducedKey); }, parallelism).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 3379d16f4c035..75fbb6b27ada3 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 75954872aedd5..69c6b48feaf92 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 @@ -31,7 +31,6 @@ import org.apache.hudi.common.model.HoodieBaseFile; 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.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -75,7 +74,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); 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 fc4ae986e6d55..bfee3adbd4c81 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; @@ -45,7 +44,7 @@ import java.util.List; @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 05fb7c0c92d1d..e662d21a5e36e 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 @@ -22,7 +22,6 @@ import org.apache.hudi.common.engine.EngineType; 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.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -48,7 +47,7 @@ import java.util.Set; import java.util.stream.Collectors; -public class ScheduleCompactionActionExecutor extends BaseActionExecutor> { +public class ScheduleCompactionActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class); 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 339e95b9e08d4..0f31f7157c6a4 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 cfd975c5098a4..202da9bf8159c 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 1fac279f8ec2c..19a688026a3ae 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 facab71c6237b..5c4ee0f0aa900 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 661cee4a2e608..39a8abde3b264 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 8e34f0fe59dac..c26abccadb84e 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 f95ec5d5c9fe1..541fa87206288 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 5315ce713eef3..759ac04f45806 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 e4054e9221969..4e63278ddcfdd 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/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java index da6f717258877..0dd3f78935dea 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 @@ -33,6 +33,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; 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.util.FileIOUtils; @@ -68,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; @@ -88,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() @@ -103,15 +104,15 @@ protected HoodieFileWriter createWriter( when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier); when(partitionSupplier.get()).thenReturn(10); - return HoodieFileWriterFactory.newHFileFileWriter( + return (HoodieAvroHFileWriter)HoodieFileWriterFactory.newHFileFileWriter( instantTime, getFilePath(), writeConfig, avroSchema, conf, mockTaskContextSupplier); } @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 @@ -143,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++) { @@ -166,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(); @@ -181,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); @@ -211,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)); @@ -221,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(); 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()); @@ -246,18 +247,18 @@ 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 = + Iterator iterator = hfileReader.getRecordsByKeyPrefixIterator(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(HoodieAvroIndexedRecord::new)).map(r -> (GenericRecord)r.getData()).collect(Collectors.toList()); assertEquals(allRecords, recordsByPrefix); @@ -267,6 +268,7 @@ public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception { hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key1"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(expectedKey1s, recordsByPrefix); @@ -276,6 +278,7 @@ public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception { hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key25"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(expectedKey25, recordsByPrefix); @@ -284,6 +287,7 @@ public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception { hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key99"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(Collections.emptyList(), recordsByPrefix); @@ -292,6 +296,7 @@ public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception { hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key1234"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) + .map(r -> (GenericRecord)r) .collect(Collectors.toList()); assertEquals(Collections.emptyList(), recordsByPrefix); } @@ -315,8 +320,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)); @@ -324,7 +329,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 282f102697d78..0fcd3437afce1 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 @@ -25,7 +25,6 @@ import org.apache.hudi.config.HoodieStorageConfig; 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; @@ -50,7 +49,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(); @@ -60,11 +59,11 @@ protected HoodieFileWriter createWriter( HoodieOrcConfig config = new HoodieOrcConfig(conf, CompressionKind.ZLIB, orcStripSize, orcBlockSize, maxFileSize, filter); TaskContextSupplier mockTaskContextSupplier = Mockito.mock(TaskContextSupplier.class); 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()); } 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 4617eb93a66e7..cb0cc63c90d58 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 @@ -20,10 +20,13 @@ package org.apache.hudi.io.storage; import org.apache.hudi.common.bloom.BloomFilter; +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; @@ -59,10 +62,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; @@ -86,7 +89,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); @@ -109,7 +112,7 @@ public void testWriteReadPrimitiveRecord() throws Exception { Configuration conf = new Configuration(); verifyMetadata(conf); verifySchema(conf, schemaPath); - verifySimpleRecords(createReader(conf).getRecordIterator()); + verifySimpleRecords(new TransformIterator(createReader(conf).getRecordIterator(HoodieAvroIndexedRecord::new))); } @Test @@ -117,7 +120,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); @@ -136,7 +139,7 @@ public void testWriteReadComplexRecord() throws Exception { Configuration conf = new Configuration(); verifyMetadata(conf); verifySchema(conf, schemaPath); - verifyComplexRecords(createReader(conf).getRecordIterator()); + verifyComplexRecords(new TransformIterator(createReader(conf).getRecordIterator(HoodieAvroIndexedRecord::new))); } @Test @@ -144,7 +147,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", @@ -165,22 +168,23 @@ 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 verifySimpleRecords(Iterator iterator) { + protected void verifySimpleRecords(Iterator iterator) { int index = 0; while (iterator.hasNext()) { - GenericRecord record = iterator.next(); + GenericRecord record = (GenericRecord) iterator.next(); String key = "key" + String.format("%02d", index); assertEquals(key, record.get("_row_key").toString()); assertEquals(Integer.toString(index), record.get("time").toString()); @@ -189,10 +193,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(); String key = "key" + String.format("%02d", index); assertEquals(key, record.get("_row_key").toString()); assertEquals(Integer.toString(index), record.get("time").toString()); @@ -216,7 +220,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) @@ -225,12 +229,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(), index); index++; } } @@ -248,4 +252,24 @@ private void verifyRecord(String schemaPath, GenericRecord record, int index) { } assertNull(record.get("added_field")); } + + class TransformIterator implements Iterator { + + private final Iterator iter; + + public TransformIterator(Iterator iter) { + this.iter = iter; + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public IndexedRecord next() { + return (GenericRecord) iter.next().getData(); + + } + } } 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 6b847d4960fb6..bf73d4dd6e9e1 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 @@ -19,11 +19,18 @@ package org.apache.hudi.testutils; +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.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; 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.HoodieFileFormat; +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; @@ -38,18 +45,11 @@ 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.io.storage.HoodieAvroOrcWriter; import org.apache.hudi.io.storage.HoodieAvroParquetConfig; -import org.apache.hudi.io.storage.HoodieOrcConfig; -import org.apache.hudi.io.storage.HoodieOrcWriter; import org.apache.hudi.io.storage.HoodieAvroParquetWriter; +import org.apache.hudi.io.storage.HoodieOrcConfig; import org.apache.hudi.metadata.HoodieTableMetadataWriter; - -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.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.orc.CompressionKind; @@ -113,7 +113,7 @@ public Path withInserts(String partition, String fileId, List reco HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, new Configuration(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue())); - 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; @@ -135,7 +135,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)) { @@ -182,7 +182,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-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 ddfbabaf36ae9..50cbbc79f2920 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 @@ -28,7 +28,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.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; @@ -81,7 +80,7 @@ import java.util.stream.Collectors; @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/execution/ExplicitWriteHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java index 46eff587575cc..ca361bbdc15b4 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.BoundedInMemoryQueueConsumer; import org.apache.hudi.io.HoodieWriteHandle; @@ -30,7 +30,7 @@ /** * Consumes stream of hoodie records from in-memory queue and writes to one explicit create handle. */ -public class ExplicitWriteHandler +public class ExplicitWriteHandler extends BoundedInMemoryQueueConsumer, List> { private final List statuses = new ArrayList<>(); @@ -42,9 +42,9 @@ public ExplicitWriteHandler(HoodieWriteHandle handle) { } @Override - public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { - final HoodieRecord insertPayload = payload.record; - handle.write(insertPayload, payload.insertValue, payload.exception); + public void consumeOneRecord(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 2377ea6ffe51a..3c664de6f2175 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 @@ -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.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; @@ -36,7 +35,7 @@ import java.util.Iterator; import java.util.List; -public class FlinkLazyInsertIterable extends HoodieLazyInsertIterable { +public class FlinkLazyInsertIterable extends HoodieLazyInsertIterable { public FlinkLazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, 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 66c1b07793ee7..941d1724f86fd 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.HoodieList; 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 b514896aa1e3a..6714a3b879df4 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..662e8381e661c 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,12 @@ package org.apache.hudi.io; +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 +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 FlinkConcatAndReplaceHandle +public class FlinkConcatAndReplaceHandle extends FlinkMergeAndReplaceHandle { private static final Logger LOG = LoggerFactory.getLogger(FlinkConcatAndReplaceHandle.class); @@ -59,10 +55,10 @@ 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) { + String key = oldRecord.getRecordKey(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..8e4fb50e52cbe 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 @@ -20,13 +20,9 @@ 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 +36,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,10 +54,10 @@ 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) { + String key = oldRecord.getRecordKey(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/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 24da25b20be1e..bc987d835ce63 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/FlinkMergeHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java index e1117712634cf..b455904c50a20 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/table/ExplicitWriteHandleTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/ExplicitWriteHandleTable.java index f799919287606..9d017996fc9fc 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 0e5f1c26e32f4..5335f07a59398 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.table.timeline.HoodieTimeline; @@ -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); @@ -412,7 +411,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 a65e03da761d3..b6dd76e106a54 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.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -45,7 +44,7 @@ import java.util.List; import java.util.Map; -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 6eae15e7e1aff..519debc4ba493 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.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; @@ -42,7 +41,7 @@ import static org.apache.hudi.common.data.HoodieList.getList; -public abstract class HoodieFlinkTable +public abstract class HoodieFlinkTable extends HoodieTable>, List, List> implements ExplicitWriteHandleTable { @@ -50,7 +49,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()) @@ -59,13 +58,13 @@ public static HoodieFlinkTable create(HoodieW return HoodieFlinkTable.create(config, context, metaClient); } - public static HoodieFlinkTable create(HoodieWriteConfig config, + public static HoodieFlinkTable create(HoodieWriteConfig config, HoodieFlinkEngineContext context, HoodieTableMetaClient metaClient) { return HoodieFlinkTable.create(config, context, metaClient, config.isMetadataTableEnabled()); } - public static HoodieFlinkTable create(HoodieWriteConfig config, + public static HoodieFlinkTable create(HoodieWriteConfig config, HoodieFlinkEngineContext context, HoodieTableMetaClient metaClient, boolean refreshTimeline) { 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 51138cd29daa6..fc670d1f4bc73 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 2064be3e67113..0308af2885a0e 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; @@ -30,7 +29,7 @@ import java.util.List; -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 363aa28675a4a..6f792d5d3f2a5 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; @@ -30,7 +29,7 @@ import java.util.List; -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 ca1ae6760b797..81c0863016102 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; @@ -31,7 +30,7 @@ import java.util.List; -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 f64d8d20d0b26..f95b83ce6a997 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; @@ -30,7 +29,7 @@ import java.util.List; -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 459a6dbc8f672..d6358fcc5cf68 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; @@ -30,7 +29,7 @@ import java.util.List; -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/FlinkMergeHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java index 31312655251ab..81b9be70a7a8a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java @@ -20,17 +20,18 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieBaseFile; +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.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.io.HoodieMergeHandle; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.table.HoodieTable; +import scala.collection.immutable.List; import org.apache.avro.Schema; import org.apache.avro.generic.GenericDatumReader; @@ -43,9 +44,7 @@ import java.io.IOException; import java.util.Iterator; -import scala.collection.immutable.List; - -public class FlinkMergeHelper extends BaseMergeHelper>, +public class FlinkMergeHelper extends BaseMergeHelper>, List, List> { private FlinkMergeHelper() { @@ -80,23 +79,24 @@ public void runMerge(HoodieTable>, List, List BoundedInMemoryExecutor wrapper = null; Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf()); - HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); + HoodieAvroFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); try { - final Iterator readerIterator; + final Iterator readerIterator; if (baseFile.getBootstrapBaseFile().isPresent()) { readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation); } else { - readerIterator = reader.getRecordIterator(readSchema); + readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new); } ThreadLocal encoderCache = new ThreadLocal<>(); ThreadLocal decoderCache = new ThreadLocal<>(); - wrapper = new BoundedInMemoryExecutor<>(table.getConfig().getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(readerIterator), + wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(readerIterator), Option.of(new UpdateHandler(mergeHandle)), record -> { if (!externalSchemaTransformation) { return record; } - return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record); + // TODO Other type of record need to change + return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) ((HoodieRecord)record).getData()); }); wrapper.execute(); } catch (Exception e) { 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 c76733db1de35..5622792e279f1 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; @@ -30,7 +29,7 @@ import java.util.List; -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 42d932a2bd7f9..1c530a6421935 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; @@ -30,7 +29,7 @@ import java.util.List; -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 9c17e77b91831..cc639a4c1656c 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 @@ -21,11 +21,9 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieList; 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.WriteOperationType; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; @@ -48,7 +46,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() { @@ -95,17 +93,14 @@ public List> deduplicateRecords( .collect(Collectors.groupingBy(record -> record.getKey().getRecordKey())); 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); + @SuppressWarnings("unchecked") final HoodieRecord reducedRec = rec2.preCombine(rec1); // 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 == reducedRec; HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey(); HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation(); - HoodieRecord hoodieRecord = new HoodieAvroRecord<>(reducedKey, reducedData, operation); + HoodieRecord hoodieRecord = reducedRec.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 1fe98204b1cb1..da6b238da9ef9 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; @@ -34,7 +33,7 @@ import java.util.Iterator; import java.util.List; -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 7053f7a16203f..c59b99afcd474 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; @@ -30,7 +29,7 @@ import java.util.List; -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 493c894c8a96c..5d51b1ae06f35 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; @@ -30,7 +29,7 @@ import java.util.List; -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 03b9f8e7ee090..0dec2615bb470 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.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; @@ -38,7 +37,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 29f0e03ae42b0..969676d55a994 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 @@ -24,6 +24,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; @@ -141,7 +142,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 7f5dc19baf274..8c3f32b92f02e 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; @@ -48,7 +47,7 @@ import java.util.Map; 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 ec7202f4d8622..ee6a5f190e733 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; @@ -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 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 233c70ecf9eb6..cbd7b38b8dea7 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,10 @@ package org.apache.hudi.client.clustering.run.strategy; +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.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -44,7 +48,7 @@ import org.apache.hudi.execution.bulkinsert.JavaBulkInsertInternalPartitionerFactory; import org.apache.hudi.execution.bulkinsert.JavaCustomColumnsSortPartitioner; import org.apache.hudi.io.IOUtils; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.KeyGenUtils; @@ -52,11 +56,6 @@ 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; @@ -73,7 +72,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); @@ -193,7 +192,7 @@ private List> readRecordsForGroupWithLogs(List baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) + Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) ? Option.empty() : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); @@ -219,7 +218,7 @@ private List> readRecordsForGroupBaseFiles(List { try { Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); - HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath())); + HoodieAvroFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath())); Iterator recordIterator = baseFileReader.getRecordIterator(readerSchema); recordIterator.forEachRemaining(record -> records.add(transform(record))); } catch (IOException e) { 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..8258c5e3e01d4 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,7 +24,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.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.CreateHandleFactory; @@ -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 9821aedc875cd..ddf6345926e4a 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 @@ -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.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; @@ -35,7 +34,7 @@ import java.util.Iterator; import java.util.List; -public class JavaLazyInsertIterable extends HoodieLazyInsertIterable { +public class JavaLazyInsertIterable extends HoodieLazyInsertIterable { public JavaLazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, HoodieWriteConfig config, 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 dd64859cad7e5..d612ebac2d36b 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.HoodieList; 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 88921334980ed..d857b7dc7a81f 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.table.timeline.HoodieTimeline; @@ -79,7 +78,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); @@ -311,7 +310,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 32d30f704ecbb..9f89e0f08482f 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.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; 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 f9c7caff6ebbc..44e5120a7095c 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; @@ -38,13 +37,13 @@ import static org.apache.hudi.common.data.HoodieList.getList; -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()) @@ -52,7 +51,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 dc6994d315f02..dde361011877f 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 @@ -26,7 +26,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; @@ -65,7 +64,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/JavaMergeHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java index 46dd30a7cb773..4dcd0a6f7b6d7 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java @@ -20,9 +20,9 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieBaseFile; +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.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; @@ -44,7 +44,7 @@ import java.util.Iterator; import java.util.List; -public class JavaMergeHelper extends BaseMergeHelper>, +public class JavaMergeHelper extends BaseMergeHelper>, List, List> { private JavaMergeHelper() { @@ -80,23 +80,24 @@ public void runMerge(HoodieTable>, List, List } BoundedInMemoryExecutor wrapper = null; - HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); try { - final Iterator readerIterator; + final Iterator readerIterator; if (baseFile.getBootstrapBaseFile().isPresent()) { readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation); } else { - readerIterator = reader.getRecordIterator(readSchema); + readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new); } ThreadLocal encoderCache = new ThreadLocal<>(); ThreadLocal decoderCache = new ThreadLocal<>(); - wrapper = new BoundedInMemoryExecutor<>(table.getConfig().getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(readerIterator), + wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(readerIterator), Option.of(new UpdateHandler(mergeHandle)), record -> { if (!externalSchemaTransformation) { return record; } - return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record); + // TODO Other type of record need to change + return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) ((HoodieRecord)record).getData()); }); wrapper.execute(); } catch (Exception e) { 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 3a1fa4b884fd0..091c2287f19b9 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 @@ -21,10 +21,8 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieList; 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.collection.Pair; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; @@ -34,7 +32,7 @@ import java.util.Objects; import java.util.stream.Collectors; -public class JavaWriteHelper extends BaseWriteHelper>, +public class JavaWriteHelper extends BaseWriteHelper>, List, List, R> { private JavaWriteHelper() { @@ -67,11 +65,11 @@ public List> deduplicateRecords( return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> { @SuppressWarnings("unchecked") - T reducedData = (T) rec1.getData().preCombine(rec2.getData()); + HoodieRecord reducedRec = rec2.preCombine(rec1); // 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 (HoodieRecord) reducedRec.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 30bdcda759ce0..d1f1a8c24ecb5 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.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; @@ -36,7 +35,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 ee507b6045f6c..4608e94a5623b 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-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 37a78a4be54c0..54494f94577a4 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 @@ -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 HoodieReadClient> implements Serializable { +public class HoodieReadClient implements Serializable { private static final long serialVersionUID = 1L; 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/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 7f9ec05e3c5eb..1a95cec5cc642 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 @@ -28,7 +28,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; @@ -70,7 +69,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); 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 88c3057f2f0ad..fd110c7a66db9 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.client.common.HoodieSparkEngineContext; 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.HoodieSparkCopyOnWriteTable; @@ -37,7 +36,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(HoodieSparkCopyOnWriteTable table, HoodieSparkEngineContext 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 6629569d096b3..b222a11e2049e 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; @@ -50,7 +49,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 e09457f0e5135..0caceefaed446 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 @@ -50,7 +50,7 @@ import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; import org.apache.hudi.execution.bulkinsert.RDDSpatialCurveSortPartitioner; import org.apache.hudi.io.IOUtils; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.KeyGenUtils; @@ -84,7 +84,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); @@ -223,7 +223,7 @@ private HoodieData> readRecordsForGroupWithLogs(JavaSparkContext .withPartition(clusteringOp.getPartitionPath()) .build(); - Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) + Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) ? Option.empty() : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); @@ -258,7 +258,7 @@ private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContex clusteringOpsPartition.forEachRemaining(clusteringOp -> { try { Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(writeConfig.getSchema())); - HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath())); + HoodieAvroFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath())); iteratorsForPartition.add(baseFileReader.getRecordIterator(readerSchema)); } catch (IOException e) { throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() 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 1158d0ada42f0..a825f790bddbc 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 @@ -72,7 +72,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); 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 b61017c34ce41..083cbb28d1db2 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 @@ -24,7 +24,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.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieClusteringException; @@ -43,7 +42,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 7db63d4169fc1..2b9a5caf960c4 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 @@ -23,7 +23,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.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.CreateHandleFactory; @@ -42,7 +41,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 3eadba25bbcfb..ee554fe92d672 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.data.HoodieData; 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.action.cluster.strategy.UpdateStrategy; import java.util.List; @@ -32,7 +31,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(HoodieSparkEngineContext engineContext, 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 59040629f718e..9a4357e1a674a 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.data.HoodieData; 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 java.util.HashSet; @@ -33,7 +32,7 @@ /** * Allow ingestion commits during clustering job. */ -public class SparkAllowUpdateStrategy> extends BaseSparkUpdateStrategy { +public class SparkAllowUpdateStrategy extends BaseSparkUpdateStrategy { public SparkAllowUpdateStrategy(HoodieSparkEngineContext engineContext, HashSet 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 d09422ee77242..22a4b31728399 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.data.HoodieData; 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; @@ -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(HoodieSparkEngineContext engineContext, 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 f08d11b571492..8c4814f3e2e17 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/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index df5bd2d3f458c..f1ae56d70bfc5 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.BoundedInMemoryExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -34,7 +33,7 @@ import java.util.Iterator; import java.util.List; -public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { +public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { private boolean useWriterSchema; 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 a184c009a1b45..c9667817953c0 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.table.BulkInsertPartitioner; import org.apache.spark.api.java.JavaRDD; @@ -31,7 +30,7 @@ * * @param HoodieRecordPayload type */ -public class GlobalSortPartitioner +public class GlobalSortPartitioner implements BulkInsertPartitioner>> { @Override 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 19c90ecb1a012..75ededbd198b8 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; @@ -30,7 +29,7 @@ * * @param HoodieRecordPayload type */ -public class NonSortPartitioner +public class NonSortPartitioner implements BulkInsertPartitioner>> { @Override 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 b1cbe47a6b72e..fe91ca70badf3 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 @@ -20,8 +20,8 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.SerializableSchema; +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.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.BulkInsertPartitioner; @@ -36,7 +36,7 @@ * * @param HoodieRecordPayload type */ -public class RDDCustomColumnsSortPartitioner +public class RDDCustomColumnsSortPartitioner implements BulkInsertPartitioner>> { private final String[] sortColumnNames; @@ -63,7 +63,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 = HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)record, sortColumns, schema, 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 9526ad5856469..6145ea2219eaa 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.table.BulkInsertPartitioner; import org.apache.spark.api.java.JavaRDD; @@ -37,7 +36,7 @@ * * @param HoodieRecordPayload type */ -public class RDDPartitionSortPartitioner +public class RDDPartitionSortPartitioner implements BulkInsertPartitioner>> { @Override 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 50a0a534f881b..a5921e5a460cf 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 @@ -25,7 +25,6 @@ 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.RewriteAvroPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; @@ -40,13 +39,14 @@ import java.util.Arrays; import java.util.List; +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 implements BulkInsertPartitioner>> { private final transient HoodieSparkEngineContext sparkEngineContext; @@ -70,7 +70,7 @@ public RDDSpatialCurveSortPartitioner(HoodieSparkEngineContext sparkEngineContex @Override public JavaRDD> repartitionRecords(JavaRDD> records, int outputSparkPartitions) { JavaRDD genericRecordsRDD = - records.map(f -> (GenericRecord) f.getData().getInsertValue(schema.get()).get()); + records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get()); Dataset sourceDataset = AvroConversionUtils.createDataFrame( 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/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 916b31d2931e5..8ad336b48401e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodiePartitionMetadata; 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.model.IOType; import org.apache.hudi.common.table.HoodieTableConfig; @@ -113,9 +114,9 @@ public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, S */ public void write(InternalRow record) throws IOException { try { - final String partitionPath = String.valueOf(record.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_POS)); + final String partitionPath = String.valueOf(record.getUTF8String(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal())); final String seqId = HoodieRecord.generateSequenceId(instantTime, taskPartitionId, SEQGEN.getAndIncrement()); - final String recordKey = String.valueOf(record.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_POS)); + final String recordKey = String.valueOf(record.getUTF8String(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal())); HoodieInternalRow internalRow = new HoodieInternalRow(instantTime, seqId, recordKey, partitionPath, path.getName(), record); try { 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 a88ca65c35a94..7fee680d92c54 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); @@ -262,7 +261,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 efc667af297be..8a68c53c66790 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.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -73,7 +72,7 @@ * action *

*/ -public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable { +public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable { HoodieSparkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); 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 20e3bd4c14ac3..3575cbf7b3cb5 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,11 +52,11 @@ 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) { return create(config, context, false); } - public static HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context, + public static HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context, boolean refreshTimeline) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) @@ -68,13 +67,13 @@ public static HoodieSparkTable create(HoodieW return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, refreshTimeline); } - public static HoodieSparkTable create(HoodieWriteConfig config, + public static HoodieSparkTable create(HoodieWriteConfig config, HoodieSparkEngineContext context, HoodieTableMetaClient metaClient) { return create(config, context, metaClient, false); } - public static HoodieSparkTable create(HoodieWriteConfig config, + public static HoodieSparkTable create(HoodieWriteConfig config, HoodieSparkEngineContext context, HoodieTableMetaClient metaClient, boolean refreshTimeline) { 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 4e488047d845e..e49d6c2a28620 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 @@ -37,7 +37,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; @@ -82,7 +81,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); 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 d712ca430b6f7..3b868d8472dd2 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 f8e4b31ff687e..0c0a336a1f7bc 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; @@ -82,7 +81,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 1652c35eb63e6..1f60948c2f840 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 149aef03e238a..c4239d407a736 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 @@ -28,7 +28,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; @@ -45,7 +44,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 c2f5a43066d36..5c18aa5c3799e 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; @@ -57,7 +56,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 61cb1ffd27bd1..edbd3b7eb65ee 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.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; @@ -36,7 +35,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 61e6f25af9429..93c4917c1a117 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/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java index a5926196ea396..45a3029352d7d 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; @@ -133,7 +134,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/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 34f470eb1b64a..28ebd0a16997e 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 @@ -49,6 +49,7 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; @@ -89,7 +90,7 @@ 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; @@ -736,9 +737,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)); @@ -983,9 +984,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(HoodieAvroIndexedRecord::new)) { 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)); @@ -1042,7 +1043,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()); @@ -1066,10 +1067,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)); @@ -2498,9 +2499,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(HoodieAvroIndexedRecord::new)) { 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 9a8fc55a20028..fb789a5ecdf08 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 @@ -29,9 +29,9 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; +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.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; @@ -43,7 +43,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; @@ -290,9 +290,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(HoodieAvroIndexedRecord::new)) { 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)); @@ -333,7 +333,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()); @@ -356,10 +356,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/execution/TestBoundedInMemoryQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java index 4707a68072e9a..9fb1862c5f8f4 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; @@ -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()); @@ -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,9 +219,9 @@ 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 = + HoodieLazyInsertIterable.HoodieInsertValueGenResult genResult = getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply((HoodieAvroRecord) hoodieRecords.get(0)); - final long objSize = sizeEstimator.sizeEstimate(payload); + final long objSize = sizeEstimator.sizeEstimate(genResult); final long memoryLimitInBytes = recordLimit * objSize; final BoundedInMemoryQueue queue = new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); @@ -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,9 +268,9 @@ public void testException() throws Exception { final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); final SizeEstimator>> sizeEstimator = new DefaultSizeEstimator<>(); // queue memory limit - HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = + HoodieLazyInsertIterable.HoodieInsertValueGenResult genResult = getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply((HoodieAvroRecord) hoodieRecords.get(0)); - final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue)); + 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 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 81% 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..d9377fc4e85c2 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,13 @@ package org.apache.hudi.io.storage; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.SparkTaskContextSupplier; 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 +35,7 @@ /** * Tests for {@link HoodieFileWriterFactory}. */ -public class TestHoodieFileWriterFactory extends HoodieClientTestBase { +public class TestHoodieAvroFileWriterFactory extends HoodieClientTestBase { @Test public void testGetFileWriter() throws IOException { @@ -47,26 +45,26 @@ 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, + HoodieFileWriter parquetWriter = HoodieFileWriterFactory.getFileWriter(instantTime, parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); 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, + HoodieFileWriter hfileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, hfilePath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); - assertTrue(hfileWriter instanceof HoodieHFileWriter); + 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, + HoodieFileWriter orcFileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, orcPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); - assertTrue(orcFileWriter instanceof HoodieOrcWriter); + 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, + HoodieFileWriter logWriter = HoodieFileWriterFactory.getFileWriter(instantTime, logPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); }, "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/TestHoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java index 5a19f0afe9c65..87fa35f41f52c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.HoodieInternalWriteStatus; import org.apache.hudi.common.config.HoodieMetadataConfig; 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.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieWriteConfig; @@ -235,9 +236,9 @@ private void assertOutput(HoodieInternalWriteStatus writeStatus, int size, Strin private void assertRows(Dataset expectedRows, Dataset actualRows, String instantTime, List filenames) { // verify 3 meta fields that are filled in within create handle actualRows.collectAsList().forEach(entry -> { - assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime); - assertTrue(filenames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)).toString())); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))); + assertEquals(entry.get(HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.ordinal()).toString(), instantTime); + assertTrue(filenames.contains(entry.get(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal()).toString())); + 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-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 75d2d14221d32..7ceb156960d21 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-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index e2b586964ef4e..cca077c99841e 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,9 +19,10 @@ package org.apache.hudi.avro; import org.apache.hudi.common.config.SerializableSchema; +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.model.HoodieRecord.HoodieMetadataField; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; @@ -73,6 +74,7 @@ import java.util.Map; import java.util.Deque; import java.util.LinkedList; +import java.util.Properties; import java.util.TimeZone; import java.util.stream.Collectors; @@ -104,6 +106,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((IndexedRecord) record.toIndexedRecord(schema, new Properties()).get())); + } + /** * Convert a given avro record to bytes. */ @@ -400,7 +409,7 @@ public static GenericRecord rewriteRecordWithMetadata(GenericRecord genericRecor copyOldValueOrSetDefault(genericRecord, newRecord, f); } // do not preserve FILENAME_METADATA_FIELD - newRecord.put(HoodieRecord.FILENAME_META_FIELD_POS, fileName); + newRecord.put(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal(), fileName); if (!GenericData.get().validate(newSchema, newRecord)) { throw new SchemaCompatibilityException( "Unable to validate the rewritten record " + genericRecord + " against schema " + newSchema); @@ -412,7 +421,7 @@ public static GenericRecord rewriteRecordWithMetadata(GenericRecord genericRecor public static GenericRecord rewriteEvolutionRecordWithMetadata(GenericRecord genericRecord, Schema newSchema, String fileName) { GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(genericRecord, newSchema, new HashMap<>()); // do not preserve FILENAME_METADATA_FIELD - newRecord.put(HoodieRecord.FILENAME_META_FIELD_POS, fileName); + newRecord.put(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal(), fileName); return newRecord; } @@ -697,11 +706,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) record.toIndexedRecord(schema, new Properties()).get(); if (columns.length == 1) { return HoodieAvroUtils.getNestedFieldVal(genericRecord, columns[0], true, consistentLogicalTimestampEnabled); } else { @@ -726,7 +735,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); 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..ac2df00151649 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java @@ -0,0 +1,192 @@ +/* + * 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.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +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.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + * 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) { + super(key, data, operation); + } + + public HoodieAvroIndexedRecord(HoodieRecord record) { + super(record); + } + + public HoodieAvroIndexedRecord() { + } + + @Override + public Option toIndexedRecord(Schema schema, Properties prop) { + return Option.of(data); + } + + public Option toIndexedRecord() { + return Option.of(data); + } + + @Override + public Comparable getOrderingValue() { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord newInstance() { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord newInstance(HoodieKey key, HoodieOperation op) { + throw new UnsupportedOperationException(); + } + + @Override + public HoodieRecord newInstance(HoodieKey key) { + throw new UnsupportedOperationException(); + } + + @Override + public String getRecordKey(Option keyGeneratorOpt) { + return keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey((GenericRecord) data) : ((GenericRecord) data).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + } + + @Override + public String getRecordKey(String keyFieldName) { + return Option.ofNullable(data.getSchema().getField(keyFieldName)) + .map(keyField -> data.get(keyField.pos())) + .map(Object::toString).orElse(null); + } + + @Override + public HoodieRecord preCombine(HoodieRecord previousRecord) { + throw new UnsupportedOperationException(); + } + + @Override + public Option combineAndGetUpdateValue(HoodieRecord previousRecord, Schema schema, Properties props) throws IOException { + return Option.empty(); + } + + @Override + public HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException { + ValidationUtils.checkState(other instanceof HoodieAvroIndexedRecord); + GenericRecord record = HoodieAvroUtils.stitchRecords((GenericRecord) data, (GenericRecord) other.getData(), writerSchema); + return new HoodieAvroIndexedRecord(record); + } + + @Override + public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException { + GenericRecord avroPayloadInNewSchema = + HoodieAvroUtils.rewriteRecord((GenericRecord) data, targetSchema); + return new HoodieAvroIndexedRecord(avroPayloadInNewSchema); + } + + @Override + public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException { + GenericRecord rewriteRecord = schemaOnReadEnabled ? HoodieAvroUtils.rewriteRecordWithNewSchema(data, writeSchemaWithMetaFields, new HashMap<>()) + : HoodieAvroUtils.rewriteRecord((GenericRecord) data, writeSchemaWithMetaFields); + return new HoodieAvroIndexedRecord(rewriteRecord); + } + + @Override + public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException { + GenericRecord rewriteRecord = schemaOnReadEnabled ? HoodieAvroUtils.rewriteEvolutionRecordWithMetadata((GenericRecord) data, writeSchemaWithMetaFields, fileName) + : HoodieAvroUtils.rewriteRecordWithMetadata((GenericRecord) data, writeSchemaWithMetaFields, fileName); + return new HoodieAvroIndexedRecord(rewriteRecord); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map renameCols) throws IOException { + GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(data, newSchema, renameCols); + return new HoodieAvroIndexedRecord(rewriteRecord); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map renameCols, Mapper mapper) throws IOException { + GenericRecord oldRecord = (GenericRecord) getData(); + GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols); + return mapper.apply(rewriteRecord); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException { + GenericRecord oldRecord = (GenericRecord) data; + GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecord(oldRecord, newSchema); + return new HoodieAvroIndexedRecord(rewriteRecord); + } + + @Override + public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map metadataValues) throws IOException { + Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> { + String value = metadataValues.get(metadataField); + if (value != null) { + ((GenericRecord) data).put(metadataField.getFieldName(), value); + } + }); + + return new HoodieAvroIndexedRecord(data); + } + + @Override + public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException { + data.put(pos, newValue); + return this; + } + + @Override + public boolean shouldIgnore(Schema schema, Properties prop) throws IOException { + return getData().equals(SENTINEL); + } + + @Override + public Option> getMetadata() { + return Option.empty(); + } + + @Override + public boolean isPresent(Schema schema, Properties prop) { + return true; + } +} 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 9a9bbb2b7427f..9a9011da37a82 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,6 +19,28 @@ package org.apache.hudi.common.model; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.metadata.HoodieMetadataPayload; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.hudi.TypeUtils.unsafeCast; + public class HoodieAvroRecord extends HoodieRecord { public HoodieAvroRecord(HoodieKey key, T data) { super(key, data); @@ -40,6 +62,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) { @@ -47,4 +79,175 @@ public T getData() { } return data; } + + @Override + public String getRecordKey(Option keyGeneratorOpt) { + return getRecordKey(); + } + + @Override + public String getRecordKey(String keyFieldName) { + return getRecordKey(); + } + + @Override + public Comparable getOrderingValue() { + return data.getOrderingValue(); + } + + @Override + public Option toIndexedRecord(Schema schema, Properties prop) throws IOException { + return getData().getInsertValue(schema, prop); + } + + ////////////////////////////////////////////////////////////////////////////// + + // + // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here + // for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload` + // is complete + // + // TODO cleanup + + // NOTE: This method is assuming semantic that `preCombine` operation is bound to pick one or the other + // object, and may not create a new one + @Override + public HoodieRecord preCombine(HoodieRecord previousRecord) { + T picked = unsafeCast(getData().preCombine(previousRecord.getData())); + if (picked instanceof HoodieMetadataPayload) { + // NOTE: HoodieMetadataPayload return a new payload + return new HoodieAvroRecord<>(getKey(), picked, getOperation()); + } + return picked.equals(getData()) ? this : previousRecord; + } + + // NOTE: This method is assuming semantic that only records bearing the same (partition, key) could + // be combined + @Override + public Option combineAndGetUpdateValue(HoodieRecord previousRecord, Schema schema, Properties props) throws IOException { + Option previousRecordAvroPayload = previousRecord.toIndexedRecord(schema, props); + if (!previousRecordAvroPayload.isPresent()) { + return Option.empty(); + } + + return getData().combineAndGetUpdateValue(previousRecordAvroPayload.get(), schema, props) + .map(combinedAvroPayload -> new HoodieAvroIndexedRecord((IndexedRecord) combinedAvroPayload)); + } + + @Override + public HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException { + ValidationUtils.checkState(other instanceof HoodieAvroRecord); + GenericRecord mergedPayload = HoodieAvroUtils.stitchRecords( + (GenericRecord) toIndexedRecord(readerSchema, new Properties()).get(), + (GenericRecord) other.toIndexedRecord(readerSchema, new Properties()).get(), + writerSchema); + return new HoodieAvroRecord(getKey(), instantiateRecordPayloadWrapper(mergedPayload, getPrecombineValue(getData())), getOperation()); + } + + @Override + public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException { + Option avroRecordPayloadOpt = getData().getInsertValue(recordSchema, props); + GenericRecord avroPayloadInNewSchema = + HoodieAvroUtils.rewriteRecord((GenericRecord) avroRecordPayloadOpt.get(), targetSchema); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(avroPayloadInNewSchema), getOperation()); + } + + @Override + public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException { + GenericRecord record = (GenericRecord) getData().getInsertValue(recordSchema, prop).get(); + GenericRecord rewriteRecord = schemaOnReadEnabled ? HoodieAvroUtils.rewriteRecordWithNewSchema(record, writeSchemaWithMetaFields, new HashMap<>()) + : HoodieAvroUtils.rewriteRecord(record, writeSchemaWithMetaFields); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation()); + } + + @Override + public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException { + GenericRecord record = (GenericRecord) getData().getInsertValue(recordSchema, prop).get(); + GenericRecord rewriteRecord = schemaOnReadEnabled ? HoodieAvroUtils.rewriteEvolutionRecordWithMetadata(record, writeSchemaWithMetaFields, fileName) + : HoodieAvroUtils.rewriteRecordWithMetadata(record, writeSchemaWithMetaFields, fileName); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation()); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map renameCols) throws IOException { + GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, prop).get(); + GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation()); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException { + GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, prop).get(); + GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecord(oldRecord, newSchema); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation()); + } + + @Override + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map renameCols, Mapper mapper) throws IOException { + GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, prop).get(); + GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols); + return mapper.apply(rewriteRecord); + } + + @Override + public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException { + IndexedRecord record = (IndexedRecord) data.getInsertValue(recordSchema, prop).get(); + record.put(pos, newValue); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload((GenericRecord) record), getOperation()); + } + + @Override + public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map metadataValues) throws IOException { + // NOTE: RewriteAvroPayload is expected here + GenericRecord avroRecordPayload = (GenericRecord) getData().getInsertValue(recordSchema, prop).get(); + + Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> { + String value = metadataValues.get(metadataField); + if (value != null) { + avroRecordPayload.put(metadataField.getFieldName(), value); + } + }); + + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(avroRecordPayload), getOperation()); + } + + public Option> getMetadata() { + return getData().getMetadata(); + } + + @Override + public boolean isPresent(Schema schema, Properties prop) throws IOException { + return getData().getInsertValue(schema, prop).isPresent(); + } + + @Override + public boolean shouldIgnore(Schema schema, Properties prop) throws IOException { + Option insertRecord = getData().getInsertValue(schema, prop); + // just skip the ignored record + if (insertRecord.isPresent() && insertRecord.get().equals(SENTINEL)) { + return true; + } else { + return false; + } + } + + @Nonnull + private T instantiateRecordPayloadWrapper(Object combinedAvroPayload, Comparable newPreCombineVal) { + return unsafeCast( + ReflectionUtils.loadPayload( + getData().getClass().getCanonicalName(), + new Object[]{combinedAvroPayload, newPreCombineVal}, + GenericRecord.class, + Comparable.class)); + } + + private static Comparable getPrecombineValue(T data) { + if (data instanceof BaseAvroPayload) { + return ((BaseAvroPayload) data).orderingVal; + } + + return -1; + } + + ////////////////////////////////////////////////////////////////////////////// } 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 e504b7b87dd9b..1b6e79f09710e 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,21 @@ package org.apache.hudi.common.model; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.config.TypedProperties; 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.List; import java.util.Map; import java.util.Objects; +import java.util.Properties; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -34,13 +41,50 @@ */ public abstract class HoodieRecord implements Serializable { - 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"; + 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; + } + } + + /** + * 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, @@ -57,10 +101,6 @@ public abstract class HoodieRecord implements Serializable { IntStream.range(0, HOODIE_META_COLUMNS.size()).mapToObj(idx -> Pair.of(HOODIE_META_COLUMNS.get(idx), idx)) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - public static int RECORD_KEY_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(RECORD_KEY_METADATA_FIELD); - public static int PARTITION_PATH_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(PARTITION_PATH_METADATA_FIELD); - public static int FILENAME_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(FILENAME_METADATA_FIELD); - /** * Identifies the record across the table. */ @@ -117,6 +157,10 @@ 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; } @@ -132,6 +176,8 @@ public T getData() { return data; } + public abstract Comparable getOrderingValue(); + /** * Release the actual payload, to ease memory pressure. To be called after the record has been written to storage. * Once deflated, cannot be inflated. @@ -200,10 +246,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(); @@ -214,6 +256,10 @@ public String getRecordKey() { return key.getRecordKey(); } + public abstract String getRecordKey(Option keyGeneratorOpt); + + public abstract String getRecordKey(String keyFieldName); + public void seal() { this.sealed = true; } @@ -227,4 +273,108 @@ public void checkState() { throw new UnsupportedOperationException("Not allowed to modify after sealed"); } } + + ////////////////////////////////////////////////////////////////////////////// + + // + // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here + // for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload` + // is complete + // + // TODO cleanup + + // NOTE: This method is assuming semantic that `preCombine` operation is bound to pick one or the other + // object, and may not create a new one + public abstract HoodieRecord preCombine(HoodieRecord previousRecord); + + // NOTE: This method is assuming semantic that only records bearing the same (partition, key) could + // be combined + public abstract Option combineAndGetUpdateValue(HoodieRecord previousRecord, Schema schema, Properties props) throws IOException; + + public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException; + + public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException; + + /** + * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields. + */ + public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException; + + public abstract HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException; + + public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map renameCols) throws IOException; + + public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map renameCols, Mapper mapper) throws IOException; + + public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException; + + public abstract HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException; + + public abstract HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map metadataValues) throws IOException; + + public abstract Option> getMetadata(); + + public abstract boolean isPresent(Schema schema, Properties prop) throws IOException; + + public abstract boolean shouldIgnore(Schema schema, Properties prop) throws IOException; + + public abstract Option toIndexedRecord(Schema schema, Properties prop) throws IOException; + + ////////////////////////////////////////////////////////////////////////////// + + public static String generateSequenceId(String instantTime, int partitionId, long recordIndex) { + return instantTime + "_" + partitionId + "_" + recordIndex; + } + + /** + * NOTE: This is temporary transition construct to be able to construct + * HoodieRecord instances w/o excessive wiring into a lot of components + * a lot of details that are irrelevant for these + * TODO remove + */ + @FunctionalInterface + public interface Mapper { + HoodieRecord apply(IndexedRecord avroPayload); + } + + /** + * 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; + } + } } 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 6752607d2f48c..a889bf80754ae 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/OverwriteWithLatestAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java index d8469ed5a148e..5268d76281394 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 @@ -85,7 +85,7 @@ public Option getInsertValue(Schema schema) throws IOException { * @returns {@code true} if record represents a delete record. {@code false} otherwise. */ protected boolean isDeleteRecord(GenericRecord genericRecord) { - final String isDeleteKey = HoodieRecord.HOODIE_IS_DELETED; + 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. 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 b76f71161d320..4affb0a15c4b5 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 @@ -22,7 +22,6 @@ import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.avro.SchemaCompatibility; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -48,8 +47,8 @@ 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.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroSchemaConverter; @@ -502,7 +501,7 @@ public MessageType readSchemaFromHFileBaseFile(Path hFilePath) throws IOExceptio 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()); } @@ -515,7 +514,7 @@ public MessageType readSchemaFromORCBaseFile(Path orcFilePath) throws IOExceptio 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 9687136444eeb..84abfec3de84a 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 @@ -19,7 +19,6 @@ package org.apache.hudi.common.table.log; import org.apache.hudi.common.model.DeleteRecord; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; @@ -27,6 +26,7 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock.RecordIterator; import org.apache.hudi.common.table.log.block.HoodieCommandBlock; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; @@ -61,6 +61,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -376,13 +377,22 @@ private boolean isNewInstantBlock(HoodieLogBlock logBlock) { * handle it. */ private void processDataBlock(HoodieDataBlock dataBlock, Option keySpecOpt) throws Exception { - try (ClosableIterator recordIterator = getRecordsIterator(dataBlock, keySpecOpt)) { + HoodieRecord.Mapper mapper = (rec) -> createHoodieRecord(rec, this.hoodieTableMetaClient.getTableConfig(), + this.payloadClassFQN, this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName); + + try (ClosableIterator recordIterator = getRecordsIterator(dataBlock, keySpecOpt, mapper)) { Option schemaOption = getMergedSchema(dataBlock); + Schema finalReadSchema; + if (recordIterator instanceof RecordIterator) { + finalReadSchema = ((RecordIterator) recordIterator).getFinalReadSchema(); + } else { + finalReadSchema = dataBlock.getSchema(); + } while (recordIterator.hasNext()) { - IndexedRecord currentRecord = recordIterator.next(); - IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get(), new HashMap<>()) : currentRecord; - processNextRecord(createHoodieRecord(record, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN, - this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName)); + HoodieRecord currentRecord = recordIterator.next(); + HoodieRecord record = schemaOption.isPresent() + ? currentRecord.rewriteRecordWithNewSchema(finalReadSchema, new Properties(), schemaOption.get(), new HashMap<>(), mapper) : currentRecord; + processNextRecord(record); totalLogRecords.incrementAndGet(); } } @@ -483,13 +493,13 @@ private void processQueuedBlocksForInstant(Deque logBlocks, int progress = numLogFilesSeen - 1 / logFilePaths.size(); } - private ClosableIterator getRecordsIterator(HoodieDataBlock dataBlock, Option keySpecOpt) throws IOException { + private ClosableIterator getRecordsIterator(HoodieDataBlock dataBlock, Option keySpecOpt, HoodieRecord.Mapper mapper) throws IOException { if (keySpecOpt.isPresent()) { KeySpec keySpec = keySpecOpt.get(); - return dataBlock.getRecordIterator(keySpec.keys, keySpec.fullKey); + return dataBlock.getRecordIterator(keySpec.keys, keySpec.fullKey, mapper); } - return dataBlock.getRecordIterator(); + return dataBlock.getRecordIterator(mapper); } /** 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 af9bcd27f933c..b2a936b476670 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 @@ -267,7 +267,7 @@ private HoodieLogBlock createCorruptBlock() 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 e3d8554d00fd8..6d313b64f9f5f 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 @@ -65,13 +65,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; @@ -120,12 +120,12 @@ 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; } @@ -142,7 +142,7 @@ public static HoodieMergedLogRecordScanner.Builder newBuilder() { } @Override - protected void processNextRecord(HoodieRecord hoodieRecord) throws IOException { + protected void processNextRecord(HoodieRecord hoodieRecord) throws IOException { String key = hoodieRecord.getRecordKey(); if (records.containsKey(key)) { // Merge and store the merged record. The HoodieRecordPayload implementation is free to decide what should be @@ -150,7 +150,7 @@ protected void processNextRecord(HoodieRecord hoo HoodieRecord oldRecord = records.get(key); HoodieRecordPayload oldValue = oldRecord.getData(); - HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(oldValue); + HoodieRecordPayload combinedValue = (HoodieRecordPayload)hoodieRecord.preCombine(oldRecord).getData(); // If combinedValue is oldValue, no need rePut oldRecord if (combinedValue != oldValue) { HoodieOperation operation = hoodieRecord.getOperation(); @@ -171,7 +171,7 @@ protected void processNextDeletedRecord(DeleteRecord deleteRecord) { // 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(); 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 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 491c6700c9067..eeec63509fc81 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 @@ -30,6 +30,7 @@ import org.apache.avro.io.EncoderFactory; import org.apache.hadoop.fs.FSDataInputStream; 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.util.ClosableIterator; import org.apache.hudi.common.util.Option; @@ -37,6 +38,7 @@ import org.apache.hudi.internal.schema.InternalSchema; import javax.annotation.Nonnull; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -51,6 +53,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; @@ -85,9 +89,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); } @@ -97,7 +102,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(); @@ -110,13 +115,14 @@ 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); + IndexedRecord data = (IndexedRecord) s.toIndexedRecord(schema, new Properties()).get(); + writer.write(data, encoder); encoder.flush(); // Get the size of the bytes @@ -136,22 +142,25 @@ 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, HoodieRecord.Mapper mapper) throws IOException { checkState(this.readerSchema != null, "Reader's schema has to be non-null"); - return RecordIterator.getInstance(this, content, internalSchema); + return RecordIterator.getInstance(this, content, internalSchema, mapper); } - private static class RecordIterator implements ClosableIterator { + public static class RecordIterator implements ClosableIterator { private byte[] content; private final SizeAwareDataInputStream dis; private final GenericDatumReader reader; private final ThreadLocal decoderCache = new ThreadLocal<>(); + private final HoodieRecord.Mapper mapper; + private final Schema finalReadSchema; private int totalRecords = 0; private int readRecords = 0; - private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, InternalSchema internalSchema) throws IOException { + private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, InternalSchema internalSchema, HoodieRecord.Mapper mapper) throws IOException { this.content = content; + this.mapper = mapper; this.dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(this.content))); @@ -168,6 +177,7 @@ private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, finalReadSchema = writerSchema; } + this.finalReadSchema = finalReadSchema; this.reader = new GenericDatumReader<>(writerSchema, finalReadSchema); if (logBlockVersion.hasRecordCount()) { @@ -175,10 +185,14 @@ private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, } } - public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content, InternalSchema internalSchema) throws IOException { + public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content, InternalSchema internalSchema, HoodieRecord.Mapper mapper) throws IOException { // Get schema from the header Schema writerSchema = new Schema.Parser().parse(dataBlock.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); - return new RecordIterator(dataBlock.readerSchema, writerSchema, content, internalSchema); + return new RecordIterator(dataBlock.readerSchema, writerSchema, content, internalSchema, mapper); + } + + public Schema getFinalReadSchema() { + return finalReadSchema; } @Override @@ -198,7 +212,7 @@ public boolean hasNext() { } @Override - public IndexedRecord next() { + public HoodieRecord next() { try { int recordLength = this.dis.readInt(); BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(this.content, this.dis.getNumberOfBytesRead(), @@ -207,7 +221,7 @@ public IndexedRecord next() { IndexedRecord record = this.reader.read(null, decoder); this.dis.skipBytes(recordLength); this.readRecords++; - return record; + return mapper.apply(record); } catch (IOException e) { throw new HoodieIOException("Unable to convert bytes to record.", e); } @@ -226,7 +240,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); } @@ -271,7 +285,7 @@ public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema, dis.skipBytes(recordLength); } dis.close(); - return new HoodieAvroDataBlock(records, readerSchema); + return new HoodieAvroDataBlock(records.stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()), readerSchema); } private static byte[] compress(String text) { @@ -313,8 +327,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(HoodieAvroIndexedRecord::new)) { recordItr.forEachRemaining(records::add); } @@ -322,9 +336,9 @@ public byte[] getBytes(Schema schema) throws IOException { output.writeInt(records.size()); // 4. Write the records - Iterator itr = records.iterator(); + Iterator itr = records.iterator(); while (itr.hasNext()) { - IndexedRecord s = itr.next(); + IndexedRecord s = (IndexedRecord)itr.next().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/HoodieDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java index c83b3bc82d56c..d88fff750f47a 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 @@ -23,8 +23,8 @@ 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 org.apache.hudi.internal.schema.InternalSchema; import java.io.IOException; @@ -50,7 +50,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 @@ -66,7 +66,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock { /** * 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) { @@ -138,13 +138,13 @@ protected static Schema getWriterSchema(Map logBlock /** * Returns all the records iterator contained w/in this block. */ - public final ClosableIterator getRecordIterator() { + public final ClosableIterator getRecordIterator(HoodieRecord.Mapper mapper) { if (records.isPresent()) { return list2Iterator(records.get()); } try { // in case records are absent, read content lazily and then convert to IndexedRecords - return readRecordsFromBlockPayload(); + return readRecordsFromBlockPayload(mapper); } catch (IOException io) { throw new HoodieIOException("Unable to convert content bytes to records", io); } @@ -162,15 +162,15 @@ 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, HoodieRecord.Mapper mapper) throws IOException { boolean fullScan = keys.isEmpty(); if (enablePointLookups && !fullScan) { - return lookupRecords(keys, fullKey); + return lookupRecords(keys, fullKey, mapper); } // Otherwise, we fetch all the records and filter out all the records, but the // ones requested - ClosableIterator allRecords = getRecordIterator(); + ClosableIterator allRecords = getRecordIterator(mapper); if (fullScan) { return allRecords; } @@ -179,29 +179,29 @@ public final ClosableIterator getRecordIterator(List keys return FilteringIterator.getInstance(allRecords, keySet, fullKey, this::getRecordKey); } - protected ClosableIterator readRecordsFromBlockPayload() throws IOException { + protected ClosableIterator readRecordsFromBlockPayload(HoodieRecord.Mapper mapper) throws IOException { if (readBlockLazily && !getContent().isPresent()) { // read log block contents from disk inflate(); } try { - return deserializeRecords(getContent().get()); + return deserializeRecords(getContent().get(), mapper); } 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, HoodieRecord.Mapper mapper) 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, HoodieRecord.Mapper mapper) throws IOException; public abstract HoodieLogBlockType getBlockType(); @@ -209,10 +209,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(keyFieldName)); } /** @@ -245,32 +243,30 @@ 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> 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> 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) { - return new FilteringIterator<>(nested, keys, fullKey, keyExtract); + Function> keyExtract) { + return new FilteringIterator(nested, keys, fullKey, keyExtract); } @Override @@ -296,7 +292,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 72cb3a0ef3b47..7cbbbe6fe0e5b 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 @@ -29,7 +29,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieHBaseKVComparator; -import org.apache.hudi.io.storage.HoodieHFileReader; +import org.apache.hudi.io.storage.HoodieAvroHFileReader; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -54,6 +55,7 @@ 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.ValidationUtils.checkState; @@ -81,16 +83,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; } @@ -101,7 +103,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()) @@ -120,11 +122,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++); @@ -132,7 +136,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); @@ -151,7 +155,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(); @@ -161,7 +165,7 @@ protected byte[] serializeRecords(List records) throws IOExceptio } @Override - protected ClosableIterator deserializeRecords(byte[] content) throws IOException { + protected ClosableIterator deserializeRecords(byte[] content, HoodieRecord.Mapper mapper) throws IOException { checkState(readerSchema != null, "Reader's schema has to be non-null"); // Get schema from the header @@ -169,9 +173,9 @@ protected ClosableIterator deserializeRecords(byte[] content) thr FileSystem fs = FSUtils.getFs(pathForReader.toString(), new Configuration()); // Read the content - HoodieHFileReader reader = new HoodieHFileReader<>(fs, pathForReader, content, Option.of(writerSchema)); + HoodieAvroHFileReader reader = new HoodieAvroHFileReader(fs, pathForReader, content, Option.of(writerSchema)); Iterator recordIterator = reader.getRecordIterator(readerSchema); - return new ClosableIterator() { + return new ClosableIterator() { @Override public void close() { reader.close(); @@ -183,15 +187,15 @@ public boolean hasNext() { } @Override - public IndexedRecord next() { - return recordIterator.next(); + public HoodieRecord next() { + return mapper.apply(recordIterator.next()); } }; } // TODO abstract this w/in HoodieDataBlock @Override - protected ClosableIterator lookupRecords(List keys, boolean fullKey) throws IOException { + protected ClosableIterator lookupRecords(List keys, boolean fullKey, HoodieRecord.Mapper mapper) throws IOException { HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get(); // NOTE: It's important to extend Hadoop configuration here to make sure configuration @@ -211,22 +215,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 = fullKey ? reader.getRecordsByKeysIterator(sortedKeys, readerSchema) : reader.getRecordsByKeyPrefixIterator(sortedKeys, readerSchema); - return new ClosableIterator() { + return new ClosableIterator() { @Override public boolean hasNext() { return recordIterator.hasNext(); } @Override - public IndexedRecord next() { - return recordIterator.next(); + public HoodieRecord next() { + return mapper.apply(recordIterator.next()); } @Override @@ -237,12 +241,12 @@ public void close() { }; } - 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.overrideMetadataFieldValue(schema, new Properties(), keyField.get().pos(), StringUtils.EMPTY_STRING); } - 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 5e7bef90a08ba..67db1d6441059 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 @@ -22,8 +22,10 @@ import org.apache.hudi.common.fs.inline.InLineFSUtils; import org.apache.hudi.common.fs.inline.InLineFileSystem; import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetReaderIterator; +import org.apache.hudi.io.storage.HoodieAvroFileReader.HoodieRecordTransformIterator; import org.apache.hudi.io.storage.HoodieAvroParquetConfig; import org.apache.hudi.io.storage.HoodieParquetStreamWriter; @@ -71,7 +73,7 @@ public HoodieParquetDataBlock(FSDataInputStream inputStream, } public HoodieParquetDataBlock( - @Nonnull List records, + @Nonnull List records, @Nonnull Map header, @Nonnull String keyField, @Nonnull CompressionCodecName compressionCodecName @@ -87,7 +89,7 @@ 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]; } @@ -110,10 +112,10 @@ protected byte[] serializeRecords(List records) throws IOExceptio ByteArrayOutputStream baos = new ByteArrayOutputStream(); try (FSDataOutputStream outputStream = new FSDataOutputStream(baos)) { - try (HoodieParquetStreamWriter parquetWriter = new HoodieParquetStreamWriter<>(outputStream, avroParquetConfig)) { - for (IndexedRecord record : records) { + try (HoodieParquetStreamWriter parquetWriter = new HoodieParquetStreamWriter(outputStream, avroParquetConfig)) { + for (HoodieRecord record : records) { String recordKey = getRecordKey(record).orElse(null); - parquetWriter.writeAvro(recordKey, record); + parquetWriter.write(recordKey, record, writerSchema); } outputStream.flush(); } @@ -139,7 +141,7 @@ public static ClosableIterator getProjectedParquetRecordsIterator * requested by the caller (providing projected Reader's schema) */ @Override - protected ClosableIterator readRecordsFromBlockPayload() throws IOException { + protected ClosableIterator readRecordsFromBlockPayload(HoodieRecord.Mapper mapper) throws IOException { HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get(); // NOTE: It's important to extend Hadoop configuration here to make sure configuration @@ -153,14 +155,15 @@ protected ClosableIterator readRecordsFromBlockPayload() throws I blockContentLoc.getContentPositionInLogFile(), blockContentLoc.getBlockSize()); - return getProjectedParquetRecordsIterator( + ClosableIterator iterator = getProjectedParquetRecordsIterator( inlineConf, readerSchema, HadoopInputFile.fromPath(inlineLogFilePath, inlineConf)); + return new HoodieRecordTransformIterator(iterator, mapper); } @Override - protected ClosableIterator deserializeRecords(byte[] content) { + protected ClosableIterator deserializeRecords(byte[] content, HoodieRecord.Mapper mapper) 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 a9b25844ec7ef..a896152f42e16 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 @@ -22,8 +22,10 @@ import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; import org.apache.hudi.avro.model.HoodieMergeArchiveFilePlan; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +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; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; @@ -257,10 +259,11 @@ 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(HoodieAvroIndexedRecord::new)) { StreamSupport.stream(Spliterators.spliteratorUnknownSize(itr, Spliterator.IMMUTABLE), true) // Filter blocks in desired time window - .filter(r -> commitsFilter.apply((GenericRecord) r)) + .map(r -> (GenericRecord) ((HoodieAvroIndexedRecord) r).toIndexedRecord().get()) + .filter(commitsFilter::apply) .map(r -> readCommit((GenericRecord) r, loadInstantDetails)) .filter(c -> filter == null || filter.isInRange(c)) .forEach(instantsInRange::add); 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 9040a04d5edcf..ac0c2bbaa1c0d 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 @@ -193,6 +193,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/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/io/storage/HoodieAvroFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReader.java new file mode 100644 index 0000000000000..f454fa28f1929 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReader.java @@ -0,0 +1,102 @@ +/* + * 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.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.Mapper; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.MappingIterator; +import org.apache.hudi.common.util.Option; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.List; + +public interface HoodieAvroFileReader extends HoodieFileReader, AutoCloseable { + + ClosableIterator getRecordIterator(Schema readerSchema) throws IOException; + + default Option getRecordByKey(String key, Schema readerSchema) throws IOException { + throw new UnsupportedOperationException(); + } + + 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 getRecordsByKeysIterator(List keys, Schema schema, HoodieRecord.Mapper mapper) throws IOException { + ClosableIterator iterator = getRecordsByKeysIterator(keys, schema); + return new HoodieRecordTransformIterator(iterator, mapper); + } + + default ClosableIterator getRecordsByKeyPrefixIterator(List keyPrefixes, Schema schema, HoodieRecord.Mapper mapper) throws IOException { + ClosableIterator iterator = getRecordsByKeyPrefixIterator(keyPrefixes, schema); + return new HoodieRecordTransformIterator(iterator, mapper); + } + + @Override + default ClosableIterator getRecordIterator(Schema schema, HoodieRecord.Mapper mapper) throws IOException { + return new MappingIterator<>(getRecordIterator(schema), mapper::apply); + } + + @Override + default Option getRecordByKey(String key, Schema readerSchema, HoodieRecord.Mapper mapper) throws IOException { + return getRecordByKey(key, readerSchema).map(mapper::apply); + } + + class HoodieRecordTransformIterator implements ClosableIterator { + private final ClosableIterator dataIterator; + private final HoodieRecord.Mapper mapper; + + public HoodieRecordTransformIterator(ClosableIterator dataIterator, Mapper mapper) { + this.dataIterator = dataIterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return dataIterator.hasNext(); + } + + @Override + public HoodieRecord next() { + return mapper.apply(dataIterator.next()); + } + + @Override + public void close() { + dataIterator.close(); + } + } +} 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..947b83fe8600c 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 = (IndexedRecord)record.toIndexedRecord(schema, props).get(); + writeAvroWithMetadata(key, avroPayload); + } + + @Override + default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException { + IndexedRecord avroPayload = (IndexedRecord)record.toIndexedRecord(schema, props).get(); + 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/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java similarity index 88% 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 899c2475da26c..28159bc23bc97 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 @@ -53,8 +53,8 @@ import java.util.List; import java.util.Objects; import java.util.Set; -import java.util.TreeSet; import java.util.stream.Collectors; +import java.util.TreeSet; import static org.apache.hudi.common.util.CollectionUtils.toStream; import static org.apache.hudi.common.util.ValidationUtils.checkState; @@ -64,7 +64,7 @@ *

* {@link HoodieFileReader} implementation allowing to read from {@link HFile}. */ -public class HoodieHFileReader implements HoodieFileReader { +public class HoodieAvroHFileReader implements HoodieAvroFileReader { // TODO HoodieHFileReader right now tightly coupled to MT, we should break that coupling public static final String SCHEMA_KEY = "schema"; @@ -75,7 +75,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; @@ -90,21 +90,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 @@ -171,38 +171,36 @@ public Set filterRowKeys(Set candidateRowKeys) { @SuppressWarnings("unchecked") @Override - public Option getRecordByKey(String key, Schema readerSchema) throws IOException { + public Option getRecordByKey(String key, Schema readerSchema) throws IOException { synchronized (sharedScannerLock) { - return (Option) fetchRecordByKeyInternal(sharedScanner, key, getSchema(), readerSchema); + return fetchRecordByKeyInternal(sharedScanner, key, getSchema(), readerSchema); } } - @SuppressWarnings("unchecked") - @Override - public ClosableIterator getRecordIterator(Schema readerSchema) throws IOException { + 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 { + public ClosableIterator getRecordsByKeysIterator(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 { + public ClosableIterator getRecordsByKeyPrefixIterator(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 @@ -227,7 +225,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 { @@ -260,8 +258,8 @@ private static Iterator getRecordByKeyPrefixIteratorInternal(HFil } } - class KeyPrefixIterator implements Iterator { - private GenericRecord next = null; + class KeyPrefixIterator implements Iterator { + private IndexedRecord next = null; private boolean eof = false; @Override @@ -294,8 +292,8 @@ public boolean hasNext() { } @Override - public GenericRecord next() { - GenericRecord next = this.next; + public IndexedRecord next() { + IndexedRecord next = this.next; this.next = null; return next; } @@ -304,7 +302,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(); @@ -368,7 +366,7 @@ 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)) .collect(Collectors.toList()); @@ -379,7 +377,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()); } @@ -389,7 +387,7 @@ 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); @@ -407,16 +405,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(); @@ -452,8 +450,8 @@ public boolean hasNext() { } @Override - public GenericRecord next() { - GenericRecord next = this.next; + public IndexedRecord next() { + IndexedRecord next = this.next; this.next = null; return next; } @@ -464,7 +462,7 @@ public void close() { } } - private static class RecordByKeyIterator implements ClosableIterator { + private static class RecordByKeyIterator implements ClosableIterator { private final Iterator keyIterator; private final HFileScanner scanner; @@ -472,7 +470,7 @@ private static class RecordByKeyIterator implements ClosableIterator keys, Schema writerSchema, Schema readerSchema) throws IOException { this.keyIterator = keys.iterator(); @@ -493,7 +491,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; @@ -506,8 +504,8 @@ public boolean hasNext() { } @Override - public GenericRecord next() { - GenericRecord next = this.next; + public IndexedRecord next() { + IndexedRecord next = this.next; this.next = null; return next; } @@ -518,13 +516,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; @@ -559,8 +557,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-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java similarity index 92% rename from hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcReader.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java index 5431bf3782af2..8f953d25b5ab6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java @@ -38,12 +38,12 @@ import java.io.IOException; import java.util.Set; -public class HoodieOrcReader implements HoodieFileReader { +public class HoodieAvroOrcReader implements HoodieAvroFileReader { private Path path; private 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); @@ -65,7 +65,7 @@ public Set filterRowKeys(Set candidateRowKeys) { } @Override - public ClosableIterator getRecordIterator(Schema schema) throws IOException { + public ClosableIterator getRecordIterator(Schema schema) throws IOException { try { Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf)); TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(schema); diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java similarity index 85% rename from hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java index 804e4354c749e..83ffaf589f8ae 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java @@ -36,14 +36,14 @@ import java.util.List; import java.util.Set; -public class HoodieParquetReader implements HoodieFileReader { - +public class HoodieAvroParquetReader implements HoodieAvroFileReader { + private final Path path; private final Configuration conf; private final BaseFileUtils parquetUtils; private List readerIterators = new ArrayList<>(); - public HoodieParquetReader(Configuration configuration, Path path) { + public HoodieAvroParquetReader(Configuration configuration, Path path) { this.conf = configuration; this.path = path; this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); @@ -65,10 +65,10 @@ public Set filterRowKeys(Set candidateRowKeys) { } @Override - public ClosableIterator getRecordIterator(Schema schema) throws IOException { + public ClosableIterator getRecordIterator(Schema schema) throws IOException { AvroReadSupport.setAvroReadSchema(conf, schema); - ParquetReader reader = AvroParquetReader.builder(path).withConf(conf).build(); - ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader); + ParquetReader reader = AvroParquetReader.builder(path).withConf(conf).build(); + ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader); readerIterators.add(parquetReaderIterator); return parquetReaderIterator; } 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 6490425c42b75..175722bb9be85 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 @@ -19,8 +19,8 @@ package org.apache.hudi.io.storage; import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; 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; @@ -29,7 +29,7 @@ import java.util.List; import java.util.Set; -public interface HoodieFileReader extends AutoCloseable { +public interface HoodieFileReader extends AutoCloseable { String[] readMinMaxRecordKeys(); @@ -37,34 +37,34 @@ public interface HoodieFileReader extends AutoCloseable Set filterRowKeys(Set candidateRowKeys); - ClosableIterator getRecordIterator(Schema readerSchema) throws IOException; + ClosableIterator getRecordIterator(Schema readerSchema, HoodieRecord.Mapper mapper) throws IOException; - default ClosableIterator getRecordIterator() throws IOException { - return getRecordIterator(getSchema()); + default ClosableIterator getRecordIterator(HoodieRecord.Mapper mapper) throws IOException { + return getRecordIterator(getSchema(), mapper); } - default Option getRecordByKey(String key, Schema readerSchema) throws IOException { + default Option getRecordByKey(String key, Schema readerSchema, HoodieRecord.Mapper mapper) throws IOException { throw new UnsupportedOperationException(); } - default Option getRecordByKey(String key) throws IOException { - return getRecordByKey(key, getSchema()); + default Option getRecordByKey(String key, HoodieRecord.Mapper mapper) throws IOException { + return getRecordByKey(key, getSchema(), mapper); } - default ClosableIterator getRecordsByKeysIterator(List keys, Schema schema) throws IOException { + default ClosableIterator getRecordsByKeysIterator(List keys, Schema schema, HoodieRecord.Mapper mapper) throws IOException { throw new UnsupportedOperationException(); } - default ClosableIterator getRecordsByKeysIterator(List keys) throws IOException { - return getRecordsByKeysIterator(keys, getSchema()); + default ClosableIterator getRecordsByKeysIterator(List keys, HoodieRecord.Mapper mapper) throws IOException { + return getRecordsByKeysIterator(keys, getSchema(), mapper); } - default ClosableIterator getRecordsByKeyPrefixIterator(List keyPrefixes, Schema schema) throws IOException { + default ClosableIterator getRecordsByKeyPrefixIterator(List keyPrefixes, Schema schema, HoodieRecord.Mapper mapper) throws IOException { throw new UnsupportedEncodingException(); } - default ClosableIterator getRecordsByKeyPrefixIterator(List keyPrefixes) throws IOException { - return getRecordsByKeyPrefixIterator(keyPrefixes, getSchema()); + default ClosableIterator getRecordsByKeyPrefixIterator(List keyPrefixes, HoodieRecord.Mapper mapper) throws IOException { + return getRecordsByKeyPrefixIterator(keyPrefixes, getSchema(), mapper); } 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 f913df7e152a9..d2e35eb450a3e 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,22 +18,20 @@ 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 java.io.IOException; +import static org.apache.hudi.common.model.HoodieFileFormat.HFILE; import static org.apache.hudi.common.model.HoodieFileFormat.ORC; import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; -import static org.apache.hudi.common.model.HoodieFileFormat.HFILE; public class HoodieFileReaderFactory { - public static HoodieFileReader getFileReader(Configuration conf, Path path) throws IOException { + public static HoodieAvroFileReader getFileReader(Configuration conf, Path path) throws IOException { final String extension = FSUtils.getFileExtension(path.toString()); if (PARQUET.getFileExtension().equals(extension)) { return newParquetFileReader(conf, path); @@ -48,16 +46,16 @@ 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); + private static HoodieAvroFileReader newParquetFileReader(Configuration conf, Path path) { + return new HoodieAvroParquetReader(conf, path); } - private static HoodieFileReader newHFileFileReader(Configuration conf, Path path) throws IOException { + private static HoodieAvroFileReader newHFileFileReader(Configuration conf, Path path) throws IOException { CacheConfig cacheConfig = new CacheConfig(conf); - return new HoodieHFileReader<>(conf, path, cacheConfig); + return new HoodieAvroHFileReader(conf, path, cacheConfig); } - private static HoodieFileReader newOrcFileReader(Configuration conf, Path path) { - return new HoodieOrcReader<>(conf, path); + private static HoodieAvroFileReader newOrcFileReader(Configuration conf, Path path) { + return new HoodieAvroOrcReader(conf, path); } } 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/HoodieHFileUtils.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java index 3767ea1832579..fafaad273c058 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 @@ -68,7 +68,7 @@ public static HFile.Reader createHFileReader( public static HFile.Reader createHFileReader( FileSystem fs, Path dummyPath, byte[] content) throws IOException { Configuration conf = new Configuration(); - 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-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 a2736018242b6..1b14c2279f80c 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 @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroWriteSupport; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.parquet.io.OutputStreamBackedOutputFile; import org.apache.parquet.hadoop.ParquetFileWriter; import org.apache.parquet.hadoop.ParquetWriter; @@ -32,15 +33,15 @@ import java.io.IOException; // TODO(HUDI-3035) unify w/ HoodieParquetWriter -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, HoodieAvroParquetConfig 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()) @@ -52,11 +53,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/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index cf941bb70cc3b..e8dddd9679e2e 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 @@ -34,6 +34,7 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieBaseFile; +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.table.HoodieTableConfig; @@ -51,7 +52,7 @@ 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.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -94,7 +95,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 Map, Pair> partitionReaders = new ConcurrentHashMap<>(); public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, @@ -155,12 +156,12 @@ public HoodieData> getRecordsByKeyPrefixes(L (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 = + Pair readers = openReaders(partitionName, fileSlice); try { List timings = new ArrayList<>(); - HoodieFileReader baseFileReader = readers.getKey(); + HoodieAvroFileReader baseFileReader = readers.getKey(); HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight(); if (baseFileReader == null && logRecordScanner == null) { @@ -198,11 +199,11 @@ public List>>> getRecord List>>> result = new ArrayList<>(); AtomicInteger fileSlicesKeysCount = new AtomicInteger(); partitionFileSliceToKeysMap.forEach((partitionFileSlicePair, fileSliceKeys) -> { - Pair readers = + Pair readers = getOrCreateReaders(partitionName, partitionFileSlicePair.getRight()); try { List timings = new ArrayList<>(); - HoodieFileReader baseFileReader = readers.getKey(); + HoodieAvroFileReader baseFileReader = readers.getKey(); HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight(); if (baseFileReader == null && logRecordScanner == null) { return; @@ -269,7 +270,7 @@ private Map>> readLogRecords( return logRecords; } - private List>>> readFromBaseAndMergeWithLogRecords(HoodieFileReader baseFileReader, + private List>>> readFromBaseAndMergeWithLogRecords(HoodieAvroFileReader baseFileReader, List keys, boolean fullKeys, Map>> logRecords, @@ -332,17 +333,20 @@ private List>>> readFrom } } - private Map> fetchBaseFileRecordsByKeys(HoodieFileReader baseFileReader, + private Map> fetchBaseFileRecordsByKeys(HoodieAvroFileReader baseFileReader, List keys, boolean fullKeys, String partitionName) throws IOException { - ClosableIterator records = fullKeys ? baseFileReader.getRecordsByKeysIterator(keys) - : baseFileReader.getRecordsByKeyPrefixIterator(keys); + ClosableIterator records = fullKeys ? baseFileReader.getRecordsByKeysIterator(keys, HoodieAvroIndexedRecord::new) + : baseFileReader.getRecordsByKeyPrefixIterator(keys, HoodieAvroIndexedRecord::new); 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)); } @@ -389,16 +393,16 @@ 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 getOrCreateReaders(String partitionName, FileSlice slice) { return partitionReaders.computeIfAbsent(Pair.of(partitionName, slice.getFileId()), k -> openReaders(partitionName, slice)); } - private Pair openReaders(String partitionName, FileSlice slice) { + private Pair openReaders(String partitionName, FileSlice slice) { try { HoodieTimer timer = new HoodieTimer().startTimer(); // Open base file reader - Pair baseFileReaderOpenTimePair = getBaseFileReader(slice, timer); - HoodieFileReader baseFileReader = baseFileReaderOpenTimePair.getKey(); + Pair baseFileReaderOpenTimePair = getBaseFileReader(slice, timer); + HoodieAvroFileReader baseFileReader = baseFileReaderOpenTimePair.getKey(); final long baseFileOpenMs = baseFileReaderOpenTimePair.getValue(); // Open the log record scanner using the log files from the latest file slice @@ -416,8 +420,8 @@ private Pair openReaders( } } - private Pair getBaseFileReader(FileSlice slice, HoodieTimer timer) throws IOException { - HoodieFileReader baseFileReader = null; + private Pair getBaseFileReader(FileSlice slice, HoodieTimer timer) throws IOException { + HoodieAvroFileReader baseFileReader = null; Long baseFileOpenMs; // If the base file is present then create a reader Option basefile = slice.getBaseFile(); @@ -562,12 +566,12 @@ public void close() { * @param partitionFileSlicePair - Partition and FileSlice */ private synchronized void close(Pair partitionFileSlicePair) { - Pair readers = + Pair readers = partitionReaders.remove(partitionFileSlicePair); closeReader(readers); } - private void closeReader(Pair readers) { + private void closeReader(Pair readers) { if (readers != null) { try { if (readers.getKey() != null) { 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 c9bdc59da9763..85f0a9aa9f1e5 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 @@ -53,7 +53,7 @@ import org.apache.hudi.common.util.hash.FileIndexID; import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.exception.HoodieMetadataException; -import org.apache.hudi.io.storage.HoodieHFileReader; +import org.apache.hudi.io.storage.HoodieAvroHFileReader; import org.apache.hudi.util.Lazy; import java.io.IOException; @@ -118,7 +118,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload convertMetadataToBloomFilterRecords( } final Path writeFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition); - try (HoodieFileReader fileReader = + try (HoodieAvroFileReader fileReader = HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), writeFilePath)) { try { final BloomFilter fileBloomFilter = fileReader.readBloomFilter(); @@ -878,7 +878,7 @@ public static HoodieData convertFilesToBloomFilterRecords(HoodieEn } final String pathWithPartition = partitionName + "/" + appendedFile; final Path appendedFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition); - try (HoodieFileReader fileReader = + try (HoodieAvroFileReader fileReader = HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), appendedFilePath)) { final BloomFilter fileBloomFilter = fileReader.readBloomFilter(); if (fileBloomFilter == null) { 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 4fa53bb41f9f8..289180580b11a 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,10 @@ 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.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.AppendResult; import org.apache.hudi.common.table.log.HoodieLogFileReader; @@ -1054,7 +1056,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,7 +1197,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) { @@ -1916,7 +1918,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); @@ -2003,10 +2005,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 AVRO_DATA_BLOCK: @@ -2038,10 +2040,10 @@ private static Stream testArguments() { * Utility to convert the given iterator to a List. */ private static List getRecords(HoodieDataBlock dataBlock) { - ClosableIterator itr = dataBlock.getRecordIterator(); + ClosableIterator itr = dataBlock.getRecordIterator(HoodieAvroIndexedRecord::new); List elements = new ArrayList<>(); - itr.forEachRemaining(elements::add); + itr.forEachRemaining(r -> elements.add((IndexedRecord) 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 6c4d69a05b296..77a01bd024e86 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; @@ -28,7 +29,6 @@ import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.minicluster.MiniClusterUtil; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; @@ -52,6 +52,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; @@ -101,7 +102,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/io/storage/TestHoodieFileReaderFactory.java b/hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileReaderFactory.java similarity index 78% 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..f049033688efb 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,6 @@ package org.apache.hudi.io.storage; -import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.Test; @@ -32,7 +31,7 @@ /** * Tests for {@link HoodieFileReaderFactory}. */ -public class TestHoodieFileReaderFactory { +public class TestHoodieAvroFileReaderFactory { @TempDir public java.nio.file.Path tempDir; @@ -41,19 +40,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); + HoodieAvroFileReader parquetReader = HoodieFileReaderFactory.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); + HoodieAvroFileReader logWriter = HoodieFileReaderFactory.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); + HoodieAvroFileReader orcReader = HoodieFileReaderFactory.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..7fc93c776f5a8 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,6 +37,7 @@ 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.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.examples.quickstart.utils.QuickstartConfigurations; import org.apache.parquet.Strings; @@ -322,7 +323,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); 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 a415ac9d46165..4955f435d679e 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 @@ -36,7 +36,7 @@ import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.IOUtils; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; import org.apache.hudi.sink.bulk.sort.SortOperatorGen; @@ -74,6 +74,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; @@ -230,7 +231,7 @@ private Iterator readRecordsForGroupWithLogs(List for (ClusteringOperation clusteringOp : clusteringOps) { try { - Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) + Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) ? Option.empty() : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() @@ -255,7 +256,7 @@ private Iterator readRecordsForGroupWithLogs(List 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()); } catch (IOException e) { throw new HoodieIOException("Failed to read next record", e); } 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 c31c2bbadae25..a073c7f7c7700 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 @@ -21,6 +21,7 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; 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.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; @@ -651,7 +652,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); 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..4fa62e79eabd0 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; @@ -34,20 +31,23 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; 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 +63,7 @@ public boolean next(NullWritable key, ArrayWritable value) throws IOException { return false; } - GenericRecord record = recordIterator.next(); + IndexedRecord record = recordIterator.next(); 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/InputSplitUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java index e485e72c25755..ae550cb335a56 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java @@ -18,19 +18,20 @@ package org.apache.hudi.hadoop; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.nio.charset.StandardCharsets; import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.FileSplit; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + public class InputSplitUtils { public static void writeString(String str, DataOutput out) throws IOException { @@ -63,7 +64,7 @@ public static boolean readBoolean(DataInput in) throws IOException { public static Schema getBaseFileSchema(FileSplit split, Configuration conf) { try { if (split instanceof BootstrapBaseFileSplit) { - HoodieFileReader storageReader = HoodieFileReaderFactory.getFileReader(conf, + HoodieAvroFileReader storageReader = HoodieFileReaderFactory.getFileReader(conf, ((BootstrapBaseFileSplit)(split)).getBootstrapFileSplit().getPath()); return HoodieAvroUtils.addMetadataFields(storageReader.getSchema()); } 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 b917f004bcd06..4fa3781aeaf4d 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 @@ -27,9 +27,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; 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.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; @@ -50,7 +48,7 @@ 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; @@ -96,9 +94,9 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept 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); } } 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 84c808865072a..e418bebb50136 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 @@ -92,7 +92,7 @@ public RealtimeUnmergedRecordReader(RealtimeSplit split, JobConf job, .withBufferSize(this.jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) .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(); ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema()); this.executor.getQueue().insertRecord(aWritable); }) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java index 0e4f9c304cb2b..e85a4d2976c82 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java @@ -44,7 +44,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; -import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -71,7 +71,7 @@ public class HoodieRealtimeRecordReaderUtils { */ public static Schema readSchema(Configuration conf, Path filePath) { try { - HoodieFileReader storageReader = HoodieFileReaderFactory.getFileReader(conf, filePath); + HoodieAvroFileReader storageReader = HoodieFileReaderFactory.getFileReader(conf, filePath); return storageReader.getSchema(); } catch (IOException e) { throw new HoodieIOException("Failed to read schema from " + filePath, e); 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 1081e43175630..4199d0cb7e25d 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; @@ -373,13 +374,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); + dataBlock = new HoodieParquetDataBlock(hoodieRecords, header, HoodieRecord.RECORD_KEY_METADATA_FIELD, CompressionCodecName.GZIP); } 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/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index 2648740f54e0f..db11bd6717688 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,6 +18,10 @@ 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; @@ -25,6 +29,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; 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.table.HoodieTableMetaClient; @@ -34,19 +39,15 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; 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; @@ -61,8 +62,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; @@ -264,7 +263,7 @@ 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(), + HoodieAvroFileReader reader = HoodieFileReaderFactory.getFileReader(metaClient.getHadoopConf(), new Path(fileSlice.getBaseFile().get().getPath())); return reader.getRecordIterator(schema); } else { @@ -287,12 +286,12 @@ private Iterator readColumnarOrLogFiles(FileSlice fileSlice) thro .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) .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-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java index 9a793c4227936..1848972dd5f5b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java @@ -21,7 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.client.HoodieInternalWriteStatus; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; @@ -123,7 +123,7 @@ public void write(InternalRow record) throws IOException { try { String partitionPath = null; if (populateMetaFields) { // usual path where meta fields are pre populated in prep step. - partitionPath = String.valueOf(record.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_POS)); + partitionPath = String.valueOf(record.getUTF8String(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal())); } else { // if meta columns are disabled. if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen partitionPath = ""; diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala index b1e03f86ff807..37bcecfda7c06 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala @@ -244,7 +244,7 @@ trait ColumnStatsIndexSupport extends SparkAdapterSupport { it.map { record => // schema and props are ignored for generating metadata record from the payload // instead, the underlying file system, or bloom filter, or columns stats metadata (part of payload) are directly used - toScalaOption(record.getData.getInsertValue(null, null)) + toScalaOption(record.toIndexedRecord(null, null)) .flatMap(avroRecord => converter(avroRecord.asInstanceOf[GenericRecord])) .orNull } 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 08f87816d7c35..5b2ba815e4be4 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 @@ -36,7 +36,7 @@ import org.apache.hudi.common.util.StringUtils import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter -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 import org.apache.spark.rdd.RDD @@ -518,7 +518,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 = requiredSchema.structTypeSchema 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 a7ca60865fbba..f022bce1b302b 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 @@ -25,8 +25,6 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.JobConf import org.apache.hudi.HoodieConversionUtils.{toJavaOption, toScalaOption} import org.apache.hudi.HoodieMergeOnReadRDD.{AvroDeserializerSupport, collectFieldOrdinals, getPartitionPath, projectAvro, projectAvroUnsafe, projectRowUnsafe, resolveAvroSchemaNullability} -import org.apache.hudi.MergeOnReadSnapshotRelation.getFilePath -import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.engine.HoodieLocalEngineContext import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath @@ -47,7 +45,6 @@ import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafePro import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.types.StructType import org.apache.spark.{Partition, SerializableWritable, SparkContext, TaskContext} - import java.io.Closeable import java.util.Properties import scala.annotation.tailrec @@ -180,13 +177,13 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, protected lazy val logRecordsIterator: Iterator[Option[GenericRecord]] = logRecords.iterator.map { case (_, record) => - val avroRecordOpt = toScalaOption(record.getData.getInsertValue(logFileReaderAvroSchema, payloadProps)) + val avroRecordOpt = toScalaOption(record.toIndexedRecord(logFileReaderAvroSchema, payloadProps)) avroRecordOpt.map { avroRecord => projectAvroUnsafe(avroRecord, requiredAvroSchema, requiredSchemaFieldOrdinals, recordBuilder) } } - protected def removeLogRecord(key: String): Option[HoodieRecord[_ <: HoodieRecordPayload[_]]] = + protected def removeLogRecord(key: String): Option[HoodieRecord[_]] = logRecords.remove(key) override def hasNext: Boolean = hasNextInternal @@ -279,7 +276,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, recordToLoad = unsafeProjection(projectRowUnsafe(curRowRecord, requiredSchema.structTypeSchema, requiredSchemaFieldOrdinals)) true } else { - val mergedAvroRecordOpt = merge(serialize(curRowRecord), updatedRecordOpt.get) + val mergedAvroRecordOpt = merge(serialize(curRowRecord), updatedRecordOpt.get.asInstanceOf[HoodieRecord[_ <: HoodieRecordPayload[_]]]) if (mergedAvroRecordOpt.isEmpty) { // Record has been deleted, skipping this.hasNextInternal 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 da2736e59bdda..78d299e3eff2a 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 @@ -485,10 +485,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) } } 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 95a023abb687d..bc6a678d120b8 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; @@ -149,13 +150,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/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 e59a609321549..12e5cef4bc2be 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 @@ -131,7 +131,7 @@ class ExpressionPayload(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 } @@ -169,7 +169,7 @@ class ExpressionPayload(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) } } 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 7c86da0c9e362..f80261cd950da 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 @@ -121,7 +121,7 @@ class TestCOWDataSource extends HoodieClientTestBase { // 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")) assertThrows(classOf[HoodieException], new Executable { override def execute(): Unit = { 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 8be2ace89f8f1..946fef969285e 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; @@ -83,6 +84,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.fail; @@ -434,8 +436,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 80cc56a4e579b..a2cb491d58419 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 @@ -62,7 +62,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; @@ -1026,7 +1025,7 @@ private List getAllColumnNames() { private Option readBloomFilterFromFile(String partitionPath, String filename) { Path path = new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPath), filename); - HoodieFileReader fileReader; + HoodieFileReader fileReader; try { fileReader = HoodieFileReaderFactory.getFileReader(metaClient.getHadoopConf(), path); } catch (IOException e) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DeleteSupportSchemaPostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DeleteSupportSchemaPostProcessor.java index e58dc4e20611b..aad095399875d 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DeleteSupportSchemaPostProcessor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DeleteSupportSchemaPostProcessor.java @@ -44,8 +44,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; } @@ -56,7 +56,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/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java index e008c04e33d1c..be15aa6d12b76 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java @@ -119,7 +119,7 @@ public JavaRDD 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/sources/TestJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java index 1d80b68449c79..a313fe87ee029 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 @@ -200,7 +200,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); @@ -220,7 +220,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();