[HUDI-2656] Generalize HoodieIndex for flexible record data type (#3893)
Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
@@ -105,7 +105,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
private static final Logger LOG = LogManager.getLogger(BaseHoodieWriteClient.class);
|
||||
|
||||
protected final transient HoodieMetrics metrics;
|
||||
private final transient HoodieIndex<T, ?, ?, ?> 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<T extends HoodieRecordPayload, I, K,
|
||||
this.txnManager = new TransactionManager(config, fs);
|
||||
}
|
||||
|
||||
protected abstract HoodieIndex<T, ?, ?, ?> 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<T extends HoodieRecordPayload, I, K,
|
||||
return metrics;
|
||||
}
|
||||
|
||||
public HoodieIndex<T, ?, ?, ?> getIndex() {
|
||||
public HoodieIndex<?, ?> getIndex() {
|
||||
return index;
|
||||
}
|
||||
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload>
|
||||
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);
|
||||
}
|
||||
|
||||
@@ -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 <T> Sub type of HoodieRecordPayload
|
||||
* @param <I> Type of inputs for deprecated APIs
|
||||
* @param <K> Type of keys for deprecated APIs
|
||||
* @param <O> Type of outputs for deprecated APIs
|
||||
*/
|
||||
@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> implements Serializable {
|
||||
public abstract class HoodieIndex<I, O> implements Serializable {
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
@@ -60,7 +57,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> implem
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public I tagLocation(I records, HoodieEngineContext context,
|
||||
HoodieTable<T, I, K, O> hoodieTable) throws HoodieIndexException {
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
throw new HoodieNotSupportedException("Deprecated API should not be called");
|
||||
}
|
||||
|
||||
@@ -70,7 +67,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> implem
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public O updateLocation(O writeStatuses, HoodieEngineContext context,
|
||||
HoodieTable<T, I, K, O> hoodieTable) throws HoodieIndexException {
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
throw new HoodieNotSupportedException("Deprecated API should not be called");
|
||||
}
|
||||
|
||||
@@ -79,8 +76,8 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> implem
|
||||
* the row (if it is actually present).
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public abstract HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public abstract <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
|
||||
@@ -102,14 +102,14 @@ public class HoodieIndexUtils {
|
||||
* @return the tagged {@link HoodieRecord}
|
||||
*/
|
||||
public static HoodieRecord getTaggedRecord(HoodieRecord inputRecord, Option<HoodieRecordLocation> location) {
|
||||
HoodieRecord record = inputRecord;
|
||||
HoodieRecord<?> record = inputRecord;
|
||||
if (location.isPresent()) {
|
||||
// When you have a record in multiple files in the same partition, then <row key, record> 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();
|
||||
|
||||
@@ -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<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable,
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs,
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> fileComparisonPairs,
|
||||
HoodieData<Pair<String, HoodieKey>> fileComparisonPairs,
|
||||
Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
Map<String, Long> recordsPerPartition);
|
||||
}
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, Object, Object, Object> {
|
||||
public class HoodieBloomIndex extends HoodieIndex<Object, Object> {
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieBloomIndex.class);
|
||||
|
||||
private final BaseHoodieBloomIndexHelper bloomIndexHelper;
|
||||
@@ -71,8 +69,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
// Step 0: cache the input records if needed
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
@@ -98,7 +96,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
|
||||
// Step 3: Tag the incoming records, as inserts or updates, by joining with existing record keys
|
||||
HoodieData<HoodieRecord<T>> taggedRecords = tagLocationBacktoRecords(keyFilenamePairs, records);
|
||||
HoodieData<HoodieRecord<R>> taggedRecords = tagLocationBacktoRecords(keyFilenamePairs, records);
|
||||
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
records.unpersist();
|
||||
@@ -133,7 +131,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload<T>>
|
||||
|
||||
// Step 3: Obtain a HoodieData, for each incoming record, that already exists, with the file id,
|
||||
// that contains it.
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> fileComparisonPairs =
|
||||
HoodieData<Pair<String, HoodieKey>> fileComparisonPairs =
|
||||
explodeRecordsWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairs);
|
||||
|
||||
return bloomIndexHelper.findMatchingFilesForRecordKeys(config, context, hoodieTable,
|
||||
@@ -261,7 +259,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload<T>>
|
||||
* 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<ImmutablePair<String, HoodieKey>> explodeRecordsWithFileComparisons(
|
||||
HoodieData<Pair<String, HoodieKey>> explodeRecordsWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs) {
|
||||
IndexFileFilter indexFileFilter =
|
||||
@@ -273,7 +271,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload<T>>
|
||||
String partitionPath = partitionRecordKeyPair.getLeft();
|
||||
|
||||
return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream()
|
||||
.map(partitionFileIdPair -> new ImmutablePair<>(partitionFileIdPair.getRight(),
|
||||
.map(partitionFileIdPair -> (Pair<String, HoodieKey>) new ImmutablePair<>(partitionFileIdPair.getRight(),
|
||||
new HoodieKey(recordKey, partitionPath)))
|
||||
.collect(Collectors.toList());
|
||||
}).flatMap(List::iterator);
|
||||
@@ -282,10 +280,10 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload<T>>
|
||||
/**
|
||||
* Tag the <rowKey, filename> back to the original HoodieRecord List.
|
||||
*/
|
||||
protected HoodieData<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
protected <R> HoodieData<HoodieRecord<R>> tagLocationBacktoRecords(
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> keyFilenamePair,
|
||||
HoodieData<HoodieRecord<T>> records) {
|
||||
HoodiePairData<HoodieKey, HoodieRecord<T>> keyRecordPairs =
|
||||
HoodieData<HoodieRecord<R>> records) {
|
||||
HoodiePairData<HoodieKey, HoodieRecord<R>> 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.
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload<T>> extends HoodieBloomIndex<T> {
|
||||
public class HoodieGlobalBloomIndex extends HoodieBloomIndex {
|
||||
public HoodieGlobalBloomIndex(HoodieWriteConfig config, BaseHoodieBloomIndexHelper bloomIndexHelper) {
|
||||
super(config, bloomIndexHelper);
|
||||
}
|
||||
@@ -73,7 +74,7 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload<T>> extends Ho
|
||||
*/
|
||||
|
||||
@Override
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> explodeRecordsWithFileComparisons(
|
||||
HoodieData<Pair<String, HoodieKey>> explodeRecordsWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs) {
|
||||
|
||||
@@ -86,7 +87,7 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload<T>> extends Ho
|
||||
String partitionPath = partitionRecordKeyPair.getLeft();
|
||||
|
||||
return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream()
|
||||
.map(partitionFileIdPair -> new ImmutablePair<>(partitionFileIdPair.getRight(),
|
||||
.map(partitionFileIdPair -> (Pair<String, HoodieKey>) new ImmutablePair<>(partitionFileIdPair.getRight(),
|
||||
new HoodieKey(recordKey, partitionFileIdPair.getLeft())))
|
||||
.collect(Collectors.toList());
|
||||
}).flatMap(List::iterator);
|
||||
@@ -96,11 +97,11 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload<T>> extends Ho
|
||||
* Tagging for global index should only consider the record key.
|
||||
*/
|
||||
@Override
|
||||
protected HoodieData<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
protected <R> HoodieData<HoodieRecord<R>> tagLocationBacktoRecords(
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> keyLocationPairs,
|
||||
HoodieData<HoodieRecord<T>> records) {
|
||||
HoodieData<HoodieRecord<R>> records) {
|
||||
|
||||
HoodiePairData<String, HoodieRecord<T>> incomingRowKeyRecordPairs =
|
||||
HoodiePairData<String, HoodieRecord<R>> incomingRowKeyRecordPairs =
|
||||
records.mapToPair(record -> new ImmutablePair<>(record.getRecordKey(), record));
|
||||
|
||||
HoodiePairData<String, Pair<HoodieRecordLocation, HoodieKey>> existingRecordKeyToRecordLocationHoodieKeyMap =
|
||||
@@ -109,29 +110,29 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload<T>> 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<T> hoodieRecord = record.getLeft();
|
||||
final HoodieRecord<R> hoodieRecord = record.getLeft();
|
||||
final Option<Pair<HoodieRecordLocation, HoodieKey>> 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<T> deleteRecord = new HoodieRecord(recordLocationHoodieKeyPair.get().getRight(),
|
||||
HoodieRecord<R> 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<T> insertRecord = HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty());
|
||||
HoodieRecord<R> 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<T>) HoodieIndexUtils.getTaggedRecord(new HoodieRecord<>(recordLocationHoodieKeyPair.get().getRight(), hoodieRecord.getData()),
|
||||
(HoodieRecord<R>) HoodieIndexUtils.getTaggedRecord(new HoodieAvroRecord(recordLocationHoodieKeyPair.get().getRight(), (HoodieRecordPayload) hoodieRecord.getData()),
|
||||
Option.ofNullable(recordLocationHoodieKeyPair.get().getLeft()))).iterator();
|
||||
}
|
||||
} else {
|
||||
return Collections.singletonList((HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty())).iterator();
|
||||
return Collections.singletonList((HoodieRecord<R>) HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty())).iterator();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@@ -57,11 +57,11 @@ public class ListBasedHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper
|
||||
public HoodiePairData<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable,
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs,
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> fileComparisonPairs,
|
||||
HoodieData<Pair<String, HoodieKey>> fileComparisonPairs,
|
||||
Map<String, List<BloomIndexFileInfo>> partitionToFileInfo, Map<String, Long> recordsPerPartition) {
|
||||
List<Pair<String, HoodieKey>> fileComparisonPairList =
|
||||
HoodieList.getList(fileComparisonPairs).stream()
|
||||
.sorted(Comparator.comparing(ImmutablePair::getLeft)).collect(toList());
|
||||
.sorted(Comparator.comparing(Pair::getLeft)).collect(toList());
|
||||
|
||||
List<HoodieKeyLookupResult> keyLookupResults = new ArrayList<>();
|
||||
Iterator<List<HoodieKeyLookupResult>> iterator = new HoodieBaseBloomIndexCheckFunction(
|
||||
|
||||
@@ -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 <T>
|
||||
*/
|
||||
public class HoodieBucketIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, Object, Object, Object> {
|
||||
public class HoodieBucketIndex extends HoodieIndex<Object, Object> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieBucketIndex.class);
|
||||
|
||||
@@ -66,14 +64,14 @@ public class HoodieBucketIndex<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(HoodieData<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable)
|
||||
throws HoodieIndexException {
|
||||
HoodieData<HoodieRecord<T>> taggedRecords = records.mapPartitions(recordIter -> {
|
||||
HoodieData<HoodieRecord<R>> taggedRecords = records.mapPartitions(recordIter -> {
|
||||
// partitionPath -> bucketId -> fileInfo
|
||||
Map<String, Map<Integer, Pair<String, String>>> partitionPathFileIDList = new HashMap<>();
|
||||
return new LazyIterableIterator<HoodieRecord<T>, HoodieRecord<T>>(recordIter) {
|
||||
return new LazyIterableIterator<HoodieRecord<R>, HoodieRecord<R>>(recordIter) {
|
||||
|
||||
@Override
|
||||
protected void start() {
|
||||
@@ -81,7 +79,7 @@ public class HoodieBucketIndex<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieRecord<T> computeNext() {
|
||||
protected HoodieRecord<R> computeNext() {
|
||||
HoodieRecord record = recordIter.next();
|
||||
int bucketId = BucketIdentifier.getBucketId(record, config.getBucketIndexHashField(), numBuckets);
|
||||
String partitionPath = record.getPartitionPath();
|
||||
|
||||
@@ -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;
|
||||
* <p>
|
||||
* ONLY USE FOR LOCAL TESTING
|
||||
*/
|
||||
public class HoodieInMemoryHashIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, Object, Object, Object> {
|
||||
public class HoodieInMemoryHashIndex
|
||||
extends HoodieIndex<Object, Object> {
|
||||
|
||||
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
|
||||
|
||||
@@ -56,13 +55,13 @@ public class HoodieInMemoryHashIndex<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
return records.mapPartitions(hoodieRecordIterator -> {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
List<HoodieRecord<R>> taggedRecords = new ArrayList<>();
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord<T> record = hoodieRecordIterator.next();
|
||||
HoodieRecord<R> record = hoodieRecordIterator.next();
|
||||
if (recordLocationMap.containsKey(record.getKey())) {
|
||||
record.unseal();
|
||||
record.setCurrentLocation(recordLocationMap.get(record.getKey()));
|
||||
|
||||
@@ -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 <T>
|
||||
*/
|
||||
public class HoodieGlobalSimpleIndex<T extends HoodieRecordPayload<T>> extends HoodieSimpleIndex<T> {
|
||||
public class HoodieGlobalSimpleIndex extends HoodieSimpleIndex {
|
||||
public HoodieGlobalSimpleIndex(HoodieWriteConfig config, Option<BaseKeyGenerator> keyGeneratorOpt) {
|
||||
super(config, keyGeneratorOpt);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
return tagLocationInternal(records, context, hoodieTable);
|
||||
}
|
||||
@@ -71,11 +70,11 @@ public class HoodieGlobalSimpleIndex<T extends HoodieRecordPayload<T>> extends H
|
||||
* @return {@link HoodieData} of records with record locations set
|
||||
*/
|
||||
@Override
|
||||
protected HoodieData<HoodieRecord<T>> tagLocationInternal(
|
||||
HoodieData<HoodieRecord<T>> inputRecords, HoodieEngineContext context,
|
||||
protected <R> HoodieData<HoodieRecord<R>> tagLocationInternal(
|
||||
HoodieData<HoodieRecord<R>> inputRecords, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
|
||||
HoodiePairData<String, HoodieRecord<T>> keyedInputRecords =
|
||||
HoodiePairData<String, HoodieRecord<R>> keyedInputRecords =
|
||||
inputRecords.mapToPair(entry -> new ImmutablePair<>(entry.getRecordKey(), entry));
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> allRecordLocationsInTable =
|
||||
fetchAllRecordLocations(context, hoodieTable, config.getGlobalSimpleIndexParallelism());
|
||||
@@ -114,8 +113,8 @@ public class HoodieGlobalSimpleIndex<T extends HoodieRecordPayload<T>> extends H
|
||||
* @param existingRecords existing records with {@link HoodieRecordLocation}s
|
||||
* @return {@link HoodieData} of {@link HoodieRecord}s with tagged {@link HoodieRecordLocation}s
|
||||
*/
|
||||
private HoodieData<HoodieRecord<T>> getTaggedRecords(
|
||||
HoodiePairData<String, HoodieRecord<T>> incomingRecords,
|
||||
private <R> HoodieData<HoodieRecord<R>> getTaggedRecords(
|
||||
HoodiePairData<String, HoodieRecord<R>> incomingRecords,
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> existingRecords) {
|
||||
HoodiePairData<String, Pair<String, HoodieRecordLocation>> existingRecordByRecordKey =
|
||||
existingRecords.mapToPair(
|
||||
@@ -124,29 +123,29 @@ public class HoodieGlobalSimpleIndex<T extends HoodieRecordPayload<T>> extends H
|
||||
|
||||
return incomingRecords.leftOuterJoin(existingRecordByRecordKey).values()
|
||||
.flatMap(entry -> {
|
||||
HoodieRecord<T> inputRecord = entry.getLeft();
|
||||
HoodieRecord<R> inputRecord = entry.getLeft();
|
||||
Option<Pair<String, HoodieRecordLocation>> partitionPathLocationPair = Option.ofNullable(entry.getRight().orElse(null));
|
||||
List<HoodieRecord<T>> taggedRecords;
|
||||
List<HoodieRecord<R>> 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<T> deleteRecord = new HoodieRecord(new HoodieKey(inputRecord.getRecordKey(), partitionPath), new EmptyHoodieRecordPayload());
|
||||
HoodieRecord<R> 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<T> insertRecord = (HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty());
|
||||
HoodieRecord<R> insertRecord = (HoodieRecord<R>) 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<T> newRecord = new HoodieRecord<>(new HoodieKey(inputRecord.getRecordKey(), partitionPath), inputRecord.getData());
|
||||
taggedRecords = Collections.singletonList((HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(newRecord, Option.ofNullable(location)));
|
||||
HoodieRecord<R> newRecord = new HoodieAvroRecord(new HoodieKey(inputRecord.getRecordKey(), partitionPath), (HoodieRecordPayload) inputRecord.getData());
|
||||
taggedRecords = Collections.singletonList((HoodieRecord<R>) HoodieIndexUtils.getTaggedRecord(newRecord, Option.ofNullable(location)));
|
||||
}
|
||||
} else {
|
||||
taggedRecords = Collections.singletonList((HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty()));
|
||||
taggedRecords = Collections.singletonList((HoodieRecord<R>) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty()));
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
});
|
||||
|
||||
@@ -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 <T> type of {@link HoodieRecordPayload}
|
||||
*/
|
||||
public class HoodieSimpleIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, Object, Object, Object> {
|
||||
public class HoodieSimpleIndex
|
||||
extends HoodieIndex<Object, Object> {
|
||||
|
||||
private final Option<BaseKeyGenerator> keyGeneratorOpt;
|
||||
|
||||
@@ -88,8 +85,8 @@ public class HoodieSimpleIndex<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
return tagLocationInternal(records, context, hoodieTable);
|
||||
}
|
||||
@@ -102,23 +99,23 @@ public class HoodieSimpleIndex<T extends HoodieRecordPayload<T>>
|
||||
* @param hoodieTable instance of {@link HoodieTable} to use
|
||||
* @return {@link HoodieData} of records with record locations set
|
||||
*/
|
||||
protected HoodieData<HoodieRecord<T>> tagLocationInternal(
|
||||
HoodieData<HoodieRecord<T>> inputRecords, HoodieEngineContext context,
|
||||
protected <R> HoodieData<HoodieRecord<R>> tagLocationInternal(
|
||||
HoodieData<HoodieRecord<R>> inputRecords, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
if (config.getSimpleIndexUseCaching()) {
|
||||
inputRecords.persist(new HoodieConfig(config.getProps())
|
||||
.getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE));
|
||||
}
|
||||
|
||||
HoodiePairData<HoodieKey, HoodieRecord<T>> keyedInputRecords =
|
||||
HoodiePairData<HoodieKey, HoodieRecord<R>> keyedInputRecords =
|
||||
inputRecords.mapToPair(record -> new ImmutablePair<>(record.getKey(), record));
|
||||
HoodiePairData<HoodieKey, HoodieRecordLocation> existingLocationsOnTable =
|
||||
fetchRecordLocationsForAffectedPartitions(keyedInputRecords.keys(), context, hoodieTable,
|
||||
config.getSimpleIndexParallelism());
|
||||
|
||||
HoodieData<HoodieRecord<T>> taggedRecords =
|
||||
HoodieData<HoodieRecord<R>> taggedRecords =
|
||||
keyedInputRecords.leftOuterJoin(existingLocationsOnTable).map(entry -> {
|
||||
final HoodieRecord<T> untaggedRecord = entry.getRight().getLeft();
|
||||
final HoodieRecord<R> untaggedRecord = entry.getRight().getLeft();
|
||||
final Option<HoodieRecordLocation> location = Option.ofNullable(entry.getRight().getRight().orElse(null));
|
||||
return HoodieIndexUtils.getTaggedRecord(untaggedRecord, location);
|
||||
});
|
||||
|
||||
@@ -393,7 +393,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
|
||||
@Override
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> insertValue) {
|
||||
Option<Map<String, String>> recordMetadata = record.getData().getMetadata();
|
||||
Option<Map<String, String>> recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata();
|
||||
try {
|
||||
init(record);
|
||||
flushToDiskIfRequired(record);
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload, I, K, O> extends
|
||||
*/
|
||||
@Override
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> avroRecord) {
|
||||
Option recordMetadata = record.getData().getMetadata();
|
||||
Option recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata();
|
||||
if (HoodieOperation.isDelete(record.getOperation())) {
|
||||
avroRecord = Option.empty();
|
||||
}
|
||||
|
||||
@@ -324,7 +324,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> 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<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key).newInstance();
|
||||
try {
|
||||
Option<IndexedRecord> combinedAvroRecord =
|
||||
hoodieRecord.getData().combineAndGetUpdateValue(oldRecord,
|
||||
|
||||
@@ -85,7 +85,7 @@ public class HoodieSortedMergeHandle<T extends HoodieRecordPayload, I, K, O> ext
|
||||
}
|
||||
|
||||
// This is a new insert
|
||||
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(keyToPreWrite));
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(keyToPreWrite).newInstance();
|
||||
if (writtenRecordKeys.contains(keyToPreWrite)) {
|
||||
throw new HoodieUpsertException("Insert/Update not in sorted order");
|
||||
}
|
||||
|
||||
@@ -210,7 +210,7 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> avroRecord, Option<Exception> 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);
|
||||
|
||||
@@ -102,7 +102,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final HoodieIndex<T, ?, ?, ?> index;
|
||||
protected final HoodieIndex<?, ?> index;
|
||||
private SerializableConfiguration hadoopConfiguration;
|
||||
protected final TaskContextSupplier taskContextSupplier;
|
||||
private final HoodieTableMetadata metadata;
|
||||
@@ -128,7 +128,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
this.taskContextSupplier = context.getTaskContextSupplier();
|
||||
}
|
||||
|
||||
protected abstract HoodieIndex<T, ?, ?, ?> 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<T extends HoodieRecordPayload, I, K, O> implem
|
||||
/**
|
||||
* Return the index.
|
||||
*/
|
||||
public HoodieIndex<T, ?, ?, ?> getIndex() {
|
||||
public HoodieIndex<?, ?> getIndex() {
|
||||
return index;
|
||||
}
|
||||
|
||||
|
||||
@@ -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<Hoodie
|
||||
@Override
|
||||
protected void consumeOneRecord(HoodieRecord record) {
|
||||
try {
|
||||
bootstrapHandle.write(record, record.getData().getInsertValue(bootstrapHandle.getWriterSchemaWithMetaFields()));
|
||||
bootstrapHandle.write(record, ((HoodieRecordPayload) record.getData())
|
||||
.getInsertValue(bootstrapHandle.getWriterSchemaWithMetaFields()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
|
||||
@@ -86,5 +86,5 @@ public abstract class BaseWriteHelper<T extends HoodieRecordPayload, I, K, O, R>
|
||||
}
|
||||
|
||||
public abstract I deduplicateRecords(
|
||||
I records, HoodieIndex<T, ?, ?, ?> index, int parallelism);
|
||||
I records, HoodieIndex<?, ?> index, int parallelism);
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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 <T> payload type
|
||||
*/
|
||||
public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<List<HoodieRecord<T>>, List<WriteStatus>> {
|
||||
protected FlinkHoodieIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
@@ -48,21 +47,22 @@ public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends Ho
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieList.of(tagLocation(HoodieList.getList(records), context, hoodieTable));
|
||||
List<HoodieRecord<T>> hoodieRecords = tagLocation(HoodieList.getList(records.map(record -> (HoodieRecord<T>) record)), context, hoodieTable);
|
||||
return HoodieList.of(hoodieRecords.stream().map(r -> (HoodieRecord<R>) r).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -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());
|
||||
}
|
||||
|
||||
@@ -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 <T> type of payload
|
||||
*/
|
||||
public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
public class FlinkInMemoryStateIndex extends HoodieIndex<List<HoodieRecord>, List<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FlinkInMemoryStateIndex.class);
|
||||
|
||||
@@ -50,8 +45,8 @@ public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
throw new UnsupportedOperationException("No need to tag location for FlinkInMemoryStateIndex");
|
||||
}
|
||||
@@ -88,4 +83,4 @@ public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload<T>>
|
||||
public boolean isImplicitWithStorage() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<R> extends
|
||||
}
|
||||
|
||||
List<HoodieRecord<EmptyHoodieRecordPayload>> 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<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords = HoodieList.getList(
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload, R> extends BaseWrit
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> deduplicateRecords(
|
||||
List<HoodieRecord<T>> records, HoodieIndex<T, ?, ?, ?> index, int parallelism) {
|
||||
List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism) {
|
||||
Map<Object, List<Pair<Object, HoodieRecord<T>>>> 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<T extends HoodieRecordPayload, R> extends BaseWrit
|
||||
boolean choosePrev = data1.equals(reducedData);
|
||||
HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey();
|
||||
HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation();
|
||||
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(reducedKey, reducedData, operation);
|
||||
HoodieRecord<T> hoodieRecord = new HoodieAvroRecord<>(reducedKey, reducedData, operation);
|
||||
// reuse the location from the first record.
|
||||
hoodieRecord.setCurrentLocation(rec1.getCurrentLocation());
|
||||
return hoodieRecord;
|
||||
|
||||
@@ -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<String> partitions = asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
List<Pair<String, BloomIndexFileInfo>> 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<HoodieRecord> 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<HoodieKey> 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<HoodieRecord> toTagRecords = new ArrayList<>();
|
||||
toTagRecords.add(new HoodieRecord(record4.getKey(), null));
|
||||
toTagRecords.add(new HoodieAvroRecord(record4.getKey(), null));
|
||||
List<HoodieRecord> taggedRecords = tagLocation(bloomIndex, toTagRecords, hoodieTable);
|
||||
Map<HoodieKey, Option<Pair<String, String>>> recordLocations = new HashMap<>();
|
||||
for (HoodieRecord taggedRecord : taggedRecords) {
|
||||
@@ -396,7 +397,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness {
|
||||
hoodieTable = HoodieFlinkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> 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());
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload<T>>
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
public abstract class JavaHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<List<HoodieRecord<T>>, List<WriteStatus>> {
|
||||
protected JavaHoodieIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
@@ -44,21 +44,22 @@ public abstract class JavaHoodieIndex<T extends HoodieRecordPayload> extends Hoo
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieList.of(tagLocation(HoodieList.getList(records), context, hoodieTable));
|
||||
List<HoodieRecord<T>> hoodieRecords = tagLocation(HoodieList.getList(records.map(record -> (HoodieRecord<T>) record)), context, hoodieTable);
|
||||
return HoodieList.of(hoodieRecords.stream().map(r -> (HoodieRecord<R>) r).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -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());
|
||||
}
|
||||
|
||||
@@ -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<R> extends
|
||||
}
|
||||
|
||||
List<HoodieRecord<EmptyHoodieRecordPayload>> 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<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords = HoodieList.getList(
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload,R> extends BaseWriteH
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> deduplicateRecords(
|
||||
List<HoodieRecord<T>> records, HoodieIndex<T, ?, ?, ?> index, int parallelism) {
|
||||
List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism) {
|
||||
boolean isIndexingGlobal = index.isGlobal();
|
||||
Map<Object, List<Pair<Object, HoodieRecord<T>>>> keyedRecords = records.stream().map(record -> {
|
||||
HoodieKey hoodieKey = record.getKey();
|
||||
@@ -70,7 +71,7 @@ public class JavaWriteHelper<T extends HoodieRecordPayload,R> 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<T>(rec1.getKey(), reducedData);
|
||||
return new HoodieAvroRecord<T>(rec1.getKey(), reducedData);
|
||||
}).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<HoodieRecord> 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);
|
||||
|
||||
@@ -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<HoodieRecord> 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<HoodieRecord> 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<HoodieRecord> 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
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload<T>> 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<T, ?, ?, ?> index;
|
||||
private final transient HoodieIndex<?, ?> index;
|
||||
private HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable;
|
||||
private transient Option<SQLContext> sqlContextOpt;
|
||||
private final transient HoodieSparkEngineContext context;
|
||||
@@ -172,7 +173,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload<T>> implements Seria
|
||||
*/
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> checkExists(JavaRDD<HoodieKey> 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()))
|
||||
|
||||
@@ -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<T extends HoodieRecordPa
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -29,6 +29,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.ClusteringGroupInfo;
|
||||
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;
|
||||
@@ -181,7 +182,7 @@ public abstract class SingleSparkJobExecutionStrategy<T extends HoodieRecordPayl
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -83,8 +83,8 @@ public class HoodieJavaRDD<T> extends HoodieData<T> {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persist(String storageLevel) {
|
||||
rddData.persist(StorageLevel.fromString(storageLevel));
|
||||
public void persist(String cacheConfig) {
|
||||
rddData.persist(StorageLevel.fromString(cacheConfig));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload>
|
||||
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;
|
||||
});
|
||||
}
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
extends HoodieIndex<JavaRDD<HoodieRecord<T>>, JavaRDD<WriteStatus>> {
|
||||
protected SparkHoodieIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
@@ -46,21 +45,23 @@ public abstract class SparkHoodieIndex<T extends HoodieRecordPayload<T>>
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> 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<T>) record)), context, hoodieTable)
|
||||
.map(r -> (HoodieRecord<R>) r));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -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());
|
||||
}
|
||||
|
||||
@@ -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<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable,
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs,
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> fileComparisonPairs,
|
||||
HoodieData<Pair<String, HoodieKey>> fileComparisonPairs,
|
||||
Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
Map<String, Long> recordsPerPartition) {
|
||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
public class SparkHoodieHBaseIndex extends HoodieIndex<Object, Object> {
|
||||
|
||||
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<T extends HoodieRecordPayload<T>>
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
private Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> locationTagFunction(
|
||||
private <R> Function2<Integer, Iterator<HoodieRecord<R>>, Iterator<HoodieRecord<R>>> 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<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>) (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<T extends HoodieRecordPayload<T>>
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
List<HoodieRecord<R>> taggedRecords = new ArrayList<>();
|
||||
try (HTable hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName))) {
|
||||
List<Get> statements = new ArrayList<>();
|
||||
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
|
||||
@@ -256,19 +254,19 @@ public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>>
|
||||
// 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<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
return HoodieJavaRDD.of(HoodieJavaRDD.getJavaRDD(records)
|
||||
.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true));
|
||||
@@ -303,7 +301,7 @@ public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>>
|
||||
|
||||
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
|
||||
|
||||
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition, statusIterator) -> {
|
||||
return (partition, statusIterator) -> {
|
||||
|
||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||
// Grab the global HBase connection
|
||||
@@ -385,7 +383,7 @@ public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>>
|
||||
mutations.clear();
|
||||
}
|
||||
|
||||
public Map<String, Integer> mapFileWithInsertsToUniquePartition(JavaRDD<WriteStatus> writeStatusRDD) {
|
||||
Map<String, Integer> mapFileWithInsertsToUniquePartition(JavaRDD<WriteStatus> writeStatusRDD) {
|
||||
final Map<String, Integer> 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<T extends HoodieRecordPayload<T>>
|
||||
}
|
||||
}
|
||||
|
||||
public Tuple2<Long, Integer> getHBasePutAccessParallelism(final JavaRDD<WriteStatus> writeStatusRDD) {
|
||||
Tuple2<Long, Integer> getHBasePutAccessParallelism(final JavaRDD<WriteStatus> writeStatusRDD) {
|
||||
final JavaPairRDD<Long, Integer> 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));
|
||||
|
||||
@@ -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();
|
||||
|
||||
@@ -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();
|
||||
|
||||
@@ -74,7 +74,7 @@ public class SparkBucketIndexPartitioner<T extends HoodieRecordPayload<T>> exten
|
||||
" Bucket index partitioner should only be used by BucketIndex other than "
|
||||
+ table.getIndex().getClass().getSimpleName());
|
||||
}
|
||||
this.numBuckets = ((HoodieBucketIndex<T>) table.getIndex()).getNumBuckets();
|
||||
this.numBuckets = ((HoodieBucketIndex) table.getIndex()).getNumBuckets();
|
||||
this.indexKeyField = config.getBucketIndexHashField();
|
||||
this.totalPartitionPaths = profile.getPartitionPaths().size();
|
||||
partitionPaths = new ArrayList<>(profile.getPartitionPaths());
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload,R> extends
|
||||
}
|
||||
|
||||
JavaRDD<HoodieRecord<T>> 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<HoodieRecord<T>> taggedRecords = HoodieJavaRDD.getJavaRDD(
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload,R> extends BaseWrite
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> deduplicateRecords(
|
||||
JavaRDD<HoodieRecord<T>> records, HoodieIndex<T, ?, ?, ?> index, int parallelism) {
|
||||
JavaRDD<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism) {
|
||||
boolean isIndexingGlobal = index.isGlobal();
|
||||
return records.mapToPair(record -> {
|
||||
HoodieKey hoodieKey = record.getKey();
|
||||
@@ -70,7 +71,7 @@ public class SparkWriteHelper<T extends HoodieRecordPayload,R> extends BaseWrite
|
||||
T reducedData = (T) rec2.getData().preCombine(rec1.getData());
|
||||
HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey();
|
||||
|
||||
return new HoodieRecord<T>(reducedKey, reducedData);
|
||||
return new HoodieAvroRecord<T>(reducedKey, reducedData);
|
||||
}, parallelism).map(Tuple2::_2);
|
||||
}
|
||||
|
||||
|
||||
@@ -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<HoodieRecord> 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<HoodieRecord> 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();
|
||||
|
||||
@@ -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");
|
||||
}
|
||||
|
||||
@@ -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<String, HoodieRecord> insertRecordMap = insertRecords.stream()
|
||||
.collect(Collectors.toMap(r -> r.getRecordKey(), Function.identity()));
|
||||
@@ -147,7 +148,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
List<HoodieRecord> 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);
|
||||
|
||||
@@ -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<RawTripTestPayload> 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<HoodieRecord<RawTripTestPayload>> 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()));
|
||||
}
|
||||
|
||||
@@ -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<HoodieRecord> 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<HoodieKey, Option<Pair<String, String>>> getRecordLocations(JavaRDD<HoodieKey> keyRDD, HoodieTable hoodieTable) {
|
||||
JavaRDD<HoodieRecord> 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())
|
||||
|
||||
@@ -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<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult> queue =
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
// Produce
|
||||
Future<Boolean> resFuture = executorService.submit(() -> {
|
||||
@@ -93,7 +94,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
final Iterator<HoodieRecord> originalRecordIterator = hoodieRecords.iterator();
|
||||
int recordsRead = 0;
|
||||
while (queue.iterator().hasNext()) {
|
||||
final HoodieRecord originalRecord = originalRecordIterator.next();
|
||||
final HoodieAvroRecord originalRecord = (HoodieAvroRecord) originalRecordIterator.next();
|
||||
final Option<IndexedRecord> originalInsertValue =
|
||||
originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA);
|
||||
final HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> 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<List<HoodieRecord>> recs = new ArrayList<>();
|
||||
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult> queue =
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Record Key to <Producer Index, Rec Index within a producer>
|
||||
@@ -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<HoodieRecord> payload = queue.iterator().next();
|
||||
final HoodieLazyInsertIterable.HoodieInsertValueGenResult payload = queue.iterator().next();
|
||||
final HoodieRecord rec = payload.record;
|
||||
Tuple2<Integer, Integer> producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey());
|
||||
Integer lastSeenPos = lastSeenMap.get(producerPos._1());
|
||||
@@ -216,12 +217,12 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
// maximum number of records to keep in memory.
|
||||
final int recordLimit = 5;
|
||||
final SizeEstimator<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> sizeEstimator = new DefaultSizeEstimator<>();
|
||||
HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload =
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0));
|
||||
final SizeEstimator<HoodieLazyInsertIterable.HoodieInsertValueGenResult> 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<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult> queue =
|
||||
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Produce
|
||||
@@ -266,8 +267,8 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
final SizeEstimator<Tuple2<HoodieRecord, Option<IndexedRecord>>> sizeEstimator = new DefaultSizeEstimator<>();
|
||||
// queue memory limit
|
||||
HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> 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;
|
||||
|
||||
|
||||
@@ -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 {
|
||||
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload<T>> extends SparkHoodieIndex<T> {
|
||||
|
||||
public DummyHoodieIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> 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) {
|
||||
|
||||
@@ -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<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
List<ImmutablePair<String, BloomIndexFileInfo>> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
|
||||
List<Pair<String, BloomIndexFileInfo>> 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<HoodieRecord> 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<HoodieKey> 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<HoodieRecord> taggedRecords = tagLocation(
|
||||
bloomIndex, keysRDD.map(k -> new HoodieRecord(k, null)), hoodieTable);
|
||||
bloomIndex, keysRDD.map(k -> new HoodieAvroRecord(k, null)), hoodieTable);
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> 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());
|
||||
|
||||
@@ -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<String> 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<ImmutablePair<String, HoodieKey>> comparisonKeyList = HoodieJavaRDD.getJavaRDD(
|
||||
List<Pair<String, HoodieKey>> 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<HoodieRecord> 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);
|
||||
|
||||
|
||||
@@ -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<String> 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(
|
||||
|
||||
@@ -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<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
JavaRDD<HoodieRecord<HoodieAvroRecord>> 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<HoodieRecord> taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table);
|
||||
HoodieData<HoodieRecord<HoodieAvroRecord>> taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table);
|
||||
assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(r -> r.isCurrentLocationKnown()));
|
||||
|
||||
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(table, SCHEMA);
|
||||
|
||||
@@ -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<HoodieRecord> 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<HoodieRecord> 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<HoodieRecord> 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<HoodieRecord> newWriteRecords = jsc().parallelize(newRecords, 1);
|
||||
@@ -764,7 +765,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// is not implemented via HoodieWriteClient
|
||||
JavaRDD<WriteStatus> 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;
|
||||
@@ -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<HoodieRecord> 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<HoodieRecord> 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<HoodieRecord> 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
|
||||
|
||||
@@ -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<EmptyHoodieRecordPayload, JavaRDD<HoodieRecord>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table;
|
||||
@Mock
|
||||
|
||||
@@ -175,7 +175,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
HoodieIndex index = new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
HoodieIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
JavaRDD<HoodieRecord> updatedTaggedRecordsRDD = tagLocation(index, updatedRecordsRDD, table);
|
||||
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
@@ -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<String, GenericRecord> 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();
|
||||
}
|
||||
|
||||
@@ -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<HoodieRecord> recordsToDelete = jsc.parallelize(records, 1)
|
||||
.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload()));
|
||||
.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
|
||||
JavaRDD<HoodieRecord> taggedRecords = tagLocation(index, recordsToDelete, table);
|
||||
return taggedRecords.map(record -> record.getKey()).collect();
|
||||
};
|
||||
|
||||
Reference in New Issue
Block a user