[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:
@@ -25,6 +25,7 @@ import org.apache.hudi.common.HoodieJsonPayload;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
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.HoodieRecordPayload;
|
||||
@@ -197,7 +198,7 @@ public class HDFSParquetImporter implements Serializable {
|
||||
LOG.warn("Unable to parse date from partition field. Assuming partition as (" + partitionField + ")");
|
||||
}
|
||||
}
|
||||
return new HoodieRecord<>(new HoodieKey(rowField.toString(), partitionPath),
|
||||
return new HoodieAvroRecord<>(new HoodieKey(rowField.toString(), partitionPath),
|
||||
new HoodieJsonPayload(genericRecord.toString()));
|
||||
});
|
||||
}
|
||||
|
||||
@@ -29,6 +29,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
@@ -451,7 +452,7 @@ public class DeltaSync implements Serializable {
|
||||
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()))))
|
||||
: DataSourceUtils.createPayload(cfg.payloadClassName, gr);
|
||||
return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
|
||||
return new HoodieAvroRecord<>(keyGenerator.getKey(gr), payload);
|
||||
});
|
||||
|
||||
return Pair.of(schemaProvider, Pair.of(checkpointStr, records));
|
||||
|
||||
@@ -20,6 +20,7 @@
|
||||
package org.apache.hudi.utilities.testutils;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
|
||||
@@ -81,7 +82,8 @@ public class JdbcTestUtils {
|
||||
.stream()
|
||||
.map(r -> {
|
||||
try {
|
||||
return ((GenericRecord) r.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, props).get());
|
||||
return ((GenericRecord) ((HoodieAvroRecord) r).getData()
|
||||
.getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, props).get());
|
||||
} catch (IOException e) {
|
||||
return null;
|
||||
}
|
||||
@@ -125,7 +127,7 @@ public class JdbcTestUtils {
|
||||
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates(commitTime, inserts);
|
||||
updateRecords.stream().map(m -> {
|
||||
try {
|
||||
return m.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, props).get();
|
||||
return ((HoodieAvroRecord) m).getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA, props).get();
|
||||
} catch (IOException e) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.utilities.testutils;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
@@ -378,7 +379,7 @@ public class UtilitiesTestBase {
|
||||
|
||||
public static GenericRecord toGenericRecord(HoodieRecord hoodieRecord, Schema schema) {
|
||||
try {
|
||||
Option<IndexedRecord> recordOpt = hoodieRecord.getData().getInsertValue(schema);
|
||||
Option<IndexedRecord> recordOpt = ((HoodieAvroRecord) hoodieRecord).getData().getInsertValue(schema);
|
||||
return (GenericRecord) recordOpt.get();
|
||||
} catch (IOException e) {
|
||||
return null;
|
||||
|
||||
Reference in New Issue
Block a user