[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:
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.sink;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -217,13 +218,13 @@ public class StreamWriteFunction<I> extends AbstractStreamWriteFunction<I> {
|
||||
return new DataItem(
|
||||
record.getRecordKey(),
|
||||
record.getCurrentLocation().getInstantTime(),
|
||||
record.getData(),
|
||||
((HoodieAvroRecord) record).getData(),
|
||||
record.getOperation());
|
||||
}
|
||||
|
||||
public HoodieRecord<?> toHoodieRecord(String partitionPath) {
|
||||
HoodieKey hoodieKey = new HoodieKey(this.key, partitionPath);
|
||||
HoodieRecord<?> record = new HoodieRecord<>(hoodieKey, data, operation);
|
||||
HoodieRecord<?> record = new HoodieAvroRecord<>(hoodieKey, data, operation);
|
||||
HoodieRecordLocation loc = new HoodieRecordLocation(instant, null);
|
||||
record.setCurrentLocation(loc);
|
||||
return record;
|
||||
@@ -264,7 +265,7 @@ public class StreamWriteFunction<I> extends AbstractStreamWriteFunction<I> {
|
||||
public void preWrite(List<HoodieRecord> records) {
|
||||
// rewrite the first record with expected fileID
|
||||
HoodieRecord<?> first = records.get(0);
|
||||
HoodieRecord<?> record = new HoodieRecord<>(first.getKey(), first.getData(), first.getOperation());
|
||||
HoodieRecord<?> record = new HoodieAvroRecord<>(first.getKey(), (HoodieRecordPayload) first.getData(), first.getOperation());
|
||||
HoodieRecordLocation newLoc = new HoodieRecordLocation(first.getCurrentLocation().getInstantTime(), fileID);
|
||||
record.setCurrentLocation(newLoc);
|
||||
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.sink.bootstrap;
|
||||
|
||||
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.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordGlobalLocation;
|
||||
@@ -251,7 +252,7 @@ public class BootstrapOperator<I, O extends HoodieRecord<?>>
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static HoodieRecord generateHoodieRecord(HoodieKey hoodieKey, FileSlice fileSlice) {
|
||||
HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, null);
|
||||
HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, null);
|
||||
hoodieRecord.setCurrentLocation(new HoodieRecordGlobalLocation(hoodieKey.getPartitionPath(), fileSlice.getBaseInstantTime(), fileSlice.getFileId()));
|
||||
hoodieRecord.seal();
|
||||
|
||||
|
||||
@@ -18,16 +18,22 @@
|
||||
|
||||
package org.apache.hudi.sink.bootstrap;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
|
||||
/**
|
||||
* The index record.
|
||||
*/
|
||||
public class IndexRecord<T extends HoodieRecordPayload> extends HoodieRecord<T> {
|
||||
public class IndexRecord<T extends HoodieRecordPayload> extends HoodieAvroRecord<T> {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public IndexRecord(HoodieRecord<T> record) {
|
||||
super(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRecord<T> newInstance() {
|
||||
return new IndexRecord<>(this);
|
||||
}
|
||||
}
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.client.FlinkTaskContextSupplier;
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.model.BaseAvroPayload;
|
||||
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.HoodieRecordGlobalLocation;
|
||||
@@ -180,7 +181,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
if (globalIndex) {
|
||||
// if partition path changes, emit a delete record for old partition path,
|
||||
// then update the index state using location with new partition path.
|
||||
HoodieRecord<?> deleteRecord = new HoodieRecord<>(new HoodieKey(recordKey, oldLoc.getPartitionPath()),
|
||||
HoodieRecord<?> deleteRecord = new HoodieAvroRecord<>(new HoodieKey(recordKey, oldLoc.getPartitionPath()),
|
||||
payloadCreation.createDeletePayload((BaseAvroPayload) record.getData()));
|
||||
deleteRecord.setCurrentLocation(oldLoc.toLocal("U"));
|
||||
deleteRecord.seal();
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.sink.transform;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -111,6 +112,6 @@ public class RowDataToHoodieFunction<I extends RowData, O extends HoodieRecord>
|
||||
|
||||
HoodieRecordPayload payload = payloadCreation.createPayload(gr);
|
||||
HoodieOperation operation = HoodieOperation.fromValue(record.getRowKind().toByteValue());
|
||||
return new HoodieRecord<>(hoodieKey, payload, operation);
|
||||
return new HoodieAvroRecord<>(hoodieKey, payload, operation);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.table.format.mor;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
@@ -340,7 +341,7 @@ public class MergeOnReadInputFormat
|
||||
while (logRecordsKeyIterator.hasNext()) {
|
||||
String curAvroKey = logRecordsKeyIterator.next();
|
||||
Option<IndexedRecord> curAvroRecord = null;
|
||||
final HoodieRecord<?> hoodieRecord = scanner.getRecords().get(curAvroKey);
|
||||
final HoodieAvroRecord<?> hoodieRecord = (HoodieAvroRecord) scanner.getRecords().get(curAvroKey);
|
||||
try {
|
||||
curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema);
|
||||
} catch (IOException e) {
|
||||
@@ -412,7 +413,7 @@ public class MergeOnReadInputFormat
|
||||
public boolean hasNext() {
|
||||
while (recordsIterator.hasNext()) {
|
||||
Option<IndexedRecord> curAvroRecord = null;
|
||||
final HoodieRecord<?> hoodieRecord = recordsIterator.next();
|
||||
final HoodieAvroRecord<?> hoodieRecord = (HoodieAvroRecord) recordsIterator.next();
|
||||
try {
|
||||
curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema);
|
||||
} catch (IOException e) {
|
||||
@@ -725,7 +726,7 @@ public class MergeOnReadInputFormat
|
||||
}
|
||||
|
||||
private Option<IndexedRecord> getInsertValue(String curKey) throws IOException {
|
||||
final HoodieRecord<?> record = scanner.getRecords().get(curKey);
|
||||
final HoodieAvroRecord<?> record = (HoodieAvroRecord) scanner.getRecords().get(curKey);
|
||||
if (!emitDelete && HoodieOperation.isDelete(record.getOperation())) {
|
||||
return Option.empty();
|
||||
}
|
||||
@@ -750,7 +751,7 @@ public class MergeOnReadInputFormat
|
||||
private Option<IndexedRecord> mergeRowWithLog(
|
||||
RowData curRow,
|
||||
String curKey) throws IOException {
|
||||
final HoodieRecord<?> record = scanner.getRecords().get(curKey);
|
||||
final HoodieAvroRecord<?> record = (HoodieAvroRecord) scanner.getRecords().get(curKey);
|
||||
GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow);
|
||||
return record.getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user