[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);
|
||||
}
|
||||
|
||||
|
||||
@@ -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