From b8601a9f58ae4dc5099cc7f828df7fd8d0aeb798 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 3 Feb 2022 20:24:04 -0800 Subject: [PATCH] [HUDI-2656] Generalize HoodieIndex for flexible record data type (#3893) Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com> --- .../hudi/client/BaseHoodieWriteClient.java | 6 +- .../execution/HoodieLazyInsertIterable.java | 4 +- .../org/apache/hudi/index/HoodieIndex.java | 13 ++-- .../apache/hudi/index/HoodieIndexUtils.java | 4 +- .../bloom/BaseHoodieBloomIndexHelper.java | 4 +- .../hudi/index/bloom/HoodieBloomIndex.java | 22 ++++--- .../index/bloom/HoodieGlobalBloomIndex.java | 23 +++---- .../ListBasedHoodieBloomIndexHelper.java | 4 +- .../hudi/index/bucket/HoodieBucketIndex.java | 16 +++-- .../inmemory/HoodieInMemoryHashIndex.java | 13 ++-- .../index/simple/HoodieGlobalSimpleIndex.java | 33 +++++----- .../hudi/index/simple/HoodieSimpleIndex.java | 21 +++---- .../apache/hudi/io/HoodieAppendHandle.java | 2 +- .../apache/hudi/io/HoodieCreateHandle.java | 4 +- .../org/apache/hudi/io/HoodieMergeHandle.java | 2 +- .../hudi/io/HoodieSortedMergeHandle.java | 2 +- .../org/apache/hudi/io/HoodieWriteHandle.java | 2 +- .../org/apache/hudi/table/HoodieTable.java | 6 +- .../bootstrap/BootstrapRecordConsumer.java | 4 +- .../table/action/commit/BaseWriteHelper.java | 2 +- .../storage/TestHoodieHFileReaderWriter.java | 5 +- .../testutils/HoodieWriteableTestTable.java | 9 +-- .../apache/hudi/index/FlinkHoodieIndex.java | 16 ++--- .../hudi/index/FlinkHoodieIndexFactory.java | 6 +- .../index/state/FlinkInMemoryStateIndex.java | 13 ++-- .../action/commit/FlinkDeleteHelper.java | 3 +- .../table/action/commit/FlinkWriteHelper.java | 5 +- .../bloom/TestFlinkHoodieBloomIndex.java | 39 ++++++------ .../HoodieFlinkWriteableTestTable.java | 3 +- .../run/strategy/JavaExecutionStrategy.java | 3 +- .../apache/hudi/index/JavaHoodieIndex.java | 15 ++--- .../hudi/index/JavaHoodieIndexFactory.java | 4 +- .../table/action/commit/JavaDeleteHelper.java | 3 +- .../table/action/commit/JavaWriteHelper.java | 5 +- .../action/commit/TestHoodieConcatHandle.java | 22 +++---- .../TestJavaCopyOnWriteActionExecutor.java | 25 ++++---- .../apache/hudi/client/HoodieReadClient.java | 7 ++- .../MultipleSparkJobExecutionStrategy.java | 3 +- .../SingleSparkJobExecutionStrategy.java | 3 +- .../org/apache/hudi/data/HoodieJavaRDD.java | 4 +- .../RDDSpatialCurveSortPartitioner.java | 8 ++- .../apache/hudi/index/SparkHoodieIndex.java | 15 ++--- .../hudi/index/SparkHoodieIndexFactory.java | 12 ++-- .../bloom/SparkHoodieBloomIndexHelper.java | 3 +- .../index/hbase/SparkHoodieHBaseIndex.java | 32 +++++----- .../OrcBootstrapMetadataHandler.java | 3 +- .../ParquetBootstrapMetadataHandler.java | 3 +- .../commit/SparkBucketIndexPartitioner.java | 2 +- .../action/commit/SparkDeleteHelper.java | 3 +- .../table/action/commit/SparkWriteHelper.java | 5 +- .../hudi/client/TestHoodieReadClient.java | 5 +- .../hudi/client/TestTableSchemaEvolution.java | 5 +- .../client/TestUpdateSchemaEvolution.java | 5 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 11 ++-- .../client/functional/TestHoodieIndex.java | 17 +++--- .../execution/TestBoundedInMemoryQueue.java | 23 +++---- .../SparkClientFunctionalTestSuite.java | 5 +- .../hudi/index/TestHoodieIndexConfigs.java | 60 +------------------ .../index/bloom/TestHoodieBloomIndex.java | 41 ++++++------- .../bloom/TestHoodieGlobalBloomIndex.java | 28 ++++----- .../index/bucket/TestBucketIdentifier.java | 14 +++-- .../index/bucket/TestHoodieBucketIndex.java | 18 +++--- .../hbase/TestSparkHoodieHBaseIndex.java} | 15 ++--- .../commit/TestCopyOnWriteActionExecutor.java | 21 +++---- .../table/action/commit/TestDeleteHelper.java | 4 +- .../action/compact/TestHoodieCompactor.java | 2 +- ...stHoodieSparkMergeOnReadTableRollback.java | 3 +- .../hudi/testutils/HoodieClientTestBase.java | 3 +- .../hudi/common/model/HoodieAvroRecord.java | 50 ++++++++++++++++ .../hudi/common/model/HoodieRecord.java | 14 +++-- .../log/AbstractHoodieLogRecordReader.java | 3 +- .../log/HoodieMergedLogRecordScanner.java | 3 +- .../hudi/common/util/SpillableMapUtils.java | 5 +- .../metadata/HoodieBackedTableMetadata.java | 3 +- .../HoodieMetadataMergedLogRecordReader.java | 36 +++++------ .../hudi/metadata/HoodieMetadataPayload.java | 15 ++--- .../common/fs/inline/TestParquetInLining.java | 4 +- .../functional/TestHoodieLogFormat.java | 23 +++---- .../hudi/common/model/TestHoodieRecord.java | 2 +- .../testutils/HoodieTestDataGenerator.java | 19 +++--- .../hudi/common/testutils/SchemaTestUtil.java | 9 +-- .../testutils/SpillableMapTestUtils.java | 3 +- .../util/collection/TestBitCaskDiskMap.java | 7 ++- .../collection/TestExternalSpillableMap.java | 11 ++-- .../util/collection/TestRocksDbDiskMap.java | 10 ++-- .../common/HoodieExampleDataGenerator.java | 5 +- .../java/HoodieJavaWriteClientExample.java | 7 ++- .../apache/hudi/sink/StreamWriteFunction.java | 7 ++- .../sink/bootstrap/BootstrapOperator.java | 3 +- .../hudi/sink/bootstrap/IndexRecord.java | 8 ++- .../partitioner/BucketAssignFunction.java | 3 +- .../transform/RowDataToHoodieFunction.java | 3 +- .../format/mor/MergeOnReadInputFormat.java | 9 +-- .../RealtimeCompactedRecordReader.java | 5 +- .../testsuite/HoodieTestSuiteWriter.java | 11 ++-- .../writers/AbstractConnectWriter.java | 3 +- .../writers/TestAbstractConnectWriter.java | 3 +- .../java/org/apache/hudi/DataSourceUtils.java | 12 ++-- .../java/org/apache/hudi/QuickstartUtils.java | 5 +- .../apache/hudi/functional/TestBootstrap.java | 3 +- .../hudi/functional/TestOrcBootstrap.java | 3 +- .../hudi/utilities/HDFSParquetImporter.java | 3 +- .../utilities/deltastreamer/DeltaSync.java | 3 +- .../utilities/testutils/JdbcTestUtils.java | 6 +- .../testutils/UtilitiesTestBase.java | 3 +- 105 files changed, 564 insertions(+), 504 deletions(-) rename hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/{client/functional/TestHBaseIndex.java => index/hbase/TestSparkHoodieHBaseIndex.java} (98%) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java 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 867e2f99e..56db18576 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 @@ -105,7 +105,7 @@ public abstract class BaseHoodieWriteClient index; + private final transient HoodieIndex index; protected transient Timer.Context writeTimer = null; protected transient Timer.Context compactionTimer; @@ -142,7 +142,7 @@ public abstract class BaseHoodieWriteClient createIndex(HoodieWriteConfig writeConfig); + protected abstract HoodieIndex createIndex(HoodieWriteConfig writeConfig); public void setOperationType(WriteOperationType operationType) { this.operationType = operationType; @@ -1160,7 +1160,7 @@ public abstract class BaseHoodieWriteClient getIndex() { + public HoodieIndex getIndex() { return index; } 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 0d28c74e1..b078076b8 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,7 +18,6 @@ package org.apache.hudi.execution; -import java.util.Properties; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.LazyIterableIterator; import org.apache.hudi.common.engine.TaskContextSupplier; @@ -36,6 +35,7 @@ import org.apache.avro.generic.IndexedRecord; import java.util.Iterator; import java.util.List; +import java.util.Properties; import java.util.function.Function; /** @@ -87,7 +87,7 @@ public abstract class HoodieLazyInsertIterable public HoodieInsertValueGenResult(T record, Schema schema, Properties properties) { this.record = record; try { - this.insertValue = record.getData().getInsertValue(schema, properties); + this.insertValue = ((HoodieRecordPayload) record.getData()).getInsertValue(schema, properties); } catch (Exception e) { this.exception = Option.of(e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java index 0428d12c4..7cf45f12e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; @@ -39,13 +38,11 @@ import java.io.Serializable; /** * Base class for different types of indexes to determine the mapping from uuid. * - * @param Sub type of HoodieRecordPayload * @param Type of inputs for deprecated APIs - * @param Type of keys for deprecated APIs * @param Type of outputs for deprecated APIs */ @PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) -public abstract class HoodieIndex implements Serializable { +public abstract class HoodieIndex implements Serializable { protected final HoodieWriteConfig config; @@ -60,7 +57,7 @@ public abstract class HoodieIndex implem @Deprecated @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public I tagLocation(I records, HoodieEngineContext context, - HoodieTable hoodieTable) throws HoodieIndexException { + HoodieTable hoodieTable) throws HoodieIndexException { throw new HoodieNotSupportedException("Deprecated API should not be called"); } @@ -70,7 +67,7 @@ public abstract class HoodieIndex implem @Deprecated @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public O updateLocation(O writeStatuses, HoodieEngineContext context, - HoodieTable hoodieTable) throws HoodieIndexException { + HoodieTable hoodieTable) throws HoodieIndexException { throw new HoodieNotSupportedException("Deprecated API should not be called"); } @@ -79,8 +76,8 @@ public abstract class HoodieIndex implem * the row (if it is actually present). */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public abstract HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException; /** 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 c6a9a14f5..b714c5033 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 @@ -102,14 +102,14 @@ public class HoodieIndexUtils { * @return the tagged {@link HoodieRecord} */ public static HoodieRecord getTaggedRecord(HoodieRecord inputRecord, Option location) { - HoodieRecord record = inputRecord; + HoodieRecord record = inputRecord; if (location.isPresent()) { // When you have a record in multiple files in the same partition, then collection // will have 2 entries with the same exact in memory copy of the HoodieRecord and the 2 // separate filenames that the record is found in. This will result in setting // currentLocation 2 times and it will fail the second time. So creating a new in memory // copy of the hoodie record. - record = new HoodieRecord<>(inputRecord); + record = inputRecord.newInstance(); record.unseal(); record.setCurrentLocation(location.get()); record.seal(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java index 9f0e81563..9430d9bb5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java @@ -24,7 +24,7 @@ import org.apache.hudi.common.data.HoodiePairData; 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.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -51,7 +51,7 @@ public abstract class BaseHoodieBloomIndexHelper implements Serializable { public abstract HoodiePairData findMatchingFilesForRecordKeys( HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable, HoodiePairData partitionRecordKeyPairs, - HoodieData> fileComparisonPairs, + HoodieData> fileComparisonPairs, Map> partitionToFileInfo, Map recordsPerPartition); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java index 5b7c7af91..d3e73c058 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java @@ -29,7 +29,6 @@ import org.apache.hudi.common.fs.FSUtils; 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.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; @@ -59,8 +58,7 @@ import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPar /** * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata. */ -public class HoodieBloomIndex> - extends HoodieIndex { +public class HoodieBloomIndex extends HoodieIndex { private static final Logger LOG = LogManager.getLogger(HoodieBloomIndex.class); private final BaseHoodieBloomIndexHelper bloomIndexHelper; @@ -71,8 +69,8 @@ public class HoodieBloomIndex> } @Override - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) { // Step 0: cache the input records if needed if (config.getBloomIndexUseCaching()) { @@ -98,7 +96,7 @@ public class HoodieBloomIndex> } // Step 3: Tag the incoming records, as inserts or updates, by joining with existing record keys - HoodieData> taggedRecords = tagLocationBacktoRecords(keyFilenamePairs, records); + HoodieData> taggedRecords = tagLocationBacktoRecords(keyFilenamePairs, records); if (config.getBloomIndexUseCaching()) { records.unpersist(); @@ -133,7 +131,7 @@ public class HoodieBloomIndex> // Step 3: Obtain a HoodieData, for each incoming record, that already exists, with the file id, // that contains it. - HoodieData> fileComparisonPairs = + HoodieData> fileComparisonPairs = explodeRecordsWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairs); return bloomIndexHelper.findMatchingFilesForRecordKeys(config, context, hoodieTable, @@ -261,7 +259,7 @@ public class HoodieBloomIndex> * Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on * recordKey ranges in the index info. */ - HoodieData> explodeRecordsWithFileComparisons( + HoodieData> explodeRecordsWithFileComparisons( final Map> partitionToFileIndexInfo, HoodiePairData partitionRecordKeyPairs) { IndexFileFilter indexFileFilter = @@ -273,7 +271,7 @@ public class HoodieBloomIndex> String partitionPath = partitionRecordKeyPair.getLeft(); return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream() - .map(partitionFileIdPair -> new ImmutablePair<>(partitionFileIdPair.getRight(), + .map(partitionFileIdPair -> (Pair) new ImmutablePair<>(partitionFileIdPair.getRight(), new HoodieKey(recordKey, partitionPath))) .collect(Collectors.toList()); }).flatMap(List::iterator); @@ -282,10 +280,10 @@ public class HoodieBloomIndex> /** * Tag the back to the original HoodieRecord List. */ - protected HoodieData> tagLocationBacktoRecords( + protected HoodieData> tagLocationBacktoRecords( HoodiePairData keyFilenamePair, - HoodieData> records) { - HoodiePairData> keyRecordPairs = + HoodieData> records) { + HoodiePairData> keyRecordPairs = records.mapToPair(record -> new ImmutablePair<>(record.getKey(), record)); // Here as the records might have more data than keyFilenamePairs (some row keys' fileId is null), // so we do left outer join. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java index f06371388..5f2007ea5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.engine.HoodieEngineContext; 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.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -46,7 +47,7 @@ import java.util.stream.Collectors; * This filter will only work with hoodie table since it will only load partitions * with .hoodie_partition_metadata file in it. */ -public class HoodieGlobalBloomIndex> extends HoodieBloomIndex { +public class HoodieGlobalBloomIndex extends HoodieBloomIndex { public HoodieGlobalBloomIndex(HoodieWriteConfig config, BaseHoodieBloomIndexHelper bloomIndexHelper) { super(config, bloomIndexHelper); } @@ -73,7 +74,7 @@ public class HoodieGlobalBloomIndex> extends Ho */ @Override - HoodieData> explodeRecordsWithFileComparisons( + HoodieData> explodeRecordsWithFileComparisons( final Map> partitionToFileIndexInfo, HoodiePairData partitionRecordKeyPairs) { @@ -86,7 +87,7 @@ public class HoodieGlobalBloomIndex> extends Ho String partitionPath = partitionRecordKeyPair.getLeft(); return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream() - .map(partitionFileIdPair -> new ImmutablePair<>(partitionFileIdPair.getRight(), + .map(partitionFileIdPair -> (Pair) new ImmutablePair<>(partitionFileIdPair.getRight(), new HoodieKey(recordKey, partitionFileIdPair.getLeft()))) .collect(Collectors.toList()); }).flatMap(List::iterator); @@ -96,11 +97,11 @@ public class HoodieGlobalBloomIndex> extends Ho * Tagging for global index should only consider the record key. */ @Override - protected HoodieData> tagLocationBacktoRecords( + protected HoodieData> tagLocationBacktoRecords( HoodiePairData keyLocationPairs, - HoodieData> records) { + HoodieData> records) { - HoodiePairData> incomingRowKeyRecordPairs = + HoodiePairData> incomingRowKeyRecordPairs = records.mapToPair(record -> new ImmutablePair<>(record.getRecordKey(), record)); HoodiePairData> existingRecordKeyToRecordLocationHoodieKeyMap = @@ -109,29 +110,29 @@ public class HoodieGlobalBloomIndex> extends Ho // Here as the records might have more data than rowKeys (some rowKeys' fileId is null), so we do left outer join. return incomingRowKeyRecordPairs.leftOuterJoin(existingRecordKeyToRecordLocationHoodieKeyMap).values().flatMap(record -> { - final HoodieRecord hoodieRecord = record.getLeft(); + final HoodieRecord hoodieRecord = record.getLeft(); final Option> recordLocationHoodieKeyPair = record.getRight(); if (recordLocationHoodieKeyPair.isPresent()) { // Record key matched to file if (config.getBloomIndexUpdatePartitionPath() && !recordLocationHoodieKeyPair.get().getRight().getPartitionPath().equals(hoodieRecord.getPartitionPath())) { // Create an empty record to delete the record in the old partition - HoodieRecord deleteRecord = new HoodieRecord(recordLocationHoodieKeyPair.get().getRight(), + HoodieRecord deleteRecord = new HoodieAvroRecord(recordLocationHoodieKeyPair.get().getRight(), new EmptyHoodieRecordPayload()); deleteRecord.setCurrentLocation(recordLocationHoodieKeyPair.get().getLeft()); deleteRecord.seal(); // Tag the incoming record for inserting to the new partition - HoodieRecord insertRecord = HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty()); + HoodieRecord insertRecord = HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty()); return Arrays.asList(deleteRecord, insertRecord).iterator(); } else { // Ignore the incoming record's partition, regardless of whether it differs from its old partition or not. // When it differs, the record will still be updated at its old partition. return Collections.singletonList( - (HoodieRecord) HoodieIndexUtils.getTaggedRecord(new HoodieRecord<>(recordLocationHoodieKeyPair.get().getRight(), hoodieRecord.getData()), + (HoodieRecord) HoodieIndexUtils.getTaggedRecord(new HoodieAvroRecord(recordLocationHoodieKeyPair.get().getRight(), (HoodieRecordPayload) hoodieRecord.getData()), Option.ofNullable(recordLocationHoodieKeyPair.get().getLeft()))).iterator(); } } else { - return Collections.singletonList((HoodieRecord) HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty())).iterator(); + return Collections.singletonList((HoodieRecord) HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty())).iterator(); } }); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java index dde93cf37..c42d80c62 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java @@ -57,11 +57,11 @@ public class ListBasedHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper public HoodiePairData findMatchingFilesForRecordKeys( HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable, HoodiePairData partitionRecordKeyPairs, - HoodieData> fileComparisonPairs, + HoodieData> fileComparisonPairs, Map> partitionToFileInfo, Map recordsPerPartition) { List> fileComparisonPairList = HoodieList.getList(fileComparisonPairs).stream() - .sorted(Comparator.comparing(ImmutablePair::getLeft)).collect(toList()); + .sorted(Comparator.comparing(Pair::getLeft)).collect(toList()); List keyLookupResults = new ArrayList<>(); Iterator> iterator = new HoodieBaseBloomIndexCheckFunction( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java index acb06ea48..a243eea76 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java @@ -24,7 +24,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.HoodieRecordLocation; -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; @@ -34,6 +33,7 @@ import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.table.HoodieTable; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -42,10 +42,8 @@ import java.util.Map; /** * Hash indexing mechanism. - * @param */ -public class HoodieBucketIndex> - extends HoodieIndex { +public class HoodieBucketIndex extends HoodieIndex { private static final Logger LOG = LogManager.getLogger(HoodieBucketIndex.class); @@ -66,14 +64,14 @@ public class HoodieBucketIndex> } @Override - public HoodieData> tagLocation(HoodieData> records, - HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { - HoodieData> taggedRecords = records.mapPartitions(recordIter -> { + HoodieData> taggedRecords = records.mapPartitions(recordIter -> { // partitionPath -> bucketId -> fileInfo Map>> partitionPathFileIDList = new HashMap<>(); - return new LazyIterableIterator, HoodieRecord>(recordIter) { + return new LazyIterableIterator, HoodieRecord>(recordIter) { @Override protected void start() { @@ -81,7 +79,7 @@ public class HoodieBucketIndex> } @Override - protected HoodieRecord computeNext() { + protected HoodieRecord computeNext() { HoodieRecord record = recordIter.next(); int bucketId = BucketIdentifier.getBucketId(record, config.getBucketIndexHashField(), numBuckets); String partitionPath = record.getPartitionPath(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java index bec675c10..42dcc1b97 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java @@ -25,7 +25,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.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.index.HoodieIndex; @@ -41,8 +40,8 @@ import java.util.concurrent.ConcurrentMap; *

* ONLY USE FOR LOCAL TESTING */ -public class HoodieInMemoryHashIndex> - extends HoodieIndex { +public class HoodieInMemoryHashIndex + extends HoodieIndex { private static ConcurrentMap recordLocationMap; @@ -56,13 +55,13 @@ public class HoodieInMemoryHashIndex> } @Override - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) { return records.mapPartitions(hoodieRecordIterator -> { - List> taggedRecords = new ArrayList<>(); + List> taggedRecords = new ArrayList<>(); while (hoodieRecordIterator.hasNext()) { - HoodieRecord record = hoodieRecordIterator.next(); + HoodieRecord record = hoodieRecordIterator.next(); if (recordLocationMap.containsKey(record.getKey())) { record.unseal(); record.setCurrentLocation(recordLocationMap.get(record.getKey())); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java index 8935fcb02..805ae462a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.engine.HoodieEngineContext; 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.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -47,17 +48,15 @@ import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPar /** * A global simple index which reads interested fields(record key and partition path) from base files and * joins with incoming records to find the tagged location. - * - * @param */ -public class HoodieGlobalSimpleIndex> extends HoodieSimpleIndex { +public class HoodieGlobalSimpleIndex extends HoodieSimpleIndex { public HoodieGlobalSimpleIndex(HoodieWriteConfig config, Option keyGeneratorOpt) { super(config, keyGeneratorOpt); } @Override - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) { return tagLocationInternal(records, context, hoodieTable); } @@ -71,11 +70,11 @@ public class HoodieGlobalSimpleIndex> extends H * @return {@link HoodieData} of records with record locations set */ @Override - protected HoodieData> tagLocationInternal( - HoodieData> inputRecords, HoodieEngineContext context, + protected HoodieData> tagLocationInternal( + HoodieData> inputRecords, HoodieEngineContext context, HoodieTable hoodieTable) { - HoodiePairData> keyedInputRecords = + HoodiePairData> keyedInputRecords = inputRecords.mapToPair(entry -> new ImmutablePair<>(entry.getRecordKey(), entry)); HoodiePairData allRecordLocationsInTable = fetchAllRecordLocations(context, hoodieTable, config.getGlobalSimpleIndexParallelism()); @@ -114,8 +113,8 @@ public class HoodieGlobalSimpleIndex> extends H * @param existingRecords existing records with {@link HoodieRecordLocation}s * @return {@link HoodieData} of {@link HoodieRecord}s with tagged {@link HoodieRecordLocation}s */ - private HoodieData> getTaggedRecords( - HoodiePairData> incomingRecords, + private HoodieData> getTaggedRecords( + HoodiePairData> incomingRecords, HoodiePairData existingRecords) { HoodiePairData> existingRecordByRecordKey = existingRecords.mapToPair( @@ -124,29 +123,29 @@ public class HoodieGlobalSimpleIndex> extends H return incomingRecords.leftOuterJoin(existingRecordByRecordKey).values() .flatMap(entry -> { - HoodieRecord inputRecord = entry.getLeft(); + HoodieRecord inputRecord = entry.getLeft(); Option> partitionPathLocationPair = Option.ofNullable(entry.getRight().orElse(null)); - List> taggedRecords; + List> taggedRecords; if (partitionPathLocationPair.isPresent()) { String partitionPath = partitionPathLocationPair.get().getKey(); HoodieRecordLocation location = partitionPathLocationPair.get().getRight(); if (config.getGlobalSimpleIndexUpdatePartitionPath() && !(inputRecord.getPartitionPath().equals(partitionPath))) { // Create an empty record to delete the record in the old partition - HoodieRecord deleteRecord = new HoodieRecord(new HoodieKey(inputRecord.getRecordKey(), partitionPath), new EmptyHoodieRecordPayload()); + HoodieRecord deleteRecord = new HoodieAvroRecord(new HoodieKey(inputRecord.getRecordKey(), partitionPath), new EmptyHoodieRecordPayload()); deleteRecord.setCurrentLocation(location); deleteRecord.seal(); // Tag the incoming record for inserting to the new partition - HoodieRecord insertRecord = (HoodieRecord) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty()); + HoodieRecord insertRecord = (HoodieRecord) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty()); taggedRecords = Arrays.asList(deleteRecord, insertRecord); } else { // Ignore the incoming record's partition, regardless of whether it differs from its old partition or not. // When it differs, the record will still be updated at its old partition. - HoodieRecord newRecord = new HoodieRecord<>(new HoodieKey(inputRecord.getRecordKey(), partitionPath), inputRecord.getData()); - taggedRecords = Collections.singletonList((HoodieRecord) HoodieIndexUtils.getTaggedRecord(newRecord, Option.ofNullable(location))); + HoodieRecord newRecord = new HoodieAvroRecord(new HoodieKey(inputRecord.getRecordKey(), partitionPath), (HoodieRecordPayload) inputRecord.getData()); + taggedRecords = Collections.singletonList((HoodieRecord) HoodieIndexUtils.getTaggedRecord(newRecord, Option.ofNullable(location))); } } else { - taggedRecords = Collections.singletonList((HoodieRecord) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty())); + taggedRecords = Collections.singletonList((HoodieRecord) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty())); } return taggedRecords.iterator(); }); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java index dfefe5ada..8ff3322a9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java @@ -28,7 +28,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.HoodieRecordLocation; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; @@ -47,11 +46,9 @@ import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPar /** * A simple index which reads interested fields(record key and partition path) from base files and * joins with incoming records to find the tagged location. - * - * @param type of {@link HoodieRecordPayload} */ -public class HoodieSimpleIndex> - extends HoodieIndex { +public class HoodieSimpleIndex + extends HoodieIndex { private final Option keyGeneratorOpt; @@ -88,8 +85,8 @@ public class HoodieSimpleIndex> } @Override - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) { return tagLocationInternal(records, context, hoodieTable); } @@ -102,23 +99,23 @@ public class HoodieSimpleIndex> * @param hoodieTable instance of {@link HoodieTable} to use * @return {@link HoodieData} of records with record locations set */ - protected HoodieData> tagLocationInternal( - HoodieData> inputRecords, HoodieEngineContext context, + protected HoodieData> tagLocationInternal( + HoodieData> inputRecords, HoodieEngineContext context, HoodieTable hoodieTable) { if (config.getSimpleIndexUseCaching()) { inputRecords.persist(new HoodieConfig(config.getProps()) .getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE)); } - HoodiePairData> keyedInputRecords = + HoodiePairData> keyedInputRecords = inputRecords.mapToPair(record -> new ImmutablePair<>(record.getKey(), record)); HoodiePairData existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecords.keys(), context, hoodieTable, config.getSimpleIndexParallelism()); - HoodieData> taggedRecords = + HoodieData> taggedRecords = keyedInputRecords.leftOuterJoin(existingLocationsOnTable).map(entry -> { - final HoodieRecord untaggedRecord = entry.getRight().getLeft(); + final HoodieRecord untaggedRecord = entry.getRight().getLeft(); final Option location = Option.ofNullable(entry.getRight().getRight().orElse(null)); return HoodieIndexUtils.getTaggedRecord(untaggedRecord, location); }); 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 3ce957339..7eafe268b 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 @@ -393,7 +393,7 @@ public class HoodieAppendHandle extends @Override public void write(HoodieRecord record, Option insertValue) { - Option> recordMetadata = record.getData().getMetadata(); + Option> recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata(); try { init(record); flushToDiskIfRequired(record); 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 a9ff1f854..096c257b1 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 @@ -18,7 +18,6 @@ package org.apache.hudi.io; -import org.apache.avro.Schema; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; @@ -37,6 +36,7 @@ import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.io.storage.HoodieFileWriterFactory; 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; @@ -128,7 +128,7 @@ public class HoodieCreateHandle extends */ @Override public void write(HoodieRecord record, Option avroRecord) { - Option recordMetadata = record.getData().getMetadata(); + Option recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata(); if (HoodieOperation.isDelete(record.getOperation())) { avroRecord = Option.empty(); } 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 87a8d133f..32d4ec2a6 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 @@ -324,7 +324,7 @@ public class HoodieMergeHandle extends H 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 = new HoodieRecord<>(keyToNewRecords.get(key)); + HoodieRecord hoodieRecord = keyToNewRecords.get(key).newInstance(); try { Option combinedAvroRecord = hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, 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 533611df2..4111abfae 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 @@ -85,7 +85,7 @@ public class HoodieSortedMergeHandle ext } // This is a new insert - HoodieRecord hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(keyToPreWrite)); + HoodieRecord hoodieRecord = keyToNewRecords.get(keyToPreWrite).newInstance(); if (writtenRecordKeys.contains(keyToPreWrite)) { throw new HoodieUpsertException("Insert/Update not in sorted order"); } 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 2afedf2f2..28e88e16a 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 @@ -210,7 +210,7 @@ public abstract class HoodieWriteHandle * Perform the actual writing of the given record into the backing file. */ public void write(HoodieRecord record, Option avroRecord, Option exception) { - Option recordMetadata = record.getData().getMetadata(); + 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); 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 d6403240b..8c958e989 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 @@ -102,7 +102,7 @@ public abstract class HoodieTable implem protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; - protected final HoodieIndex index; + protected final HoodieIndex index; private SerializableConfiguration hadoopConfiguration; protected final TaskContextSupplier taskContextSupplier; private final HoodieTableMetadata metadata; @@ -128,7 +128,7 @@ public abstract class HoodieTable implem this.taskContextSupplier = context.getTaskContextSupplier(); } - protected abstract HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context); + protected abstract HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context); protected HoodieStorageLayout getStorageLayout(HoodieWriteConfig config) { return HoodieLayoutFactory.createLayout(config); @@ -367,7 +367,7 @@ public abstract class HoodieTable implem /** * Return the index. */ - public HoodieIndex getIndex() { + public HoodieIndex getIndex() { return index; } 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 4d3cd479d..8966a5d51 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 @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.bootstrap; 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; @@ -39,7 +40,8 @@ public class BootstrapRecordConsumer extends BoundedInMemoryQueueConsumer } public abstract I deduplicateRecords( - I records, HoodieIndex index, int parallelism); + I records, HoodieIndex index, int parallelism); } 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 190ebcbdb..fd25d92cb 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 @@ -23,6 +23,7 @@ import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -122,7 +123,7 @@ public class TestHoodieHFileReaderWriter { record.put("time", Integer.toString(RANDOM.nextInt())); record.put("number", i); if (testAvroWithMeta) { - writer.writeAvroWithMetadata(record, new HoodieRecord(new HoodieKey((String) record.get("_row_key"), + writer.writeAvroWithMetadata(record, new HoodieAvroRecord(new HoodieKey((String) record.get("_row_key"), Integer.toString((Integer) record.get("number"))), new EmptyHoodieRecordPayload())); // payload does not matter. GenericRecord passed in is what matters // only HoodieKey will be looked up from the 2nd arg(HoodieRecord). } else { @@ -170,4 +171,4 @@ public class TestHoodieHFileReaderWriter { } return rowKeys; } -} \ No newline at end of file +} 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 8ef94629b..3488a1365 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 @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; 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.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; @@ -41,6 +42,7 @@ 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.HoodieParquetWriter; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -48,7 +50,6 @@ 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.metadata.HoodieTableMetadataWriter; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.orc.CompressionKind; @@ -118,7 +119,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable { config, schema, contextSupplier, populateMetaFields)) { int seqId = 1; for (HoodieRecord record : records) { - GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); + GenericRecord avroRecord = (GenericRecord) ((HoodieRecordPayload) record.getData()).getInsertValue(schema).get(); if (populateMetaFields) { HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++)); HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName); @@ -141,7 +142,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable { config, schema, contextSupplier)) { int seqId = 1; for (HoodieRecord record : records) { - GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); + GenericRecord avroRecord = (GenericRecord) ((HoodieRecordPayload) record.getData()).getInsertValue(schema).get(); HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++)); HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName); writer.writeAvro(record.getRecordKey(), avroRecord); @@ -175,7 +176,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable { header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); logWriter.appendBlock(new HoodieAvroDataBlock(groupedRecords.stream().map(r -> { try { - GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get(); + GenericRecord val = (GenericRecord) ((HoodieRecordPayload) r.getData()).getInsertValue(schema).get(); HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), ""); return (IndexedRecord) val; } catch (IOException e) { 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 847a2183a..66c1b0779 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 @@ -25,7 +25,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodieList; 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.config.HoodieWriteConfig; @@ -33,12 +32,12 @@ import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.table.HoodieTable; import java.util.List; +import java.util.stream.Collectors; /** * Base flink implementation of {@link HoodieIndex}. - * @param payload type */ -public abstract class FlinkHoodieIndex extends HoodieIndex>, List, List> { +public abstract class FlinkHoodieIndex extends HoodieIndex>, List> { protected FlinkHoodieIndex(HoodieWriteConfig config) { super(config); } @@ -48,21 +47,22 @@ public abstract class FlinkHoodieIndex extends Ho @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract List updateLocation(List writeStatuses, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; + HoodieTable hoodieTable) throws HoodieIndexException; @Override @Deprecated @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract List> tagLocation(List> records, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; + HoodieTable hoodieTable) throws HoodieIndexException; @Override @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { - return HoodieList.of(tagLocation(HoodieList.getList(records), context, hoodieTable)); + List> hoodieRecords = tagLocation(HoodieList.getList(records.map(record -> (HoodieRecord) record)), context, hoodieTable); + return HoodieList.of(hoodieRecords.stream().map(r -> (HoodieRecord) r).collect(Collectors.toList())); } @Override diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndexFactory.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndexFactory.java index a9196ca9a..54110d935 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndexFactory.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndexFactory.java @@ -46,11 +46,11 @@ public final class FlinkHoodieIndexFactory { // TODO more indexes to be added switch (config.getIndexType()) { case INMEMORY: - return new FlinkInMemoryStateIndex<>(context, config); + return new FlinkInMemoryStateIndex(context, config); case BLOOM: - return new HoodieBloomIndex<>(config, ListBasedHoodieBloomIndexHelper.getInstance()); + return new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); case SIMPLE: - return new HoodieSimpleIndex<>(config, Option.empty()); + return new HoodieSimpleIndex(config, Option.empty()); default: throw new HoodieIndexException("Unsupported index type " + config.getIndexType()); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java index aa779c425..af9785edb 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java @@ -22,9 +22,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.common.data.HoodieData; 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.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex; @@ -37,11 +35,8 @@ import java.util.List; /** * Hoodie index implementation backed by flink state. - * - * @param type of payload */ -public class FlinkInMemoryStateIndex> - extends HoodieIndex>, List, List> { +public class FlinkInMemoryStateIndex extends HoodieIndex, List> { private static final Logger LOG = LogManager.getLogger(FlinkInMemoryStateIndex.class); @@ -50,8 +45,8 @@ public class FlinkInMemoryStateIndex> } @Override - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { throw new UnsupportedOperationException("No need to tag location for FlinkInMemoryStateIndex"); } @@ -88,4 +83,4 @@ public class FlinkInMemoryStateIndex> public boolean isImplicitWithStorage() { return true; } -} \ No newline at end of file +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java index 8df050078..8dd0c99ba 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java @@ -22,6 +22,7 @@ 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.EmptyHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.collection.Pair; @@ -93,7 +94,7 @@ public class FlinkDeleteHelper extends } List> dedupedRecords = - dedupedKeys.stream().map(key -> new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); + dedupedKeys.stream().map(key -> new HoodieAvroRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); Instant beginTag = Instant.now(); // perform index look up to get existing location of records List> taggedRecords = HoodieList.getList( 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 3c689539f..d28aafcc4 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,6 +21,7 @@ package org.apache.hudi.table.action.commit; 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; @@ -89,7 +90,7 @@ public class FlinkWriteHelper extends BaseWrit @Override public List> deduplicateRecords( - List> records, HoodieIndex index, int parallelism) { + List> records, HoodieIndex index, int parallelism) { Map>>> keyedRecords = records.stream().map(record -> { // If index used is global, then records are expected to differ in their partitionPath final Object key = record.getKey().getRecordKey(); @@ -107,7 +108,7 @@ public class FlinkWriteHelper extends BaseWrit boolean choosePrev = data1.equals(reducedData); HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey(); HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation(); - HoodieRecord hoodieRecord = new HoodieRecord<>(reducedKey, reducedData, operation); + HoodieRecord hoodieRecord = new HoodieAvroRecord<>(reducedKey, reducedData, operation); // reuse the location from the first record. hoodieRecord.setCurrentLocation(rec1.getCurrentLocation()); return hoodieRecord; diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java index b3e89e5ba..50adabbd5 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.data.HoodieList; import org.apache.hudi.common.data.HoodieMapPair; +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.table.HoodieTableMetaClient; @@ -115,19 +116,19 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { RawTripTestPayload rowChange1 = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record3 = - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record4 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); List partitions = asList("2016/01/21", "2016/04/01", "2015/03/12"); List> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); @@ -212,16 +213,16 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}"; RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); HoodieRecord record3 = - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); HoodieRecord record4 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); // We write record1, record2 to a base file, but the bloom filter contains (record1, // record2, record3). @@ -286,16 +287,16 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); HoodieRecord record3 = - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); HoodieRecord record4 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); List records = asList(record1, record2, record3, record4); // Also create the metadata and config @@ -354,15 +355,15 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { + "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()); - HoodieRecord record1 = new HoodieRecord(key1, rowChange1); + HoodieRecord record1 = new HoodieAvroRecord(key1, rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()); - HoodieRecord record2 = new HoodieRecord(key2, rowChange2); + HoodieRecord record2 = new HoodieAvroRecord(key2, rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()); - HoodieRecord record4 = new HoodieRecord(key4, rowChange4); + HoodieRecord record4 = new HoodieAvroRecord(key4, rowChange4); List keys = asList(key1, key2, key3, key4); // Also create the metadata and config @@ -373,7 +374,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); List toTagRecords = new ArrayList<>(); - toTagRecords.add(new HoodieRecord(record4.getKey(), null)); + toTagRecords.add(new HoodieAvroRecord(record4.getKey(), null)); List taggedRecords = tagLocation(bloomIndex, toTagRecords, hoodieTable); Map>> recordLocations = new HashMap<>(); for (HoodieRecord taggedRecord : taggedRecords) { @@ -396,7 +397,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { hoodieTable = HoodieFlinkTable.create(config, context, metaClient); List toTagRecords1 = new ArrayList<>(); for (HoodieKey key : keys) { - taggedRecords.add(new HoodieRecord(key, null)); + taggedRecords.add(new HoodieAvroRecord(key, null)); } taggedRecords = tagLocation(bloomIndex, toTagRecords1, hoodieTable); @@ -436,9 +437,9 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { // We write record1 to a base file, using a bloom filter having both records RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); filter.add(record2.getRecordKey()); 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 b33bbd2dd..de9552085 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 @@ -27,6 +27,7 @@ import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.model.HoodieLogFile; 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.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; @@ -132,7 +133,7 @@ public class HoodieFlinkWriteableTestTable extends HoodieWriteableTestTable { header.put(HeaderMetadataType.SCHEMA, schema.toString()); logWriter.appendBlock(new HoodieAvroDataBlock(groupedRecords.stream().map(r -> { try { - GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get(); + GenericRecord val = (GenericRecord) ((HoodieRecordPayload) r.getData()).getInsertValue(schema).get(); HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), ""); return (IndexedRecord) val; } catch (IOException e) { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java index a7dc4a3c0..7e67b087f 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 @@ -26,6 +26,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.JavaTaskContextSupplier; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -237,7 +238,7 @@ public abstract class JavaExecutionStrategy> HoodieKey hoodieKey = new HoodieKey(key, partition); HoodieRecordPayload avroPayload = new RewriteAvroPayload(record); - HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, avroPayload); + HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, avroPayload); return hoodieRecord; } } 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 7f8b83f5c..dd64859ca 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 @@ -25,7 +25,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodieList; 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.config.HoodieWriteConfig; @@ -33,8 +32,9 @@ import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.table.HoodieTable; import java.util.List; +import java.util.stream.Collectors; -public abstract class JavaHoodieIndex extends HoodieIndex>, List, List> { +public abstract class JavaHoodieIndex extends HoodieIndex>, List> { protected JavaHoodieIndex(HoodieWriteConfig config) { super(config); } @@ -44,21 +44,22 @@ public abstract class JavaHoodieIndex extends Hoo @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract List updateLocation(List writeStatuses, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; + HoodieTable hoodieTable) throws HoodieIndexException; @Override @Deprecated @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract List> tagLocation(List> records, HoodieEngineContext context, - HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; + HoodieTable hoodieTable) throws HoodieIndexException; @Override @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { - return HoodieList.of(tagLocation(HoodieList.getList(records), context, hoodieTable)); + List> hoodieRecords = tagLocation(HoodieList.getList(records.map(record -> (HoodieRecord) record)), context, hoodieTable); + return HoodieList.of(hoodieRecords.stream().map(r -> (HoodieRecord) r).collect(Collectors.toList())); } @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndexFactory.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndexFactory.java index f6135fb13..9f4adad8e 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndexFactory.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndexFactory.java @@ -44,9 +44,9 @@ public final class JavaHoodieIndexFactory { // TODO more indexes to be added switch (config.getIndexType()) { case INMEMORY: - return new HoodieInMemoryHashIndex<>(config); + return new HoodieInMemoryHashIndex(config); case BLOOM: - return new HoodieBloomIndex<>(config, ListBasedHoodieBloomIndexHelper.getInstance()); + return new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); default: throw new HoodieIndexException("Unsupported index type " + config.getIndexType()); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java index c75d829cf..f82c1c561 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java @@ -22,6 +22,7 @@ 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.EmptyHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.collection.Pair; @@ -95,7 +96,7 @@ public class JavaDeleteHelper extends } List> dedupedRecords = - dedupedKeys.stream().map(key -> new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); + dedupedKeys.stream().map(key -> new HoodieAvroRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); Instant beginTag = Instant.now(); // perform index look up to get existing location of records List> taggedRecords = HoodieList.getList( 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 6d420e373..3a1fa4b88 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,6 +21,7 @@ package org.apache.hudi.table.action.commit; 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; @@ -55,7 +56,7 @@ public class JavaWriteHelper extends BaseWriteH @Override public List> deduplicateRecords( - List> records, HoodieIndex index, int parallelism) { + List> records, HoodieIndex index, int parallelism) { boolean isIndexingGlobal = index.isGlobal(); Map>>> keyedRecords = records.stream().map(record -> { HoodieKey hoodieKey = record.getKey(); @@ -70,7 +71,7 @@ public class JavaWriteHelper extends BaseWriteH // 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 HoodieRecord(rec1.getKey(), reducedData); + return new HoodieAvroRecord(rec1.getKey(), reducedData); }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList()); } } diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestHoodieConcatHandle.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestHoodieConcatHandle.java index 859a8064b..d81b76b0f 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestHoodieConcatHandle.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestHoodieConcatHandle.java @@ -18,16 +18,11 @@ package org.apache.hudi.table.action.commit; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.JobConf; import org.apache.hudi.client.HoodieJavaWriteClient; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -40,6 +35,13 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.testutils.HoodieJavaClientTestBase; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.JobConf; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -111,8 +113,8 @@ public class TestHoodieConcatHandle extends HoodieJavaClientTestBase { List records1 = new ArrayList<>(); RawTripTestPayload insertRow1 = new RawTripTestPayload(insertRecordStr1); RawTripTestPayload insertRow2 = new RawTripTestPayload(insertRecordStr2); - records1.add(new HoodieRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1)); - records1.add(new HoodieRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2)); + records1.add(new HoodieAvroRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1)); + records1.add(new HoodieAvroRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2)); int startInstant = 1; String firstCommitTime = makeNewCommitTime(startInstant++); @@ -140,8 +142,8 @@ public class TestHoodieConcatHandle extends HoodieJavaClientTestBase { insertRow1 = new RawTripTestPayload(insertRecordStr1); insertRow2 = new RawTripTestPayload(insertRecordStr2); // The recordKey of records2 and records1 are the same, but the values of other fields are different - records2.add(new HoodieRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1)); - records2.add(new HoodieRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2)); + records2.add(new HoodieAvroRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1)); + records2.add(new HoodieAvroRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2)); String newCommitTime = makeNewCommitTime(startInstant++); writeClient.startCommitWithTime(newCommitTime); diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index 63e41cc72..793b26703 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -23,6 +23,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -42,6 +43,8 @@ import org.apache.hudi.io.HoodieCreateHandle; import org.apache.hudi.table.HoodieJavaCopyOnWriteTable; import org.apache.hudi.table.HoodieJavaTable; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieJavaClientTestBase; +import org.apache.hudi.testutils.MetadataMergeWriteStatus; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -49,8 +52,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.JobConf; -import org.apache.hudi.testutils.HoodieJavaClientTestBase; -import org.apache.hudi.testutils.MetadataMergeWriteStatus; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroReadSupport; @@ -142,11 +143,11 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase List records = new ArrayList<>(); RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); - records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); - records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); - records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records writeClient.insert(records, firstCommitTime); @@ -175,12 +176,12 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; RawTripTestPayload updateRowChanges1 = new RawTripTestPayload(updateRecordStr1); - HoodieRecord updatedRecord1 = new HoodieRecord( + HoodieRecord updatedRecord1 = new HoodieAvroRecord( new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); HoodieRecord insertedRecord1 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); List updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1); @@ -254,7 +255,7 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase String recordStr = String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}", UUID.randomUUID().toString(), time, i); RawTripTestPayload rowChange = new RawTripTestPayload(recordStr); - records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); } return records; } @@ -280,11 +281,11 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase List records = new ArrayList<>(); RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); - records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); - records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); - records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records BaseJavaCommitActionExecutor actionExecutor = new JavaInsertCommitActionExecutor(context, config, table, @@ -382,7 +383,7 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString() + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}"; RawTripTestPayload rowChange = new RawTripTestPayload(recordStr); - records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); } // Insert new 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 84040f906..8988aa582 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 @@ -18,9 +18,9 @@ package org.apache.hudi.client; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; @@ -39,6 +39,7 @@ import org.apache.hudi.index.SparkHoodieIndexFactory; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; +import org.apache.hadoop.conf.Configuration; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -66,7 +67,7 @@ public class HoodieReadClient> implements Seria * TODO: We need to persist the index type into hoodie.properties and be able to access the index just with a simple * basepath pointing to the table. Until, then just always assume a BloomIndex */ - private final transient HoodieIndex index; + private final transient HoodieIndex index; private HoodieTable>, JavaRDD, JavaRDD> hoodieTable; private transient Option sqlContextOpt; private final transient HoodieSparkEngineContext context; @@ -172,7 +173,7 @@ public class HoodieReadClient> implements Seria */ public JavaPairRDD>> checkExists(JavaRDD hoodieKeys) { return HoodieJavaRDD.getJavaRDD( - index.tagLocation(HoodieJavaRDD.of(hoodieKeys.map(k -> new HoodieRecord<>(k, null))), + index.tagLocation(HoodieJavaRDD.of(hoodieKeys.map(k -> new HoodieAvroRecord<>(k, null))), context, hoodieTable)) .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) 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 d3c92e211..282cc28a3 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 @@ -32,6 +32,7 @@ import org.apache.hudi.client.utils.ConcatenatingIterator; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -290,7 +291,7 @@ public abstract class MultipleSparkJobExecutionStrategy extends HoodieData { } @Override - public void persist(String storageLevel) { - rddData.persist(StorageLevel.fromString(storageLevel)); + public void persist(String cacheConfig) { + rddData.persist(StorageLevel.fromString(cacheConfig)); } @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 c5e110055..219fb0b16 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java @@ -18,12 +18,11 @@ package org.apache.hudi.execution.bulkinsert; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -32,6 +31,9 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.sort.SpaceCurveSortingHelper; import org.apache.hudi.table.BulkInsertPartitioner; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -86,7 +88,7 @@ public class RDDSpatialCurveSortPartitioner String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); HoodieKey hoodieKey = new HoodieKey(key, partition); - HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, new RewriteAvroPayload(record)); + HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record)); return hoodieRecord; }); } 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 62bf5c100..aece86a38 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 @@ -24,7 +24,6 @@ import org.apache.hudi.PublicAPIMethod; 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.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; @@ -36,7 +35,7 @@ import org.apache.spark.api.java.JavaRDD; @SuppressWarnings("checkstyle:LineLength") public abstract class SparkHoodieIndex> - extends HoodieIndex>, JavaRDD, JavaRDD> { + extends HoodieIndex>, JavaRDD> { protected SparkHoodieIndex(HoodieWriteConfig config) { super(config); } @@ -46,21 +45,23 @@ public abstract class SparkHoodieIndex> @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract JavaRDD updateLocation(JavaRDD writeStatusRDD, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException; + HoodieTable hoodieTable) throws HoodieIndexException; @Override @Deprecated @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) public abstract JavaRDD> tagLocation(JavaRDD> records, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException; + HoodieTable hoodieTable) throws HoodieIndexException; @Override @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { - return HoodieJavaRDD.of(tagLocation(HoodieJavaRDD.getJavaRDD(records), context, hoodieTable)); + return HoodieJavaRDD.of(tagLocation( + HoodieJavaRDD.getJavaRDD(records.map(record -> (HoodieRecord) record)), context, hoodieTable) + .map(r -> (HoodieRecord) r)); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java index 69e18714c..d1f40dca4 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java @@ -53,19 +53,19 @@ public final class SparkHoodieIndexFactory { } switch (config.getIndexType()) { case HBASE: - return new SparkHoodieHBaseIndex<>(config); + return new SparkHoodieHBaseIndex(config); case INMEMORY: - return new HoodieInMemoryHashIndex<>(config); + return new HoodieInMemoryHashIndex(config); case BUCKET: return new HoodieBucketIndex(config); case BLOOM: - return new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance()); + return new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); case GLOBAL_BLOOM: - return new HoodieGlobalBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance()); + return new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); case SIMPLE: - return new HoodieSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config)); + return new HoodieSimpleIndex(config, getKeyGeneratorForSimpleIndex(config)); case GLOBAL_SIMPLE: - return new HoodieGlobalSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config)); + return new HoodieGlobalSimpleIndex(config, getKeyGeneratorForSimpleIndex(config)); default: throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java index 04afc4dd1..1659fe016 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.data.HoodiePairData; 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.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaPairRDD; @@ -63,7 +62,7 @@ public class SparkHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper { public HoodiePairData findMatchingFilesForRecordKeys( HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable, HoodiePairData partitionRecordKeyPairs, - HoodieData> fileComparisonPairs, + HoodieData> fileComparisonPairs, Map> partitionToFileInfo, Map recordsPerPartition) { JavaRDD> fileComparisonsRDD = diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java index e940c0b82..fc73a0aed 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java @@ -24,6 +24,7 @@ import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -85,8 +86,7 @@ import scala.Tuple2; /** * Hoodie Index implementation backed by HBase. */ -public class SparkHoodieHBaseIndex> - extends HoodieIndex>, JavaRDD, JavaRDD> { +public class SparkHoodieHBaseIndex extends HoodieIndex { public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME = "spark.executor.instances"; public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME = "spark.dynamicAllocation.enabled"; @@ -203,15 +203,13 @@ public class SparkHoodieHBaseIndex> /** * Function that tags each HoodieRecord with an existing location, if known. */ - private Function2>, Iterator>> locationTagFunction( + private Function2>, Iterator>> locationTagFunction( HoodieTableMetaClient metaClient) { // `multiGetBatchSize` is intended to be a batch per 100ms. To create a rate limiter that measures // operations per second, we need to multiply `multiGetBatchSize` by 10. Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize(); - return (Function2>, Iterator>>) (partitionNum, - hoodieRecordIterator) -> { - + return (partitionNum, hoodieRecordIterator) -> { boolean updatePartitionPath = config.getHbaseIndexUpdatePartitionPath(); RateLimiter limiter = RateLimiter.create(multiGetBatchSize * 10, TimeUnit.SECONDS); // Grab the global HBase connection @@ -220,7 +218,7 @@ public class SparkHoodieHBaseIndex> hbaseConnection = getHBaseConnection(); } } - List> taggedRecords = new ArrayList<>(); + List> taggedRecords = new ArrayList<>(); try (HTable hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName))) { List statements = new ArrayList<>(); List currentBatchOfRecords = new LinkedList<>(); @@ -256,19 +254,19 @@ public class SparkHoodieHBaseIndex> // check whether to do partition change processing if (updatePartitionPath && !partitionPath.equals(currentRecord.getPartitionPath())) { // delete partition old data record - HoodieRecord emptyRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(), partitionPath), + HoodieRecord emptyRecord = new HoodieAvroRecord(new HoodieKey(currentRecord.getRecordKey(), partitionPath), new EmptyHoodieRecordPayload()); emptyRecord.unseal(); emptyRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId)); emptyRecord.seal(); // insert partition new data record - currentRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(), currentRecord.getPartitionPath()), - currentRecord.getData()); + currentRecord = new HoodieAvroRecord(new HoodieKey(currentRecord.getRecordKey(), currentRecord.getPartitionPath()), + (HoodieRecordPayload) currentRecord.getData()); taggedRecords.add(emptyRecord); taggedRecords.add(currentRecord); } else { - currentRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(), partitionPath), - currentRecord.getData()); + currentRecord = new HoodieAvroRecord(new HoodieKey(currentRecord.getRecordKey(), partitionPath), + (HoodieRecordPayload) currentRecord.getData()); currentRecord.unseal(); currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId)); currentRecord.seal(); @@ -294,8 +292,8 @@ public class SparkHoodieHBaseIndex> } @Override - public HoodieData> tagLocation( - HoodieData> records, HoodieEngineContext context, + public HoodieData> tagLocation( + HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) { return HoodieJavaRDD.of(HoodieJavaRDD.getJavaRDD(records) .mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true)); @@ -303,7 +301,7 @@ public class SparkHoodieHBaseIndex> private Function2, Iterator> updateLocationFunction() { - return (Function2, Iterator>) (partition, statusIterator) -> { + return (partition, statusIterator) -> { List writeStatusList = new ArrayList<>(); // Grab the global HBase connection @@ -385,7 +383,7 @@ public class SparkHoodieHBaseIndex> mutations.clear(); } - public Map mapFileWithInsertsToUniquePartition(JavaRDD writeStatusRDD) { + Map mapFileWithInsertsToUniquePartition(JavaRDD writeStatusRDD) { final Map fileIdPartitionMap = new HashMap<>(); int partitionIndex = 0; // Map each fileId that has inserts to a unique partition Id. This will be used while @@ -466,7 +464,7 @@ public class SparkHoodieHBaseIndex> } } - public Tuple2 getHBasePutAccessParallelism(final JavaRDD writeStatusRDD) { + Tuple2 getHBasePutAccessParallelism(final JavaRDD writeStatusRDD) { final JavaPairRDD insertOnlyWriteStatusRDD = writeStatusRDD .filter(w -> w.getStat().getNumInserts() > 0).mapToPair(w -> new Tuple2<>(w.getStat().getNumInserts(), 1)); return insertOnlyWriteStatusRDD.fold(new Tuple2<>(0L, 0), (w, c) -> new Tuple2<>(w._1 + c._1, w._2 + c._2)); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java index 9587c5b30..fe6dcc127 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java @@ -21,6 +21,7 @@ package org.apache.hudi.table.action.bootstrap; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.bootstrap.BootstrapRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.AvroOrcUtils; @@ -73,7 +74,7 @@ class OrcBootstrapMetadataHandler extends BaseBootstrapMetadataHandler { GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA); gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey); BootstrapRecordPayload payload = new BootstrapRecordPayload(gr); - HoodieRecord rec = new HoodieRecord(new HoodieKey(recKey, partitionPath), payload); + HoodieRecord rec = new HoodieAvroRecord(new HoodieKey(recKey, partitionPath), payload); return rec; }); wrapper.execute(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java index 058c2d426..2bb9bc4ea 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java @@ -21,6 +21,7 @@ package org.apache.hudi.table.action.bootstrap; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.bootstrap.BootstrapRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.ParquetReaderIterator; @@ -77,7 +78,7 @@ class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler { GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA); gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey); BootstrapRecordPayload payload = new BootstrapRecordPayload(gr); - HoodieRecord rec = new HoodieRecord(new HoodieKey(recKey, partitionPath), payload); + HoodieRecord rec = new HoodieAvroRecord(new HoodieKey(recKey, partitionPath), payload); return rec; }); wrapper.execute(); 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 71da2244d..a75723504 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 @@ -74,7 +74,7 @@ public class SparkBucketIndexPartitioner> exten " Bucket index partitioner should only be used by BucketIndex other than " + table.getIndex().getClass().getSimpleName()); } - this.numBuckets = ((HoodieBucketIndex) table.getIndex()).getNumBuckets(); + this.numBuckets = ((HoodieBucketIndex) table.getIndex()).getNumBuckets(); this.indexKeyField = config.getBucketIndexHashField(); this.totalPartitionPaths = profile.getPartitionPaths().size(); partitionPaths = new ArrayList<>(profile.getPartitionPaths()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java index bae3556df..381c11553 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java @@ -22,6 +22,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -93,7 +94,7 @@ public class SparkDeleteHelper extends } JavaRDD> dedupedRecords = - dedupedKeys.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload())); + dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload())); Instant beginTag = Instant.now(); // perform index loop up to get existing location of records JavaRDD> taggedRecords = HoodieJavaRDD.getJavaRDD( diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java index 489526410..23dceb138 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java @@ -20,6 +20,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; 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; @@ -58,7 +59,7 @@ public class SparkWriteHelper extends BaseWrite @Override public JavaRDD> deduplicateRecords( - JavaRDD> records, HoodieIndex index, int parallelism) { + JavaRDD> records, HoodieIndex index, int parallelism) { boolean isIndexingGlobal = index.isGlobal(); return records.mapToPair(record -> { HoodieKey hoodieKey = record.getKey(); @@ -70,7 +71,7 @@ public class SparkWriteHelper extends BaseWrite T reducedData = (T) rec2.getData().preCombine(rec1.getData()); HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey(); - return new HoodieRecord(reducedKey, reducedData); + return new HoodieAvroRecord(reducedKey, reducedData); }, parallelism).map(Tuple2::_2); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java index 1cd7d6ee9..872a4a421 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java @@ -18,6 +18,7 @@ package org.apache.hudi.client; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; @@ -209,7 +210,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { // since they have been modified in the DAG JavaRDD recordRDD = jsc.parallelize(result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream) - .map(record -> new HoodieRecord(record.getKey(), null)).collect(Collectors.toList())); + .map(record -> new HoodieAvroRecord(record.getKey(), null)).collect(Collectors.toList())); // Should have 100 records in table (check using Index), all in locations marked at commit HoodieReadClient readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath()); List taggedRecords = readClient.tagLocation(recordRDD).collect(); @@ -225,7 +226,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase { numRecords, 200, 2); recordRDD = jsc.parallelize(result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream) - .map(record -> new HoodieRecord(record.getKey(), null)).collect(Collectors.toList())); + .map(record -> new HoodieAvroRecord(record.getKey(), null)).collect(Collectors.toList())); // Index should be able to locate all updates in correct locations. readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath()); taggedRecords = readClient.tagLocation(recordRDD).collect(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index dda396a13..b2831419e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -19,6 +19,7 @@ package org.apache.hudi.client; import org.apache.hudi.avro.HoodieAvroUtils; +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.HoodieTableType; @@ -497,9 +498,9 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { HoodieKey key = r.getKey(); GenericRecord payload; try { - payload = (GenericRecord)r.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get(); + payload = (GenericRecord) ((HoodieAvroRecord) r).getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get(); GenericRecord newPayload = HoodieAvroUtils.rewriteRecord(payload, newSchema); - return new HoodieRecord(key, new RawTripTestPayload(newPayload.toString(), key.getRecordKey(), key.getPartitionPath(), schemaStr)); + return new HoodieAvroRecord(key, new RawTripTestPayload(newPayload.toString(), key.getRecordKey(), key.getPartitionPath(), schemaStr)); } catch (IOException e) { throw new RuntimeException("Conversion to new schema failed"); } 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 00e65a67c..70f5e9f3b 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 @@ -19,6 +19,7 @@ package org.apache.hudi.client; import org.apache.hudi.common.fs.FSUtils; +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.HoodieRecordLocation; @@ -82,7 +83,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness { for (String recordStr : recordsStrs) { RawTripTestPayload rowChange = new RawTripTestPayload(recordStr); insertRecords - .add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); + .add(new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); } Map insertRecordMap = insertRecords.stream() .collect(Collectors.toMap(r -> r.getRecordKey(), Function.identity())); @@ -147,7 +148,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness { List updateRecords = new ArrayList<>(); RawTripTestPayload rowChange = new RawTripTestPayload(recordStr); HoodieRecord record = - new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange); + new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange); record.setCurrentLocation(new HoodieRecordLocation("101", insertFileId)); record.seal(); updateRecords.add(record); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index e97a0689a..514007494 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -36,6 +36,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -438,15 +439,15 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { String recordKey = UUID.randomUUID().toString(); HoodieKey keyOne = new HoodieKey(recordKey, "2018-01-01"); HoodieRecord recordOne = - new HoodieRecord(keyOne, dataGen.generateRandomValue(keyOne, newCommitTime)); + new HoodieAvroRecord(keyOne, dataGen.generateRandomValue(keyOne, newCommitTime)); HoodieKey keyTwo = new HoodieKey(recordKey, "2018-02-01"); HoodieRecord recordTwo = - new HoodieRecord(keyTwo, dataGen.generateRandomValue(keyTwo, newCommitTime)); + new HoodieAvroRecord(keyTwo, dataGen.generateRandomValue(keyTwo, newCommitTime)); // Same key and partition as keyTwo HoodieRecord recordThree = - new HoodieRecord(keyTwo, dataGen.generateRandomValue(keyTwo, newCommitTime)); + new HoodieAvroRecord(keyTwo, dataGen.generateRandomValue(keyTwo, newCommitTime)); JavaRDD> records = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1); @@ -973,8 +974,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { throw new IllegalStateException("Unknown partition path " + rec.getPartitionPath()); } recordsToUpsert.add( - new HoodieRecord(new HoodieKey(rec.getRecordKey(), newPartitionPath), - rec.getData())); + new HoodieAvroRecord(new HoodieKey(rec.getRecordKey(), newPartitionPath), + (HoodieRecordPayload) rec.getData())); // populate expected partition path and record keys expectedPartitionPathRecKeyPairs.add(Pair.of(newPartitionPath, rec.getRecordKey())); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index 6cd25f399..154ce5294 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -22,6 +22,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +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.HoodieTableType; @@ -309,16 +310,16 @@ public class TestHoodieIndex extends HoodieClientTestHarness { String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); HoodieRecord record3 = - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); HoodieRecord record4 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); @@ -405,7 +406,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { RawTripTestPayload originalPayload = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord originalRecord = - new HoodieRecord(new HoodieKey(originalPayload.getRowKey(), originalPayload.getPartitionPath()), + new HoodieAvroRecord(new HoodieKey(originalPayload.getRowKey(), originalPayload.getPartitionPath()), originalPayload); /* @@ -418,7 +419,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { RawTripTestPayload incomingPayload = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-02-28T03:16:41.415Z\",\"number\":12}"); HoodieRecord incomingRecord = - new HoodieRecord(new HoodieKey(incomingPayload.getRowKey(), incomingPayload.getPartitionPath()), + new HoodieAvroRecord(new HoodieKey(incomingPayload.getRowKey(), incomingPayload.getPartitionPath()), incomingPayload); /* This record has the same record key as originalRecord and the same partition @@ -428,7 +429,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { RawTripTestPayload incomingPayloadSamePartition = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T04:16:41.415Z\",\"number\":15}"); HoodieRecord incomingRecordSamePartition = - new HoodieRecord( + new HoodieAvroRecord( new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()), incomingPayloadSamePartition); @@ -487,7 +488,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness { private JavaPairRDD>> getRecordLocations(JavaRDD keyRDD, HoodieTable hoodieTable) { JavaRDD recordRDD = tagLocation( - index, keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), hoodieTable); + index, keyRDD.map(k -> new HoodieAvroRecord(k, new EmptyHoodieRecordPayload())), hoodieTable); return recordRDD.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) : Option.empty()) 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 c30635bb1..4707a6807 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 @@ -18,6 +18,7 @@ package org.apache.hudi.execution; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -82,7 +83,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { public void testRecordReading() throws Exception { final int numRecords = 128; final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); - final BoundedInMemoryQueue> queue = + final BoundedInMemoryQueue queue = new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce Future resFuture = executorService.submit(() -> { @@ -93,7 +94,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { final Iterator originalRecordIterator = hoodieRecords.iterator(); int recordsRead = 0; while (queue.iterator().hasNext()) { - final HoodieRecord originalRecord = originalRecordIterator.next(); + final HoodieAvroRecord originalRecord = (HoodieAvroRecord) originalRecordIterator.next(); final Option originalInsertValue = originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA); final HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = queue.iterator().next(); @@ -101,7 +102,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { assertEquals(originalRecord, payload.record); // cached insert value matches the expected insert value. assertEquals(originalInsertValue, - payload.record.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA)); + ((HoodieAvroRecord) payload.record).getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA)); recordsRead++; } assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext()); @@ -122,7 +123,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { final int numProducers = 40; final List> recs = new ArrayList<>(); - final BoundedInMemoryQueue> queue = + final BoundedInMemoryQueue queue = new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Record Key to @@ -188,7 +189,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { // Read recs and ensure we have covered all producer recs. while (queue.iterator().hasNext()) { - final HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = queue.iterator().next(); + final HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = queue.iterator().next(); final HoodieRecord rec = payload.record; Tuple2 producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); Integer lastSeenPos = lastSeenMap.get(producerPos._1()); @@ -216,12 +217,12 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); // maximum number of records to keep in memory. final int recordLimit = 5; - final SizeEstimator> sizeEstimator = new DefaultSizeEstimator<>(); - HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0)); + final SizeEstimator sizeEstimator = new DefaultSizeEstimator<>(); + HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply((HoodieAvroRecord) hoodieRecords.get(0)); final long objSize = sizeEstimator.sizeEstimate(payload); final long memoryLimitInBytes = recordLimit * objSize; - final BoundedInMemoryQueue> queue = + final BoundedInMemoryQueue queue = new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce @@ -266,8 +267,8 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); final SizeEstimator>> sizeEstimator = new DefaultSizeEstimator<>(); // queue memory limit - HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0)); + HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply((HoodieAvroRecord) hoodieRecords.get(0)); final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue)); final long memoryLimitInBytes = 4 * objSize; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/functional/SparkClientFunctionalTestSuite.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/functional/SparkClientFunctionalTestSuite.java index ee7427866..5b20a51f5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/functional/SparkClientFunctionalTestSuite.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/functional/SparkClientFunctionalTestSuite.java @@ -25,7 +25,10 @@ import org.junit.platform.suite.api.SelectPackages; import org.junit.runner.RunWith; @RunWith(JUnitPlatform.class) -@SelectPackages({"org.apache.hudi.client.functional", "org.apache.hudi.table.functional"}) +@SelectPackages({ + "org.apache.hudi.client.functional", + "org.apache.hudi.table.functional", + "org.apache.hudi.index.hbase"}) @IncludeTags("functional") public class SparkClientFunctionalTestSuite { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java index 665e3a6a8..171403eb0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java @@ -19,16 +19,10 @@ package org.apache.hudi.index; -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.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieHBaseIndexConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.index.bloom.HoodieBloomIndex; import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex; @@ -36,9 +30,7 @@ import org.apache.hudi.index.bucket.HoodieBucketIndex; import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex; import org.apache.hudi.index.simple.HoodieSimpleIndex; -import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -62,7 +54,7 @@ public class TestHoodieIndexConfigs { @ParameterizedTest @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE", "BUCKET"}) - public void testCreateIndex(IndexType indexType) throws Exception { + public void testCreateIndex(IndexType indexType) { HoodieWriteConfig config; HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); @@ -104,15 +96,6 @@ public class TestHoodieIndexConfigs { } } - @Test - public void testCreateDummyIndex() { - HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); - HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); - HoodieWriteConfig config = clientConfigBuilder.withPath(basePath) - .withIndexConfig(indexConfigBuilder.withIndexClass(DummyHoodieIndex.class.getName()).build()).build(); - assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof DummyHoodieIndex); - } - @Test public void testCreateIndexWithException() { HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); @@ -132,47 +115,6 @@ public class TestHoodieIndexConfigs { assertTrue(thrown2.getMessage().contains("Unable to instantiate class")); } - public static class DummyHoodieIndex> extends SparkHoodieIndex { - - public DummyHoodieIndex(HoodieWriteConfig config) { - super(config); - } - - @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException { - return null; - } - - @Override - public JavaRDD> tagLocation(JavaRDD> records, - HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) throws HoodieIndexException { - return null; - } - - @Override - public boolean rollbackCommit(String instantTime) { - return false; - } - - @Override - public boolean isGlobal() { - return false; - } - - @Override - public boolean canIndexLogFiles() { - return false; - } - - @Override - public boolean isImplicitWithStorage() { - return false; - } - } - public static class IndexWithConstructor { public IndexWithConstructor(HoodieWriteConfig config) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java index b9063a781..e61d6057c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java @@ -25,6 +25,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.config.HoodieMetadataConfig; +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.table.HoodieTableMetaClient; @@ -122,22 +123,22 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase { RawTripTestPayload rowChange1 = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record3 = - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record4 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); - List> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); + List> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable); // Still 0, as no valid commit assertEquals(0, filesList.size()); @@ -213,16 +214,16 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase { + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}"; RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); HoodieRecord record3 = - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); HoodieRecord record4 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); // We write record1, record2 to a parquet file, but the bloom filter contains (record1, // record2, record3). @@ -289,16 +290,16 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase { String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); HoodieRecord record3 = - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); HoodieRecord record4 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); // Also create the metadata and config @@ -355,15 +356,15 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase { + "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()); - HoodieRecord record1 = new HoodieRecord(key1, rowChange1); + HoodieRecord record1 = new HoodieAvroRecord(key1, rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()); - HoodieRecord record2 = new HoodieRecord(key2, rowChange2); + HoodieRecord record2 = new HoodieAvroRecord(key2, rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()); - HoodieRecord record4 = new HoodieRecord(key4, rowChange4); + HoodieRecord record4 = new HoodieAvroRecord(key4, rowChange4); JavaRDD keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4)); // Also create the metadata and config @@ -374,7 +375,7 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase { // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); JavaRDD taggedRecords = tagLocation( - bloomIndex, keysRDD.map(k -> new HoodieRecord(k, null)), hoodieTable); + bloomIndex, keysRDD.map(k -> new HoodieAvroRecord(k, null)), hoodieTable); JavaPairRDD>> recordLocationsRDD = taggedRecords .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) @@ -394,7 +395,7 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase { // We do the tag again metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); - taggedRecords = tagLocation(bloomIndex, keysRDD.map(k -> new HoodieRecord(k, null)), hoodieTable); + taggedRecords = tagLocation(bloomIndex, keysRDD.map(k -> new HoodieAvroRecord(k, null)), hoodieTable); recordLocationsRDD = taggedRecords .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown() ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId())) @@ -431,10 +432,10 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase { // We write record1 to a parquet file, using a bloom filter having both records RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java index 3f473ba17..9d25907b4 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java @@ -19,10 +19,10 @@ package org.apache.hudi.index.bloom; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +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.testutils.RawTripTestPayload; -import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -92,19 +92,19 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { RawTripTestPayload rowChange1 = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record3 = - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record4 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); // intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up List partitions = Arrays.asList("2016/01/21", "2016/04/01"); @@ -158,7 +158,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/21", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/23", "004"))).mapToPair(t -> t); - List> comparisonKeyList = HoodieJavaRDD.getJavaRDD( + List> comparisonKeyList = HoodieJavaRDD.getJavaRDD( index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieJavaPairRDD.of(partitionRecordKeyPairRDD))).collect(); @@ -200,28 +200,28 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { RawTripTestPayload rowChange1 = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record2 = - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record3 = - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); // this record will be saved in table and will be tagged to the incoming record5 RawTripTestPayload rowChange4 = new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record4 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); // this has the same record key as record4 but different time so different partition, but globalbloomIndex should // tag the original partition of the saved record4 RawTripTestPayload rowChange5 = new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-02-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord record5 = - new HoodieRecord(new HoodieKey(rowChange5.getRowKey(), rowChange5.getPartitionPath()), rowChange5); + new HoodieAvroRecord(new HoodieKey(rowChange5.getRowKey(), rowChange5.getPartitionPath()), rowChange5); JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5)); @@ -281,7 +281,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { RawTripTestPayload originalPayload = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); HoodieRecord originalRecord = - new HoodieRecord(new HoodieKey(originalPayload.getRowKey(), originalPayload.getPartitionPath()), + new HoodieAvroRecord(new HoodieKey(originalPayload.getRowKey(), originalPayload.getPartitionPath()), originalPayload); /* @@ -294,7 +294,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { RawTripTestPayload incomingPayload = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-02-28T03:16:41.415Z\",\"number\":12}"); HoodieRecord incomingRecord = - new HoodieRecord(new HoodieKey(incomingPayload.getRowKey(), incomingPayload.getPartitionPath()), + new HoodieAvroRecord(new HoodieKey(incomingPayload.getRowKey(), incomingPayload.getPartitionPath()), incomingPayload); /* @@ -305,7 +305,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness { RawTripTestPayload incomingPayloadSamePartition = new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T04:16:41.415Z\",\"number\":15}"); HoodieRecord incomingRecordSamePartition = - new HoodieRecord( + new HoodieAvroRecord( new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()), incomingPayloadSamePartition); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java index 879d99339..4491a74fa 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java @@ -18,16 +18,18 @@ package org.apache.hudi.index.bucket; -import java.util.Arrays; -import java.util.List; - -import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.testutils.KeyGeneratorTestUtilities; + +import org.apache.avro.generic.GenericRecord; import org.junit.jupiter.api.Test; +import java.util.Arrays; +import java.util.List; + public class TestBucketIdentifier { @Test @@ -44,7 +46,7 @@ public class TestBucketIdentifier { String recordKeyField = "_row_key"; String indexKeyField = "_row_key"; GenericRecord record = KeyGeneratorTestUtilities.getRecord(); - HoodieRecord hoodieRecord = new HoodieRecord( + HoodieRecord hoodieRecord = new HoodieAvroRecord( new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField, false), ""), null); int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8); assert bucketId == BucketIdentifier.getBucketId( @@ -56,7 +58,7 @@ public class TestBucketIdentifier { List recordKeyField = Arrays.asList("_row_key","ts_ms"); String indexKeyField = "_row_key"; GenericRecord record = KeyGeneratorTestUtilities.getRecord(); - HoodieRecord hoodieRecord = new HoodieRecord( + HoodieRecord hoodieRecord = new HoodieAvroRecord( new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField, false), ""), null); int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8); assert bucketId == BucketIdentifier.getBucketId( diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java index c79f9aec7..2b3765948 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java @@ -19,8 +19,8 @@ package org.apache.hudi.index.bucket; -import org.apache.avro.Schema; import org.apache.hudi.common.data.HoodieData; +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.testutils.RawTripTestPayload; @@ -34,6 +34,8 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; + +import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -46,8 +48,8 @@ import java.util.Properties; import java.util.UUID; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestHoodieBucketIndex extends HoodieClientTestHarness { @@ -93,23 +95,23 @@ public class TestHoodieBucketIndex extends HoodieClientTestHarness { String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord( + HoodieRecord record1 = new HoodieAvroRecord( new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord( + HoodieRecord record2 = new HoodieAvroRecord( new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); - HoodieRecord record3 = new HoodieRecord( + HoodieRecord record3 = new HoodieAvroRecord( new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); - HoodieRecord record4 = new HoodieRecord( + HoodieRecord record4 = new HoodieAvroRecord( new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); - JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); + JavaRDD> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); HoodieWriteConfig config = makeConfig(); HoodieTable table = HoodieSparkTable.create(config, context, metaClient); HoodieBucketIndex bucketIndex = new HoodieBucketIndex(config); - HoodieData taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table); + HoodieData> taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table); assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(r -> r.isCurrentLocationKnown())); HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(table, SCHEMA); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java similarity index 98% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java index b35fee015..d8a488d38 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java @@ -16,15 +16,17 @@ * limitations under the License. */ -package org.apache.hudi.client.functional; +package org.apache.hudi.index.hbase; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +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.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -37,7 +39,6 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; @@ -93,7 +94,7 @@ import static org.mockito.Mockito.when; */ @TestMethodOrder(MethodOrderer.Alphanumeric.class) @Tag("functional") -public class TestHBaseIndex extends SparkClientFunctionalTestHarness { +public class TestSparkHoodieHBaseIndex extends SparkClientFunctionalTestHarness { private static final String TABLE_NAME = "test_table"; private static HBaseTestingUtility utility; @@ -196,7 +197,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { List oldRecords = new LinkedList(); for (HoodieRecord newRecord: newRecords) { HoodieKey key = new HoodieKey(newRecord.getRecordKey(), oldPartitionPath); - HoodieRecord hoodieRecord = new HoodieRecord(key, newRecord.getData()); + HoodieRecord hoodieRecord = new HoodieAvroRecord(key, (HoodieRecordPayload) newRecord.getData()); oldRecords.add(hoodieRecord); } @@ -230,7 +231,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { assertEquals(numRecords, taggedRecords.stream().filter(record -> !record.getKey().getPartitionPath().equals(oldPartitionPath)).count()); // not allowed path change test - index = new SparkHoodieHBaseIndex<>(getConfig(false, false)); + index = new SparkHoodieHBaseIndex(getConfig(false, false)); List notAllowPathChangeRecords = tagLocation(index, newWriteRecords, hoodieTable).collect(); assertEquals(numRecords, notAllowPathChangeRecords.stream().count()); assertEquals(numRecords, taggedRecords.stream().filter(hoodieRecord -> hoodieRecord.isCurrentLocationKnown() @@ -291,7 +292,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { List oldRecords = new LinkedList(); for (HoodieRecord newRecord: newRecords) { HoodieKey key = new HoodieKey(newRecord.getRecordKey(), oldPartitionPath); - HoodieRecord hoodieRecord = new HoodieRecord(key, newRecord.getData()); + HoodieRecord hoodieRecord = new HoodieAvroRecord(key, (HoodieRecordPayload) newRecord.getData()); oldRecords.add(hoodieRecord); } JavaRDD newWriteRecords = jsc().parallelize(newRecords, 1); @@ -764,7 +765,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness { // is not implemented via HoodieWriteClient JavaRDD deleteWriteStatues = writeStatues.map(w -> { WriteStatus newWriteStatus = new WriteStatus(true, 1.0); - w.getWrittenRecords().forEach(r -> newWriteStatus.markSuccess(new HoodieRecord(r.getKey(), null), Option.empty())); + w.getWrittenRecords().forEach(r -> newWriteStatus.markSuccess(new HoodieAvroRecord(r.getKey(), null), Option.empty())); assertEquals(w.getTotalRecords(), newWriteStatus.getTotalRecords()); newWriteStatus.setStat(new HoodieWriteStat()); return newWriteStatus; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index cba77b0c7..53cd6e5d1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -22,6 +22,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.fs.FSUtils; +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.HoodieTableType; @@ -177,11 +178,11 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { List records = new ArrayList<>(); RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); - records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); - records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); - records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records final HoodieSparkCopyOnWriteTable cowTable = table; @@ -210,12 +211,12 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; RawTripTestPayload updateRowChanges1 = new RawTripTestPayload(updateRecordStr1); - HoodieRecord updatedRecord1 = new HoodieRecord( + HoodieRecord updatedRecord1 = new HoodieAvroRecord( new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); HoodieRecord insertedRecord1 = - new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); List updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1); @@ -290,7 +291,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { String recordStr = String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}", UUID.randomUUID().toString(), time, i); RawTripTestPayload rowChange = new RawTripTestPayload(recordStr); - records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); } return records; } @@ -316,11 +317,11 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { List records = new ArrayList<>(); RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); - records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); - records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); - records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table, @@ -416,7 +417,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString() + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}"; RawTripTestPayload rowChange = new RawTripTestPayload(recordStr); - records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); + records.add(new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); } // Insert new records diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java index 8617c8487..2d852f810 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java @@ -24,7 +24,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; -import org.apache.hudi.index.bloom.HoodieBloomIndex; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -66,7 +66,7 @@ public class TestDeleteHelper { private static final int DELETE_PARALLELISM = 200; @Mock - private HoodieBloomIndex index; + private HoodieIndex index; @Mock private HoodieTable, JavaRDD, JavaRDD> table; @Mock diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java index 454c289db..c28758241 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java @@ -175,7 +175,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); - HoodieIndex index = new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance()); + HoodieIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); JavaRDD updatedTaggedRecordsRDD = tagLocation(index, updatedRecordsRDD, table); writeClient.startCommitWithTime(newCommitTime); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java index 8097741dd..f5f9fb8a1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroup; 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.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -592,7 +593,7 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction Map expectedRecords = new HashMap<>(); inputRecords.forEach(entry -> { try { - expectedRecords.put(entry.getRecordKey(), ((GenericRecord) entry.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get())); + expectedRecords.put(entry.getRecordKey(), (GenericRecord) ((HoodieRecordPayload) entry.getData()).getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get()); } catch (IOException e) { e.printStackTrace(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index ee3c309b3..16fd48af6 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -23,6 +23,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodiePartitionMetadata; @@ -274,7 +275,7 @@ public class HoodieClientTestBase extends HoodieClientTestHarness { final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient); JavaRDD recordsToDelete = jsc.parallelize(records, 1) - .map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload())); + .map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload())); JavaRDD taggedRecords = tagLocation(index, recordsToDelete, table); return taggedRecords.map(record -> record.getKey()).collect(); }; 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 new file mode 100644 index 000000000..9a9bbb2b7 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java @@ -0,0 +1,50 @@ +/* + * 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; + +public class HoodieAvroRecord extends HoodieRecord { + public HoodieAvroRecord(HoodieKey key, T data) { + super(key, data); + } + + public HoodieAvroRecord(HoodieKey key, T data, HoodieOperation operation) { + super(key, data, operation); + } + + public HoodieAvroRecord(HoodieRecord record) { + super(record); + } + + public HoodieAvroRecord() { + } + + @Override + public HoodieRecord newInstance() { + return new HoodieAvroRecord<>(this); + } + + @Override + public T getData() { + if (data == null) { + throw new IllegalStateException("Payload already deflated for record."); + } + return data; + } +} 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 17427781e..f90448e7d 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,21 +18,21 @@ package org.apache.hudi.common.model; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import java.io.Serializable; import java.util.List; +import java.util.Map; import java.util.Objects; -import org.apache.hudi.common.util.collection.Pair; +import java.util.stream.Collectors; +import java.util.stream.IntStream; /** * A Single Record managed by Hoodie. */ -public class HoodieRecord implements Serializable { +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"; @@ -64,7 +64,7 @@ public class HoodieRecord implements Serializable /** * Actual payload of the record. */ - private T data; + protected T data; /** * Current location of record on storage. Filled in by looking up index @@ -110,6 +110,8 @@ public class HoodieRecord implements Serializable public HoodieRecord() { } + public abstract HoodieRecord newInstance(); + public HoodieKey getKey() { return key; } 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 e8ffecedc..2b9761176 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.table.log; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; @@ -384,7 +385,7 @@ public abstract class AbstractHoodieLogRecordReader { * @param partitionName - Partition name * @return HoodieRecord created from the IndexedRecord */ - protected HoodieRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, + protected HoodieAvroRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, final String payloadClassFQN, final String preCombineField, final boolean withOperationField, final Option> simpleKeyGenFields, 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 2e47e695d..d0ab73ab0 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 @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.log; import org.apache.hudi.common.config.HoodieCommonConfig; +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; @@ -144,7 +145,7 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(oldValue); boolean choosePrev = combinedValue.equals(oldValue); HoodieOperation operation = choosePrev ? oldRecord.getOperation() : hoodieRecord.getOperation(); - records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation)); + records.put(key, new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation)); } else { // Put the record as is records.put(key, hoodieRecord); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java index 934b5b5f6..b38eeba3f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.util; import org.apache.hudi.common.fs.SizeAwareDataOutputStream; +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; @@ -144,7 +145,7 @@ public class SpillableMapUtils { Object preCombineVal = getPreCombineVal(record, preCombineField); HoodieOperation operation = withOperationField ? HoodieOperation.fromName(getNullableValAsString(record, HoodieRecord.OPERATION_METADATA_FIELD)) : null; - HoodieRecord hoodieRecord = new HoodieRecord<>(new HoodieKey(recKey, partitionPath), + HoodieRecord hoodieRecord = new HoodieAvroRecord<>(new HoodieKey(recKey, partitionPath), ReflectionUtils.loadPayload(payloadClazz, new Object[]{record, preCombineVal}, GenericRecord.class, Comparable.class), operation); @@ -170,7 +171,7 @@ public class SpillableMapUtils { * Utility method to convert bytes to HoodieRecord using schema and payload class. */ public static R generateEmptyPayload(String recKey, String partitionPath, String payloadClazz) { - HoodieRecord hoodieRecord = new HoodieRecord<>(new HoodieKey(recKey, partitionPath), + HoodieRecord hoodieRecord = new HoodieAvroRecord<>(new HoodieKey(recKey, partitionPath), ReflectionUtils.loadPayload(payloadClazz, new Object[] {Option.empty()}, Option.class)); return (R) hoodieRecord; } 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 301a3e104..fb6ce7d5b 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 @@ -30,6 +30,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; 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.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; @@ -215,7 +216,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { // merge base file record w/ log record if present if (logRecords.containsKey(key) && logRecords.get(key).isPresent()) { HoodieRecordPayload mergedPayload = logRecords.get(key).get().getData().preCombine(hoodieRecord.getData()); - result.add(Pair.of(key, Option.of(new HoodieRecord(hoodieRecord.getKey(), mergedPayload)))); + result.add(Pair.of(key, Option.of(new HoodieAvroRecord(hoodieRecord.getKey(), mergedPayload)))); } else { // only base record result.add(Pair.of(key, Option.of(hoodieRecord))); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java index d85ee5dff..4f616c362 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java @@ -18,31 +18,31 @@ package org.apache.hudi.metadata; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Set; +import org.apache.hudi.common.config.HoodieMetadataConfig; +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.table.HoodieTableConfig; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.table.log.InstantRange; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.SpillableMapUtils; +import org.apache.hudi.common.util.collection.ExternalSpillableMap; +import org.apache.hudi.common.util.collection.Pair; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; - -import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.table.HoodieTableConfig; -import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -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.log.HoodieMergedLogRecordScanner; -import org.apache.hudi.common.table.log.InstantRange; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.ExternalSpillableMap; -import org.apache.hudi.common.util.collection.Pair; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; /** * A {@code HoodieMergedLogRecordScanner} implementation which only merged records matching providing keys. This is @@ -87,7 +87,7 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc } @Override - protected HoodieRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, + protected HoodieAvroRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, final String payloadClassFQN, final String preCombineField, final boolean withOperationField, final Option> simpleKeyGenFields, 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 5f10fa081..650e86146 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 @@ -18,12 +18,13 @@ package org.apache.hudi.metadata; -import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataBloomFilter; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataFileInfo; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -32,9 +33,10 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.hash.ColumnIndexID; 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.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -42,7 +44,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hudi.io.storage.HoodieHFileReader; import java.io.IOException; import java.nio.ByteBuffer; @@ -204,7 +205,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload(key, payload); + return new HoodieAvroRecord<>(key, payload); } /** @@ -225,7 +226,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload(key, payload); + return new HoodieAvroRecord<>(key, payload); } /** @@ -256,7 +257,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload(key, metadataPayload); + return new HoodieAvroRecord<>(key, metadataPayload); } @Override @@ -447,7 +448,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload(key, payload); + return new HoodieAvroRecord<>(key, payload); }); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestParquetInLining.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestParquetInLining.java index c4e728dc2..9ed27c4b2 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestParquetInLining.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestParquetInLining.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.fs.inline; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.FileSystemTestUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -144,7 +145,8 @@ public class TestParquetInLining { List hoodieRecords = dataGenerator.generateInsertsWithHoodieAvroPayload(commitTime, 10); List toReturn = new ArrayList<>(); for (HoodieRecord record : hoodieRecords) { - toReturn.add((GenericRecord) record.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get()); + toReturn.add((GenericRecord) ((HoodieAvroRecord) record).getData() + .getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get()); } return toReturn; } 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 796bc6111..0a5ecc098 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 @@ -18,18 +18,10 @@ package org.apache.hudi.common.functional; -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.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieArchivedLogFile; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; @@ -59,6 +51,16 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.exception.CorruptedLogFileException; import org.apache.hudi.exception.HoodieIOException; + +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.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.hadoop.util.counters.BenchmarkCounter; import org.junit.jupiter.api.AfterAll; @@ -585,7 +587,8 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness { List scannedRecords = new ArrayList<>(); for (HoodieRecord record : scanner) { - scannedRecords.add((IndexedRecord) record.getData().getInsertValue(schema).get()); + scannedRecords.add((IndexedRecord) + ((HoodieAvroRecord) record).getData().getInsertValue(schema).get()); } assertEquals(scannedRecords.size(), allRecords.stream().mapToLong(Collection::size).sum(), diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieRecord.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieRecord.java index e31286d10..b6bbc34cc 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieRecord.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieRecord.java @@ -44,7 +44,7 @@ public class TestHoodieRecord { public void setUp() throws Exception { final List indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1); final List hoodieRecords = - indexedRecords.stream().map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), + indexedRecords.stream().map(r -> new HoodieAvroRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), new AvroBinaryTestPayload(Option.of((GenericRecord) r)))).collect(Collectors.toList()); hoodieRecord = hoodieRecords.get(0); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index 21816a56c..9fcc3695e 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -23,6 +23,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodiePartitionMetadata; @@ -510,7 +511,7 @@ public class HoodieTestDataGenerator implements AutoCloseable { populateKeysBySchema(schemaStr, currSize + i, kp); incrementNumExistingKeysBySchema(schemaStr); try { - return new HoodieRecord(key, generateRandomValueAsPerSchema(schemaStr, key, instantTime, isFlattened)); + return new HoodieAvroRecord(key, generateRandomValueAsPerSchema(schemaStr, key, instantTime, isFlattened)); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } @@ -541,7 +542,7 @@ public class HoodieTestDataGenerator implements AutoCloseable { List copy = new ArrayList<>(); for (HoodieRecord r : origin) { HoodieKey key = r.getKey(); - HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, instantTime)); + HoodieRecord record = new HoodieAvroRecord(key, generateRandomValue(key, instantTime)); copy.add(record); } return copy; @@ -553,7 +554,7 @@ public class HoodieTestDataGenerator implements AutoCloseable { for (int i = 0; i < limit; i++) { String partitionPath = partitionPaths[RAND.nextInt(partitionPaths.length)]; HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath); - HoodieRecord record = new HoodieRecord(key, generateAvroPayload(key, instantTime)); + HoodieRecord record = new HoodieAvroRecord(key, generateAvroPayload(key, instantTime)); inserts.add(record); KeyPartition kp = new KeyPartition(); @@ -568,7 +569,7 @@ public class HoodieTestDataGenerator implements AutoCloseable { public List generateUpdatesWithHoodieAvroPayload(String instantTime, List baseRecords) { List updates = new ArrayList<>(); for (HoodieRecord baseRecord : baseRecords) { - HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateAvroPayload(baseRecord.getKey(), instantTime)); + HoodieRecord record = new HoodieAvroRecord(baseRecord.getKey(), generateAvroPayload(baseRecord.getKey(), instantTime)); updates.add(record); } return updates; @@ -596,11 +597,11 @@ public class HoodieTestDataGenerator implements AutoCloseable { public HoodieRecord generateDeleteRecord(HoodieKey key) throws IOException { RawTripTestPayload payload = new RawTripTestPayload(Option.empty(), key.getRecordKey(), key.getPartitionPath(), null, true, 0L); - return new HoodieRecord(key, payload); + return new HoodieAvroRecord(key, payload); } public HoodieRecord generateUpdateRecord(HoodieKey key, String instantTime) throws IOException { - return new HoodieRecord(key, generateRandomValue(key, instantTime)); + return new HoodieAvroRecord(key, generateRandomValue(key, instantTime)); } public List generateUpdates(String instantTime, List baseRecords) throws IOException { @@ -615,7 +616,7 @@ public class HoodieTestDataGenerator implements AutoCloseable { public List generateUpdatesWithTS(String instantTime, List baseRecords, int ts) throws IOException { List updates = new ArrayList<>(); for (HoodieRecord baseRecord : baseRecords) { - HoodieRecord record = new HoodieRecord(baseRecord.getKey(), + HoodieRecord record = new HoodieAvroRecord(baseRecord.getKey(), generateRandomValue(baseRecord.getKey(), instantTime, false, ts)); updates.add(record); } @@ -735,7 +736,7 @@ public class HoodieTestDataGenerator implements AutoCloseable { logger.debug("key getting updated: " + kp.key.getRecordKey()); used.add(kp); try { - return new HoodieRecord(kp.key, generateRandomValueAsPerSchema(schemaStr, kp.key, instantTime, false)); + return new HoodieAvroRecord(kp.key, generateRandomValueAsPerSchema(schemaStr, kp.key, instantTime, false)); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } @@ -801,7 +802,7 @@ public class HoodieTestDataGenerator implements AutoCloseable { numExistingKeys--; used.add(kp); try { - result.add(new HoodieRecord(kp.key, generateRandomDeleteValue(kp.key, instantTime))); + result.add(new HoodieAvroRecord(kp.key, generateRandomDeleteValue(kp.key, instantTime))); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/SchemaTestUtil.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/SchemaTestUtil.java index ba5a2895c..ab77caa1b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/SchemaTestUtil.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/SchemaTestUtil.java @@ -21,6 +21,7 @@ package org.apache.hudi.common.testutils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.MercifulJsonConverter; import org.apache.hudi.common.model.HoodieAvroPayload; +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.table.timeline.HoodieActiveTimeline; @@ -152,7 +153,7 @@ public final class SchemaTestUtil { } private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String key, String partitionPath) { - return new HoodieRecord<>(new HoodieKey(key, partitionPath), + return new HoodieAvroRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) iRecord))); } @@ -172,7 +173,7 @@ public final class SchemaTestUtil { throws IOException, URISyntaxException { List iRecords = generateTestRecords(from, limit); - return iRecords.stream().map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), + return iRecords.stream().map(r -> new HoodieAvroRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), new HoodieAvroPayload(Option.of((GenericRecord) r)))).collect(Collectors.toList()); } @@ -180,9 +181,9 @@ public final class SchemaTestUtil { Schema schema, String fieldNameToUpdate, String newValue) { return oldRecords.stream().map(r -> { try { - GenericRecord rec = (GenericRecord) r.getData().getInsertValue(schema).get(); + GenericRecord rec = (GenericRecord) ((HoodieAvroRecord) r).getData().getInsertValue(schema).get(); rec.put(fieldNameToUpdate, newValue); - return new HoodieRecord<>(r.getKey(), new HoodieAvroPayload(Option.of(rec))); + return new HoodieAvroRecord<>(r.getKey(), new HoodieAvroPayload(Option.of(rec))); } catch (IOException io) { throw new HoodieIOException("unable to get data from hoodie record", io); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/SpillableMapTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/SpillableMapTestUtils.java index 89155904e..2e450660b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/SpillableMapTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/SpillableMapTestUtils.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.testutils; import org.apache.hudi.common.model.HoodieAvroPayload; +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.HoodieRecordLocation; @@ -48,7 +49,7 @@ public class SpillableMapTestUtils { String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); recordKeys.add(key); HoodieRecord record = - new HoodieRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) r))); + new HoodieAvroRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) r))); record.unseal(); record.setCurrentLocation(new HoodieRecordLocation("DUMMY_COMMIT_TIME", "DUMMY_FILE_ID")); record.seal(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestBitCaskDiskMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestBitCaskDiskMap.java index 208c6d969..9bbe42771 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestBitCaskDiskMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestBitCaskDiskMap.java @@ -20,6 +20,7 @@ package org.apache.hudi.common.util.collection; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieAvroPayload; +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; @@ -186,7 +187,7 @@ public class TestBitCaskDiskMap extends HoodieCommonTestHarness { schema = SchemaTestUtil.getSimpleSchema(); List indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1); hoodieRecords = - indexedRecords.stream().map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), + indexedRecords.stream().map(r -> new HoodieAvroRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), new AvroBinaryTestPayload(Option.of((GenericRecord) r)))).collect(Collectors.toList()); payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0), new HoodieRecordSizeEstimator(schema)); assertTrue(payloadSize > 0); @@ -195,7 +196,7 @@ public class TestBitCaskDiskMap extends HoodieCommonTestHarness { final Schema simpleSchemaWithMetadata = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1); hoodieRecords = indexedRecords.stream() - .map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), + .map(r -> new HoodieAvroRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"), new AvroBinaryTestPayload( Option.of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata))))) .collect(Collectors.toList()); @@ -212,7 +213,7 @@ public class TestBitCaskDiskMap extends HoodieCommonTestHarness { iRecords.forEach(r -> { String key = ((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); - HoodieRecord value = new HoodieRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) r))); + HoodieRecord value = new HoodieAvroRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) r))); recordMap.put(key, value); }); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java index f7b45e9d8..e33baf149 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java @@ -20,6 +20,7 @@ package org.apache.hudi.common.util.collection; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieAvroPayload; +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; @@ -135,7 +136,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { updatedRecords.forEach(record -> { HoodieRecord rec = records.get(((GenericRecord) record).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); try { - assertEquals(rec.getData().getInsertValue(schema).get(), record); + assertEquals(((HoodieAvroRecord) rec).getData().getInsertValue(schema).get(), record); } catch (IOException io) { throw new UncheckedIOException(io); } @@ -159,13 +160,13 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { IndexedRecord inMemoryRecord = iRecords.get(0); String ikey = ((GenericRecord) inMemoryRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String iPartitionPath = ((GenericRecord) inMemoryRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); - HoodieRecord inMemoryHoodieRecord = new HoodieRecord<>(new HoodieKey(ikey, iPartitionPath), + HoodieRecord inMemoryHoodieRecord = new HoodieAvroRecord<>(new HoodieKey(ikey, iPartitionPath), new HoodieAvroPayload(Option.of((GenericRecord) inMemoryRecord))); IndexedRecord onDiskRecord = iRecords.get(99); String dkey = ((GenericRecord) onDiskRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String dPartitionPath = ((GenericRecord) onDiskRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); - HoodieRecord onDiskHoodieRecord = new HoodieRecord<>(new HoodieKey(dkey, dPartitionPath), + HoodieRecord onDiskHoodieRecord = new HoodieAvroRecord<>(new HoodieKey(dkey, dPartitionPath), new HoodieAvroPayload(Option.of((GenericRecord) onDiskRecord))); // assert size assert records.size() == 100; @@ -241,7 +242,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { // Get a record from the in-Memory map String key = recordKeys.get(0); - HoodieRecord record = records.get(key); + HoodieAvroRecord record = (HoodieAvroRecord) records.get(key); List recordsToUpdate = new ArrayList<>(); recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get()); @@ -259,7 +260,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness { // Get a record from the disk based map key = recordKeys.get(recordKeys.size() - 1); - record = records.get(key); + record = (HoodieAvroRecord) records.get(key); recordsToUpdate = new ArrayList<>(); recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbDiskMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbDiskMap.java index 2ae521fc8..31daaab21 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbDiskMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbDiskMap.java @@ -18,12 +18,9 @@ package org.apache.hudi.common.util.collection; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; - import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieAvroPayload; +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; @@ -33,6 +30,9 @@ import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.SpillableMapTestUtils; import org.apache.hudi.common.util.Option; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -166,7 +166,7 @@ public class TestRocksDbDiskMap extends HoodieCommonTestHarness { iRecords.forEach(r -> { String key = ((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); - HoodieRecord value = new HoodieRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) r))); + HoodieRecord value = new HoodieAvroRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) r))); recordMap.put(key, value); }); diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/common/HoodieExampleDataGenerator.java b/hudi-examples/src/main/java/org/apache/hudi/examples/common/HoodieExampleDataGenerator.java index 71c6408cc..78df2e78e 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/common/HoodieExampleDataGenerator.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/common/HoodieExampleDataGenerator.java @@ -20,6 +20,7 @@ package org.apache.hudi.examples.common; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieAvroPayload; +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; @@ -126,7 +127,7 @@ public class HoodieExampleDataGenerator> { kp.partitionPath = partitionPath; existingKeys.put(currSize + i, kp); numExistingKeys++; - return new HoodieRecord<>(key, generateRandomValue(key, commitTime)); + return new HoodieAvroRecord<>(key, generateRandomValue(key, commitTime)); }); } @@ -149,7 +150,7 @@ public class HoodieExampleDataGenerator> { } public HoodieRecord generateUpdateRecord(HoodieKey key, String commitTime) { - return new HoodieRecord<>(key, generateRandomValue(key, commitTime)); + return new HoodieAvroRecord<>(key, generateRandomValue(key, commitTime)); } private Option convertToString(HoodieRecord record) { diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java index 587f73b0f..4890a6529 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java @@ -18,11 +18,11 @@ package org.apache.hudi.examples.java; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.HoodieJavaWriteClient; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroPayload; +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.HoodieTableType; @@ -33,6 +33,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.examples.common.HoodieExampleDataGenerator; import org.apache.hudi.index.HoodieIndex; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -95,7 +96,7 @@ public class HoodieJavaWriteClientExample { List> records = dataGen.generateInserts(newCommitTime, 10); List> recordsSoFar = new ArrayList<>(records); List> writeRecords = - recordsSoFar.stream().map(r -> new HoodieRecord(r)).collect(Collectors.toList()); + recordsSoFar.stream().map(r -> new HoodieAvroRecord(r)).collect(Collectors.toList()); client.insert(writeRecords, newCommitTime); // updates @@ -105,7 +106,7 @@ public class HoodieJavaWriteClientExample { records.addAll(toBeUpdated); recordsSoFar.addAll(toBeUpdated); writeRecords = - recordsSoFar.stream().map(r -> new HoodieRecord(r)).collect(Collectors.toList()); + recordsSoFar.stream().map(r -> new HoodieAvroRecord(r)).collect(Collectors.toList()); client.upsert(writeRecords, newCommitTime); // Delete diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java index 0e7e35e7e..a8234a30d 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java @@ -19,6 +19,7 @@ package org.apache.hudi.sink; import org.apache.hudi.client.WriteStatus; +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; @@ -217,13 +218,13 @@ public class StreamWriteFunction extends AbstractStreamWriteFunction { return new DataItem( record.getRecordKey(), record.getCurrentLocation().getInstantTime(), - record.getData(), + ((HoodieAvroRecord) record).getData(), record.getOperation()); } public HoodieRecord toHoodieRecord(String partitionPath) { HoodieKey hoodieKey = new HoodieKey(this.key, partitionPath); - HoodieRecord record = new HoodieRecord<>(hoodieKey, data, operation); + HoodieRecord record = new HoodieAvroRecord<>(hoodieKey, data, operation); HoodieRecordLocation loc = new HoodieRecordLocation(instant, null); record.setCurrentLocation(loc); return record; @@ -264,7 +265,7 @@ public class StreamWriteFunction extends AbstractStreamWriteFunction { public void preWrite(List records) { // rewrite the first record with expected fileID HoodieRecord first = records.get(0); - HoodieRecord record = new HoodieRecord<>(first.getKey(), first.getData(), first.getOperation()); + HoodieRecord record = new HoodieAvroRecord<>(first.getKey(), (HoodieRecordPayload) first.getData(), first.getOperation()); HoodieRecordLocation newLoc = new HoodieRecordLocation(first.getCurrentLocation().getInstantTime(), fileID); record.setCurrentLocation(newLoc); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java index 4832f18bf..eee9347ba 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java @@ -20,6 +20,7 @@ package org.apache.hudi.sink.bootstrap; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordGlobalLocation; @@ -251,7 +252,7 @@ public class BootstrapOperator> @SuppressWarnings("unchecked") public static HoodieRecord generateHoodieRecord(HoodieKey hoodieKey, FileSlice fileSlice) { - HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, null); + HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, null); hoodieRecord.setCurrentLocation(new HoodieRecordGlobalLocation(hoodieKey.getPartitionPath(), fileSlice.getBaseInstantTime(), fileSlice.getFileId())); hoodieRecord.seal(); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/IndexRecord.java b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/IndexRecord.java index 2fe83b71c..edae0389b 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/IndexRecord.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/IndexRecord.java @@ -18,16 +18,22 @@ package org.apache.hudi.sink.bootstrap; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; /** * The index record. */ -public class IndexRecord extends HoodieRecord { +public class IndexRecord extends HoodieAvroRecord { private static final long serialVersionUID = 1L; public IndexRecord(HoodieRecord record) { super(record); } + + @Override + public HoodieRecord newInstance() { + return new IndexRecord<>(this); + } } \ No newline at end of file diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java index 73fd66855..d01db962c 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java @@ -22,6 +22,7 @@ import org.apache.hudi.client.FlinkTaskContextSupplier; import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.model.BaseAvroPayload; +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.HoodieRecordGlobalLocation; @@ -180,7 +181,7 @@ public class BucketAssignFunction> if (globalIndex) { // if partition path changes, emit a delete record for old partition path, // then update the index state using location with new partition path. - HoodieRecord deleteRecord = new HoodieRecord<>(new HoodieKey(recordKey, oldLoc.getPartitionPath()), + HoodieRecord deleteRecord = new HoodieAvroRecord<>(new HoodieKey(recordKey, oldLoc.getPartitionPath()), payloadCreation.createDeletePayload((BaseAvroPayload) record.getData())); deleteRecord.setCurrentLocation(oldLoc.toLocal("U")); deleteRecord.seal(); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunction.java index b600a5d2f..31adcdf29 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunction.java @@ -18,6 +18,7 @@ package org.apache.hudi.sink.transform; +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; @@ -111,6 +112,6 @@ public class RowDataToHoodieFunction HoodieRecordPayload payload = payloadCreation.createPayload(gr); HoodieOperation operation = HoodieOperation.fromValue(record.getRowKind().toByteValue()); - return new HoodieRecord<>(hoodieKey, payload, operation); + return new HoodieAvroRecord<>(hoodieKey, payload, operation); } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index 7a72bca05..4404e15ea 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.format.mor; +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.table.log.HoodieMergedLogRecordScanner; @@ -340,7 +341,7 @@ public class MergeOnReadInputFormat while (logRecordsKeyIterator.hasNext()) { String curAvroKey = logRecordsKeyIterator.next(); Option curAvroRecord = null; - final HoodieRecord hoodieRecord = scanner.getRecords().get(curAvroKey); + final HoodieAvroRecord hoodieRecord = (HoodieAvroRecord) scanner.getRecords().get(curAvroKey); try { curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema); } catch (IOException e) { @@ -412,7 +413,7 @@ public class MergeOnReadInputFormat public boolean hasNext() { while (recordsIterator.hasNext()) { Option curAvroRecord = null; - final HoodieRecord hoodieRecord = recordsIterator.next(); + final HoodieAvroRecord hoodieRecord = (HoodieAvroRecord) recordsIterator.next(); try { curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema); } catch (IOException e) { @@ -725,7 +726,7 @@ public class MergeOnReadInputFormat } private Option getInsertValue(String curKey) throws IOException { - final HoodieRecord record = scanner.getRecords().get(curKey); + final HoodieAvroRecord record = (HoodieAvroRecord) scanner.getRecords().get(curKey); if (!emitDelete && HoodieOperation.isDelete(record.getOperation())) { return Option.empty(); } @@ -750,7 +751,7 @@ public class MergeOnReadInputFormat private Option mergeRowWithLog( RowData curRow, String curKey) throws IOException { - final HoodieRecord record = scanner.getRecords().get(curKey); + final HoodieAvroRecord record = (HoodieAvroRecord) scanner.getRecords().get(curKey); GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow); return record.getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema); } 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 f00efa5ef..c8f074652 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 @@ -21,6 +21,7 @@ package org.apache.hudi.hadoop.realtime; 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; @@ -96,9 +97,9 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader private Option buildGenericRecordwithCustomPayload(HoodieRecord record) throws IOException { if (usesCustomPayload) { - return record.getData().getInsertValue(getWriterSchema()); + return ((HoodieAvroRecord) record).getData().getInsertValue(getWriterSchema()); } else { - return record.getData().getInsertValue(getReaderSchema()); + return ((HoodieAvroRecord) record).getData().getInsertValue(getReaderSchema()); } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java index 41ef3f4ab..e46343c1a 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java @@ -18,15 +18,12 @@ package org.apache.hudi.integ.testsuite; -import java.io.IOException; -import java.io.Serializable; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -45,6 +42,8 @@ import org.apache.hudi.integ.testsuite.dag.nodes.ScheduleCompactNode; import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -52,6 +51,8 @@ import org.apache.spark.rdd.RDD; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.io.Serializable; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; @@ -128,7 +129,7 @@ public class HoodieTestSuiteWriter implements Serializable { Pair>> nextBatch = fetchSource(); lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); JavaRDD inputRDD = nextBatch.getRight().getRight(); - return inputRDD.map(r -> (GenericRecord) r.getData() + return inputRDD.map(r -> (GenericRecord) ((HoodieAvroRecord) r).getData() .getInsertValue(new Schema.Parser().parse(schema)).get()).rdd(); } diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/AbstractConnectWriter.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/AbstractConnectWriter.java index a579484f6..649150d16 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/AbstractConnectWriter.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/AbstractConnectWriter.java @@ -20,6 +20,7 @@ package org.apache.hudi.connect.writers; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.util.Option; @@ -81,7 +82,7 @@ public abstract class AbstractConnectWriter implements ConnectWriter hoodieRecord = new HoodieRecord<>(keyGenerator.getKey(avroRecord.get()), new HoodieAvroPayload(avroRecord)); + HoodieRecord hoodieRecord = new HoodieAvroRecord<>(keyGenerator.getKey(avroRecord.get()), new HoodieAvroPayload(avroRecord)); String fileId = KafkaConnectUtils.hashDigest(String.format("%s-%s", record.kafkaPartition(), hoodieRecord.getPartitionPath())); hoodieRecord.unseal(); hoodieRecord.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId)); diff --git a/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestAbstractConnectWriter.java b/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestAbstractConnectWriter.java index c8a3ad6ff..7a286e565 100644 --- a/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestAbstractConnectWriter.java +++ b/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestAbstractConnectWriter.java @@ -22,6 +22,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieAvroPayload; +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.testutils.SchemaTestUtil; @@ -168,7 +169,7 @@ public class TestAbstractConnectWriter { } private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String key, String partitionPath) { - return new HoodieRecord<>(new HoodieKey(key, partitionPath), + return new HoodieAvroRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) iRecord))); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index a632b5a4e..92fa99c71 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -18,14 +18,12 @@ package org.apache.hudi; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.HoodieWriteResult; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.TypedProperties; +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; @@ -47,6 +45,10 @@ import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.util.DataTypeUtils; + +import org.apache.avro.generic.GenericRecord; +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.spark.api.java.JavaRDD; @@ -235,13 +237,13 @@ public class DataSourceUtils { public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, HoodieKey hKey, String payloadClass) throws IOException { HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal); - return new HoodieRecord<>(hKey, payload); + return new HoodieAvroRecord<>(hKey, payload); } public static HoodieRecord createHoodieRecord(GenericRecord gr, HoodieKey hKey, String payloadClass) throws IOException { HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr); - return new HoodieRecord<>(hKey, payload); + return new HoodieAvroRecord<>(hKey, payload); } /** diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java index e0929efed..9aa7ac1a6 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java @@ -19,6 +19,7 @@ package org.apache.hudi; import org.apache.hudi.avro.HoodieAvroUtils; +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.OverwriteWithLatestAvroPayload; @@ -149,7 +150,7 @@ public class QuickstartUtils { existingKeys.put(currSize + i, key); numExistingKeys++; try { - return new HoodieRecord(key, generateRandomValue(key, randomString)); + return new HoodieAvroRecord(key, generateRandomValue(key, randomString)); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } @@ -165,7 +166,7 @@ public class QuickstartUtils { } public HoodieRecord generateUpdateRecord(HoodieKey key, String randomString) throws IOException { - return new HoodieRecord(key, generateRandomValue(key, randomString)); + return new HoodieAvroRecord(key, generateRandomValue(key, randomString)); } /** diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java index 2e89baa70..9b7d55347 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -510,7 +511,7 @@ public class TestBootstrap extends HoodieClientTestBase { try { String key = gr.get("_row_key").toString(); String pPath = p.getKey(); - return new HoodieRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath, + return new HoodieAvroRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java index fba09091a..91eab6ddd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.bootstrap.FileStatusUtils; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -425,7 +426,7 @@ public class TestOrcBootstrap extends HoodieClientTestBase { try { String key = gr.get("_row_key").toString(); String pPath = p.getKey(); - return new HoodieRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath, + return new HoodieAvroRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java index 8651e30c0..b5d7dc4b1 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.HoodieJsonPayload; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; +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; @@ -197,7 +198,7 @@ public class HDFSParquetImporter implements Serializable { LOG.warn("Unable to parse date from partition field. Assuming partition as (" + partitionField + ")"); } } - return new HoodieRecord<>(new HoodieKey(rowField.toString(), partitionPath), + return new HoodieAvroRecord<>(new HoodieKey(rowField.toString(), partitionPath), new HoodieJsonPayload(genericRecord.toString())); }); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 108cd78a7..828980506 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -29,6 +29,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper; import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -451,7 +452,7 @@ public class DeltaSync implements Serializable { KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())))) : DataSourceUtils.createPayload(cfg.payloadClassName, gr); - return new HoodieRecord<>(keyGenerator.getKey(gr), payload); + return new HoodieAvroRecord<>(keyGenerator.getKey(gr), payload); }); return Pair.of(schemaProvider, Pair.of(checkpointStr, records)); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/JdbcTestUtils.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/JdbcTestUtils.java index 377063eb0..79173dbdc 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/JdbcTestUtils.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/JdbcTestUtils.java @@ -20,6 +20,7 @@ package org.apache.hudi.utilities.testutils; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -81,7 +82,8 @@ public class JdbcTestUtils { .stream() .map(r -> { try { - return ((GenericRecord) r.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, props).get()); + return ((GenericRecord) ((HoodieAvroRecord) r).getData() + .getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, props).get()); } catch (IOException e) { return null; } @@ -125,7 +127,7 @@ public class JdbcTestUtils { List updateRecords = dataGenerator.generateUpdates(commitTime, inserts); updateRecords.stream().map(m -> { try { - return m.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, props).get(); + return ((HoodieAvroRecord) m).getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, props).get(); } catch (IOException e) { return null; } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index 90a3f5af3..8464740bf 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -20,6 +20,7 @@ package org.apache.hudi.utilities.testutils; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -378,7 +379,7 @@ public class UtilitiesTestBase { public static GenericRecord toGenericRecord(HoodieRecord hoodieRecord, Schema schema) { try { - Option recordOpt = hoodieRecord.getData().getInsertValue(schema); + Option recordOpt = ((HoodieAvroRecord) hoodieRecord).getData().getInsertValue(schema); return (GenericRecord) recordOpt.get(); } catch (IOException e) { return null;