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

@@ -225,15 +225,15 @@ public class TestHoodieAvroUtils {
rec.put("non_pii_col", "val1");
rec.put("pii_col", "val2");
Object rowKey = HoodieAvroUtils.getNestedFieldVal(rec, "_row_key", true);
Object rowKey = HoodieAvroUtils.getNestedFieldVal(rec, "_row_key", true, false);
assertEquals("key1", rowKey);
Object rowKeyNotExist = HoodieAvroUtils.getNestedFieldVal(rec, "fake_key", true);
Object rowKeyNotExist = HoodieAvroUtils.getNestedFieldVal(rec, "fake_key", true, false);
assertNull(rowKeyNotExist);
// Field does not exist
try {
HoodieAvroUtils.getNestedFieldVal(rec, "fake_key", false);
HoodieAvroUtils.getNestedFieldVal(rec, "fake_key", false, false);
} catch (Exception e) {
assertEquals("fake_key(Part -fake_key) field not found in record. Acceptable fields were :[timestamp, _row_key, non_pii_col, pii_col]",
e.getMessage());
@@ -241,7 +241,7 @@ public class TestHoodieAvroUtils {
// Field exist while value not
try {
HoodieAvroUtils.getNestedFieldVal(rec, "timestamp", false);
HoodieAvroUtils.getNestedFieldVal(rec, "timestamp", false, false);
} catch (Exception e) {
assertEquals("The value of timestamp can not be null", e.getMessage());
}
@@ -255,7 +255,7 @@ public class TestHoodieAvroUtils {
ByteBuffer byteBuffer = ByteBuffer.wrap(bigDecimal.unscaledValue().toByteArray());
rec.put("decimal_col", byteBuffer);
Object decimalCol = HoodieAvroUtils.getNestedFieldVal(rec, "decimal_col", true);
Object decimalCol = HoodieAvroUtils.getNestedFieldVal(rec, "decimal_col", true, false);
assertEquals(bigDecimal, decimalCol);
Object obj = rec.get(1);

View File

@@ -202,6 +202,10 @@ public final class SchemaTestUtil {
return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/timestamp-test-evolved.avsc"));
}
public static Schema getTimestampWithLogicalTypeSchema() throws IOException {
return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/timestamp-logical-type.avsc"));
}
public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber, String instantTime,
String fileId) throws IOException {
return generateAvroRecordFromJson(schema, recordNumber, instantTime, fileId, true);

View File

@@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
{
"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "field1", "type": ["null", "string"], "default": null},
{"name": "createTime", "type": ["null", {"type" : "long", "logicalType" : "timestamp-micros"}], "default": null}
]
}