[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:
@@ -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);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user