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

@@ -59,6 +59,7 @@ import java.io.OutputStream;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.Timestamp;
import java.time.LocalDate;
import java.util.ArrayList;
import java.util.Arrays;
@@ -444,15 +445,15 @@ public class HoodieAvroUtils {
/**
* Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c
*/
public static String getNestedFieldValAsString(GenericRecord record, String fieldName, boolean returnNullIfNotFound) {
Object obj = getNestedFieldVal(record, fieldName, returnNullIfNotFound);
public static String getNestedFieldValAsString(GenericRecord record, String fieldName, boolean returnNullIfNotFound, boolean consistentLogicalTimestampEnabled) {
Object obj = getNestedFieldVal(record, fieldName, returnNullIfNotFound, consistentLogicalTimestampEnabled);
return StringUtils.objToString(obj);
}
/**
* Obtain value of the provided field, denoted by dot notation. e.g: a.b.c
*/
public static Object getNestedFieldVal(GenericRecord record, String fieldName, boolean returnNullIfNotFound) {
public static Object getNestedFieldVal(GenericRecord record, String fieldName, boolean returnNullIfNotFound, boolean consistentLogicalTimestampEnabled) {
String[] parts = fieldName.split("\\.");
GenericRecord valueNode = record;
int i = 0;
@@ -466,7 +467,7 @@ public class HoodieAvroUtils {
// return, if last part of name
if (i == parts.length - 1) {
Schema fieldSchema = valueNode.getSchema().getField(part).schema();
return convertValueForSpecificDataTypes(fieldSchema, val);
return convertValueForSpecificDataTypes(fieldSchema, val, consistentLogicalTimestampEnabled);
} else {
// VC: Need a test here
if (!(val instanceof GenericRecord)) {
@@ -510,7 +511,7 @@ public class HoodieAvroUtils {
* @param fieldValue avro field value
* @return field value either converted (for certain data types) or as it is.
*/
public static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object fieldValue) {
public static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object fieldValue, boolean consistentLogicalTimestampEnabled) {
if (fieldSchema == null) {
return fieldValue;
}
@@ -518,11 +519,11 @@ public class HoodieAvroUtils {
if (fieldSchema.getType() == Schema.Type.UNION) {
for (Schema schema : fieldSchema.getTypes()) {
if (schema.getType() != Schema.Type.NULL) {
return convertValueForAvroLogicalTypes(schema, fieldValue);
return convertValueForAvroLogicalTypes(schema, fieldValue, consistentLogicalTimestampEnabled);
}
}
}
return convertValueForAvroLogicalTypes(fieldSchema, fieldValue);
return convertValueForAvroLogicalTypes(fieldSchema, fieldValue, consistentLogicalTimestampEnabled);
}
/**
@@ -538,9 +539,13 @@ public class HoodieAvroUtils {
* @param fieldValue avro field value
* @return field value either converted (for certain data types) or as it is.
*/
private static Object convertValueForAvroLogicalTypes(Schema fieldSchema, Object fieldValue) {
private static Object convertValueForAvroLogicalTypes(Schema fieldSchema, Object fieldValue, boolean consistentLogicalTimestampEnabled) {
if (fieldSchema.getLogicalType() == LogicalTypes.date()) {
return LocalDate.ofEpochDay(Long.parseLong(fieldValue.toString()));
} else if (fieldSchema.getLogicalType() == LogicalTypes.timestampMillis() && consistentLogicalTimestampEnabled) {
return new Timestamp(Long.parseLong(fieldValue.toString()));
} else if (fieldSchema.getLogicalType() == LogicalTypes.timestampMicros() && consistentLogicalTimestampEnabled) {
return new Timestamp(Long.parseLong(fieldValue.toString()) / 1000);
} else if (fieldSchema.getLogicalType() instanceof LogicalTypes.Decimal) {
Decimal dc = (Decimal) fieldSchema.getLogicalType();
DecimalConversion decimalConversion = new DecimalConversion();
@@ -585,15 +590,15 @@ public class HoodieAvroUtils {
*/
public static Object getRecordColumnValues(HoodieRecord<? extends HoodieRecordPayload> record,
String[] columns,
Schema schema) {
Schema schema, boolean consistentLogicalTimestampEnabled) {
try {
GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
if (columns.length == 1) {
return HoodieAvroUtils.getNestedFieldVal(genericRecord, columns[0], true);
return HoodieAvroUtils.getNestedFieldVal(genericRecord, columns[0], true, consistentLogicalTimestampEnabled);
} else {
StringBuilder sb = new StringBuilder();
for (String col : columns) {
sb.append(HoodieAvroUtils.getNestedFieldValAsString(genericRecord, col, true));
sb.append(HoodieAvroUtils.getNestedFieldValAsString(genericRecord, col, true, consistentLogicalTimestampEnabled));
}
return sb.toString();
@@ -613,7 +618,7 @@ public class HoodieAvroUtils {
*/
public static Object getRecordColumnValues(HoodieRecord<? extends HoodieRecordPayload> record,
String[] columns,
SerializableSchema schema) {
return getRecordColumnValues(record, columns, schema.get());
SerializableSchema schema, boolean consistentLogicalTimestampEnabled) {
return getRecordColumnValues(record, columns, schema.get(), consistentLogicalTimestampEnabled);
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.model;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
@@ -87,7 +88,10 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
}
private static Option<Object> updateEventTime(GenericRecord record, Properties properties) {
return Option.ofNullable(getNestedFieldVal(record, properties.getProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY), true));
boolean consistentLogicalTimestampEnabled = Boolean.parseBoolean(properties.getProperty(
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()));
return Option.ofNullable(getNestedFieldVal(record, properties.getProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY), true, consistentLogicalTimestampEnabled));
}
@Override
@@ -110,10 +114,13 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
* NOTE: Deletes sent via EmptyHoodieRecordPayload and/or Delete operation type do not hit this code path
* and need to be dealt with separately.
*/
boolean consistentLogicalTimestampEnabled = Boolean.parseBoolean(properties.getProperty(
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()));
Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue,
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true);
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true, consistentLogicalTimestampEnabled);
Comparable incomingOrderingVal = (Comparable) getNestedFieldVal((GenericRecord) incomingRecord,
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true);
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true, consistentLogicalTimestampEnabled);
return persistedOrderingVal == null || ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) <= 0;
}
}

View File

@@ -32,6 +32,7 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
protected List<String> partitionPathFields;
protected final boolean encodePartitionPath;
protected final boolean hiveStylePartitioning;
protected final boolean consistentLogicalTimestampEnabled;
protected BaseKeyGenerator(TypedProperties config) {
super(config);
@@ -39,6 +40,8 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING.defaultValue()));
this.hiveStylePartitioning = config.getBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(),
Boolean.parseBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.defaultValue()));
this.consistentLogicalTimestampEnabled = config.getBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()));
}
/**
@@ -78,4 +81,8 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
public List<String> getPartitionPathFields() {
return partitionPathFields;
}
public boolean isConsistentLogicalTimestampEnabled() {
return consistentLogicalTimestampEnabled;
}
}

View File

@@ -56,6 +56,16 @@ public class KeyGeneratorOptions extends HoodieConfig {
.withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
+ "Actual value ontained by invoking .toString()");
public static final ConfigProperty<String> KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED = ConfigProperty
.key("hoodie.datasource.write.keygenerator.consistent.logical.timestamp.enabled")
.defaultValue("false")
.withDocumentation("When set to true, consistent value will be generated for a logical timestamp type column, "
+ "like timestamp-millis and timestamp-micros, irrespective of whether row-writer is enabled. Disabled by default so "
+ "as not to break the pipeline that deploy either fully row-writer path or non row-writer path. For example, "
+ "if it is kept disabled then record key of timestamp type with value `2016-12-29 09:54:00` will be written as timestamp "
+ "`2016-12-29 09:54:00.0` in row-writer path, while it will be written as long value `1483023240000000` in non row-writer path. "
+ "If enabled, then the timestamp value will be written in both the cases.");
/**
* @deprecated Use {@link #URL_ENCODE_PARTITIONING} and its methods.
*/