1
0

[HUDI-2909] Handle logical type in TimestampBasedKeyGenerator (#4203)

* [HUDI-2909] Handle logical type in TimestampBasedKeyGenerator

Timestampbased key generator was returning diff values for row writer and non row writer path. this patch fixes it and is guarded by a config flag (`hoodie.datasource.write.keygenerator.consistent.logical.timestamp.enabled`)
This commit is contained in:
Sagar Sumit
2022-01-08 20:52:44 +05:30
committed by GitHub
parent 03a83ffeb5
commit 827549949c
36 changed files with 364 additions and 101 deletions

View File

@@ -108,7 +108,6 @@ public abstract class MultipleSparkJobExecutionStrategy<T extends HoodieRecordPa
return writeMetadata;
}
/**
* Execute clustering to write inputRecords into new files as defined by rules in strategy parameters.
* The number of new file groups created is bounded by numOutputGroups.
@@ -141,7 +140,7 @@ public abstract class MultipleSparkJobExecutionStrategy<T extends HoodieRecordPa
getWriteConfig(), HoodieAvroUtils.addMetadataFields(schema)));
} else if (strategyParams.containsKey(PLAN_STRATEGY_SORT_COLUMNS.key())) {
return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key()).split(","),
HoodieAvroUtils.addMetadataFields(schema)));
HoodieAvroUtils.addMetadataFields(schema), getWriteConfig().isConsistentLogicalTimestampEnabled()));
} else {
return Option.empty();
}

View File

@@ -39,15 +39,18 @@ public class RDDCustomColumnsSortPartitioner<T extends HoodieRecordPayload>
private final String[] sortColumnNames;
private final SerializableSchema serializableSchema;
private final boolean consistentLogicalTimestampEnabled;
public RDDCustomColumnsSortPartitioner(HoodieWriteConfig config) {
this.serializableSchema = new SerializableSchema(new Schema.Parser().parse(config.getSchema()));
this.sortColumnNames = getSortColumnName(config);
this.consistentLogicalTimestampEnabled = config.isConsistentLogicalTimestampEnabled();
}
public RDDCustomColumnsSortPartitioner(String[] columnNames, Schema schema) {
public RDDCustomColumnsSortPartitioner(String[] columnNames, Schema schema, boolean consistentLogicalTimestampEnabled) {
this.sortColumnNames = columnNames;
this.serializableSchema = new SerializableSchema(schema);
this.consistentLogicalTimestampEnabled = consistentLogicalTimestampEnabled;
}
@Override
@@ -55,9 +58,10 @@ public class RDDCustomColumnsSortPartitioner<T extends HoodieRecordPayload>
int outputSparkPartitions) {
final String[] sortColumns = this.sortColumnNames;
final SerializableSchema schema = this.serializableSchema;
final boolean consistentLogicalTimestampEnabled = this.consistentLogicalTimestampEnabled;
return records.sortBy(
record -> {
Object recordValue = HoodieAvroUtils.getRecordColumnValues(record, sortColumns, schema);
Object recordValue = HoodieAvroUtils.getRecordColumnValues(record, sortColumns, schema, consistentLogicalTimestampEnabled);
// null values are replaced with empty string for null_first order
if (recordValue == null) {
return StringUtils.EMPTY_STRING;
@@ -66,7 +70,6 @@ public class RDDCustomColumnsSortPartitioner<T extends HoodieRecordPayload>
}
},
true, outputSparkPartitions);
}
@Override

View File

@@ -147,9 +147,9 @@ public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase {
JavaRDD<HoodieRecord> records1 = generateTestRecordsForBulkInsert(jsc);
JavaRDD<HoodieRecord> records2 = generateTripleTestRecordsForBulkInsert(jsc);
testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA),
testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA, false),
records1, true, true, generateExpectedPartitionNumRecords(records1), Option.of(columnComparator));
testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA),
testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA, false),
records2, true, true, generateExpectedPartitionNumRecords(records2), Option.of(columnComparator));
HoodieWriteConfig config = HoodieWriteConfig

View File

@@ -45,7 +45,7 @@ public class TestBucketIdentifier {
String indexKeyField = "_row_key";
GenericRecord record = KeyGeneratorTestUtilities.getRecord();
HoodieRecord hoodieRecord = new HoodieRecord(
new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField), ""), null);
new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField, false), ""), null);
int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8);
assert bucketId == BucketIdentifier.getBucketId(
Arrays.asList(record.get(indexKeyField).toString()), 8);
@@ -57,7 +57,7 @@ public class TestBucketIdentifier {
String indexKeyField = "_row_key";
GenericRecord record = KeyGeneratorTestUtilities.getRecord();
HoodieRecord hoodieRecord = new HoodieRecord(
new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField), ""), null);
new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField, false), ""), null);
int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8);
assert bucketId == BucketIdentifier.getBucketId(
Arrays.asList(record.get(indexKeyField).toString()), 8);

View File

@@ -238,6 +238,40 @@ public class TestTimestampBasedKeyGenerator {
assertEquals("2021-04-19", keyGen.getPartitionPath(baseRow));
}
@Test
public void testScalarWithLogicalType() throws IOException {
schema = SchemaTestUtil.getTimestampWithLogicalTypeSchema();
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
baseRecord = SchemaTestUtil.generateAvroRecordFromJson(schema, 1, "001", "f1");
baseRecord.put("createTime", 1638513806000000L);
properties = getBaseKeyConfig("SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS");
properties.setProperty(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "true");
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
assertEquals("2021/12/03", hk1.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2021/12/03", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("2021/12/03", keyGen.getPartitionPath(internalRow, baseRow.schema()));
// timezone is GMT, createTime is null
baseRecord.put("createTime", null);
properties = getBaseKeyConfig("SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS");
properties.setProperty(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "true");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("1970/01/01", hk2.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("1970/01/01", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("1970/01/01", keyGen.getPartitionPath(internalRow, baseRow.schema()));
}
@Test
public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");