[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:
@@ -26,6 +26,7 @@ import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.JavaTaskContextSupplier;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.ClusteringOperation;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -237,7 +238,7 @@ public abstract class JavaExecutionStrategy<T extends HoodieRecordPayload<T>>
|
||||
HoodieKey hoodieKey = new HoodieKey(key, partition);
|
||||
|
||||
HoodieRecordPayload avroPayload = new RewriteAvroPayload(record);
|
||||
HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, avroPayload);
|
||||
HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, avroPayload);
|
||||
return hoodieRecord;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -25,7 +25,6 @@ import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.data.HoodieList;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -33,8 +32,9 @@ import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public abstract class JavaHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
public abstract class JavaHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<List<HoodieRecord<T>>, List<WriteStatus>> {
|
||||
protected JavaHoodieIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
@@ -44,21 +44,22 @@ public abstract class JavaHoodieIndex<T extends HoodieRecordPayload> extends Hoo
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
public <R> HoodieData<HoodieRecord<R>> tagLocation(
|
||||
HoodieData<HoodieRecord<R>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieList.of(tagLocation(HoodieList.getList(records), context, hoodieTable));
|
||||
List<HoodieRecord<T>> hoodieRecords = tagLocation(HoodieList.getList(records.map(record -> (HoodieRecord<T>) record)), context, hoodieTable);
|
||||
return HoodieList.of(hoodieRecords.stream().map(r -> (HoodieRecord<R>) r).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -44,9 +44,9 @@ public final class JavaHoodieIndexFactory {
|
||||
// TODO more indexes to be added
|
||||
switch (config.getIndexType()) {
|
||||
case INMEMORY:
|
||||
return new HoodieInMemoryHashIndex<>(config);
|
||||
return new HoodieInMemoryHashIndex(config);
|
||||
case BLOOM:
|
||||
return new HoodieBloomIndex<>(config, ListBasedHoodieBloomIndexHelper.getInstance());
|
||||
return new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance());
|
||||
default:
|
||||
throw new HoodieIndexException("Unsupported index type " + config.getIndexType());
|
||||
}
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.data.HoodieList;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -95,7 +96,7 @@ public class JavaDeleteHelper<R> extends
|
||||
}
|
||||
|
||||
List<HoodieRecord<EmptyHoodieRecordPayload>> dedupedRecords =
|
||||
dedupedKeys.stream().map(key -> new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList());
|
||||
dedupedKeys.stream().map(key -> new HoodieAvroRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList());
|
||||
Instant beginTag = Instant.now();
|
||||
// perform index look up to get existing location of records
|
||||
List<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords = HoodieList.getList(
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.table.action.commit;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.data.HoodieList;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
@@ -55,7 +56,7 @@ public class JavaWriteHelper<T extends HoodieRecordPayload,R> extends BaseWriteH
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> deduplicateRecords(
|
||||
List<HoodieRecord<T>> records, HoodieIndex<T, ?, ?, ?> index, int parallelism) {
|
||||
List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism) {
|
||||
boolean isIndexingGlobal = index.isGlobal();
|
||||
Map<Object, List<Pair<Object, HoodieRecord<T>>>> keyedRecords = records.stream().map(record -> {
|
||||
HoodieKey hoodieKey = record.getKey();
|
||||
@@ -70,7 +71,7 @@ public class JavaWriteHelper<T extends HoodieRecordPayload,R> extends BaseWriteH
|
||||
// we cannot allow the user to change the key or partitionPath, since that will affect
|
||||
// everything
|
||||
// so pick it from one of the records.
|
||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||
return new HoodieAvroRecord<T>(rec1.getKey(), reducedData);
|
||||
}).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,16 +18,11 @@
|
||||
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hudi.client.HoodieJavaWriteClient;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.engine.EngineType;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -40,6 +35,13 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
import org.apache.hudi.testutils.HoodieJavaClientTestBase;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
@@ -111,8 +113,8 @@ public class TestHoodieConcatHandle extends HoodieJavaClientTestBase {
|
||||
List<HoodieRecord> records1 = new ArrayList<>();
|
||||
RawTripTestPayload insertRow1 = new RawTripTestPayload(insertRecordStr1);
|
||||
RawTripTestPayload insertRow2 = new RawTripTestPayload(insertRecordStr2);
|
||||
records1.add(new HoodieRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1));
|
||||
records1.add(new HoodieRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2));
|
||||
records1.add(new HoodieAvroRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1));
|
||||
records1.add(new HoodieAvroRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2));
|
||||
|
||||
int startInstant = 1;
|
||||
String firstCommitTime = makeNewCommitTime(startInstant++);
|
||||
@@ -140,8 +142,8 @@ public class TestHoodieConcatHandle extends HoodieJavaClientTestBase {
|
||||
insertRow1 = new RawTripTestPayload(insertRecordStr1);
|
||||
insertRow2 = new RawTripTestPayload(insertRecordStr2);
|
||||
// The recordKey of records2 and records1 are the same, but the values of other fields are different
|
||||
records2.add(new HoodieRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1));
|
||||
records2.add(new HoodieRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2));
|
||||
records2.add(new HoodieAvroRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1));
|
||||
records2.add(new HoodieAvroRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2));
|
||||
|
||||
String newCommitTime = makeNewCommitTime(startInstant++);
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.engine.EngineType;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -42,6 +43,8 @@ import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.table.HoodieJavaCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieJavaTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieJavaClientTestBase;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -49,8 +52,6 @@ import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hudi.testutils.HoodieJavaClientTestBase;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
@@ -142,11 +143,11 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
// Insert new records
|
||||
writeClient.insert(records, firstCommitTime);
|
||||
@@ -175,12 +176,12 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
RawTripTestPayload updateRowChanges1 = new RawTripTestPayload(updateRecordStr1);
|
||||
HoodieRecord updatedRecord1 = new HoodieRecord(
|
||||
HoodieRecord updatedRecord1 = new HoodieAvroRecord(
|
||||
new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1);
|
||||
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord insertedRecord1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
List<HoodieRecord> updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1);
|
||||
|
||||
@@ -254,7 +255,7 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
String recordStr =
|
||||
String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}", UUID.randomUUID().toString(), time, i);
|
||||
RawTripTestPayload rowChange = new RawTripTestPayload(recordStr);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
}
|
||||
return records;
|
||||
}
|
||||
@@ -280,11 +281,11 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
// Insert new records
|
||||
BaseJavaCommitActionExecutor actionExecutor = new JavaInsertCommitActionExecutor(context, config, table,
|
||||
@@ -382,7 +383,7 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString()
|
||||
+ "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}";
|
||||
RawTripTestPayload rowChange = new RawTripTestPayload(recordStr);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
}
|
||||
|
||||
// Insert new records
|
||||
|
||||
Reference in New Issue
Block a user