[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:
@@ -20,6 +20,7 @@ package org.apache.hudi.connect.writers;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
@@ -81,7 +82,7 @@ public abstract class AbstractConnectWriter implements ConnectWriter<WriteStatus
|
||||
}
|
||||
|
||||
// Tag records with a file ID based on kafka partition and hudi partition.
|
||||
HoodieRecord<?> hoodieRecord = new HoodieRecord<>(keyGenerator.getKey(avroRecord.get()), new HoodieAvroPayload(avroRecord));
|
||||
HoodieRecord<?> hoodieRecord = new HoodieAvroRecord<>(keyGenerator.getKey(avroRecord.get()), new HoodieAvroPayload(avroRecord));
|
||||
String fileId = KafkaConnectUtils.hashDigest(String.format("%s-%s", record.kafkaPartition(), hoodieRecord.getPartitionPath()));
|
||||
hoodieRecord.unseal();
|
||||
hoodieRecord.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId));
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
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.SchemaTestUtil;
|
||||
@@ -168,7 +169,7 @@ public class TestAbstractConnectWriter {
|
||||
}
|
||||
|
||||
private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String key, String partitionPath) {
|
||||
return new HoodieRecord<>(new HoodieKey(key, partitionPath),
|
||||
return new HoodieAvroRecord<>(new HoodieKey(key, partitionPath),
|
||||
new HoodieAvroPayload(Option.of((GenericRecord) iRecord)));
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user