[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:
@@ -23,6 +23,7 @@ import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
@@ -122,7 +123,7 @@ public class TestHoodieHFileReaderWriter {
|
||||
record.put("time", Integer.toString(RANDOM.nextInt()));
|
||||
record.put("number", i);
|
||||
if (testAvroWithMeta) {
|
||||
writer.writeAvroWithMetadata(record, new HoodieRecord(new HoodieKey((String) record.get("_row_key"),
|
||||
writer.writeAvroWithMetadata(record, new HoodieAvroRecord(new HoodieKey((String) record.get("_row_key"),
|
||||
Integer.toString((Integer) record.get("number"))), new EmptyHoodieRecordPayload())); // payload does not matter. GenericRecord passed in is what matters
|
||||
// only HoodieKey will be looked up from the 2nd arg(HoodieRecord).
|
||||
} else {
|
||||
@@ -170,4 +171,4 @@ public class TestHoodieHFileReaderWriter {
|
||||
}
|
||||
return rowKeys;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
@@ -41,6 +42,7 @@ import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
|
||||
import org.apache.hudi.io.storage.HoodieOrcConfig;
|
||||
import org.apache.hudi.io.storage.HoodieOrcWriter;
|
||||
import org.apache.hudi.io.storage.HoodieParquetWriter;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -48,7 +50,6 @@ import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.orc.CompressionKind;
|
||||
@@ -118,7 +119,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable {
|
||||
config, schema, contextSupplier, populateMetaFields)) {
|
||||
int seqId = 1;
|
||||
for (HoodieRecord record : records) {
|
||||
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
||||
GenericRecord avroRecord = (GenericRecord) ((HoodieRecordPayload) record.getData()).getInsertValue(schema).get();
|
||||
if (populateMetaFields) {
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++));
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName);
|
||||
@@ -141,7 +142,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable {
|
||||
config, schema, contextSupplier)) {
|
||||
int seqId = 1;
|
||||
for (HoodieRecord record : records) {
|
||||
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
||||
GenericRecord avroRecord = (GenericRecord) ((HoodieRecordPayload) record.getData()).getInsertValue(schema).get();
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++));
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName);
|
||||
writer.writeAvro(record.getRecordKey(), avroRecord);
|
||||
@@ -175,7 +176,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable {
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
|
||||
logWriter.appendBlock(new HoodieAvroDataBlock(groupedRecords.stream().map(r -> {
|
||||
try {
|
||||
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
|
||||
GenericRecord val = (GenericRecord) ((HoodieRecordPayload) r.getData()).getInsertValue(schema).get();
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), "");
|
||||
return (IndexedRecord) val;
|
||||
} catch (IOException e) {
|
||||
|
||||
Reference in New Issue
Block a user