Skip to content

Commit 8ffcb2f

Browse files
authored
[HUDI-4782] Support TIMESTAMP_LTZ type for flink (#6607)
1 parent d2d1cb8 commit 8ffcb2f

9 files changed

Lines changed: 125 additions & 16 deletions

File tree

hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java

Lines changed: 26 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@
2828
import org.apache.flink.table.types.logical.ArrayType;
2929
import org.apache.flink.table.types.logical.DecimalType;
3030
import org.apache.flink.table.types.logical.IntType;
31+
import org.apache.flink.table.types.logical.LocalZonedTimestampType;
3132
import org.apache.flink.table.types.logical.LogicalType;
3233
import org.apache.flink.table.types.logical.LogicalTypeFamily;
3334
import org.apache.flink.table.types.logical.MapType;
@@ -149,8 +150,12 @@ public static DataType convertToDataType(Schema schema) {
149150
// logical timestamp type
150151
if (schema.getLogicalType() == LogicalTypes.timestampMillis()) {
151152
return DataTypes.TIMESTAMP(3).notNull();
153+
} else if (schema.getLogicalType() == LogicalTypes.localTimestampMillis()) {
154+
return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull();
152155
} else if (schema.getLogicalType() == LogicalTypes.timestampMicros()) {
153156
return DataTypes.TIMESTAMP(6).notNull();
157+
} else if (schema.getLogicalType() == LogicalTypes.localTimestampMicros()) {
158+
return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6).notNull();
154159
} else if (schema.getLogicalType() == LogicalTypes.timeMillis()) {
155160
return DataTypes.TIME(3).notNull();
156161
} else if (schema.getLogicalType() == LogicalTypes.timeMicros()) {
@@ -242,19 +247,36 @@ public static Schema convertToSchema(LogicalType logicalType, String rowName) {
242247
// use long to represents Timestamp
243248
final TimestampType timestampType = (TimestampType) logicalType;
244249
precision = timestampType.getPrecision();
245-
org.apache.avro.LogicalType avroLogicalType;
250+
org.apache.avro.LogicalType timestampLogicalType;
246251
if (precision <= 3) {
247-
avroLogicalType = LogicalTypes.timestampMillis();
252+
timestampLogicalType = LogicalTypes.timestampMillis();
248253
} else if (precision <= 6) {
249-
avroLogicalType = LogicalTypes.timestampMicros();
254+
timestampLogicalType = LogicalTypes.timestampMicros();
250255
} else {
251256
throw new IllegalArgumentException(
252257
"Avro does not support TIMESTAMP type with precision: "
253258
+ precision
254259
+ ", it only supports precision less than 6.");
255260
}
256-
Schema timestamp = avroLogicalType.addToSchema(SchemaBuilder.builder().longType());
261+
Schema timestamp = timestampLogicalType.addToSchema(SchemaBuilder.builder().longType());
257262
return nullable ? nullableSchema(timestamp) : timestamp;
263+
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
264+
// use long to represents LocalZonedTimestampType
265+
final LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) logicalType;
266+
precision = localZonedTimestampType.getPrecision();
267+
org.apache.avro.LogicalType localZonedTimestampLogicalType;
268+
if (precision <= 3) {
269+
localZonedTimestampLogicalType = LogicalTypes.localTimestampMillis();
270+
} else if (precision <= 6) {
271+
localZonedTimestampLogicalType = LogicalTypes.localTimestampMicros();
272+
} else {
273+
throw new IllegalArgumentException(
274+
"Avro does not support LOCAL TIMESTAMP type with precision: "
275+
+ precision
276+
+ ", it only supports precision less than 6.");
277+
}
278+
Schema localZonedTimestamp = localZonedTimestampLogicalType.addToSchema(SchemaBuilder.builder().longType());
279+
return nullable ? nullableSchema(localZonedTimestamp) : localZonedTimestamp;
258280
case DATE:
259281
// use int to represents Date
260282
Schema date = LogicalTypes.date().addToSchema(SchemaBuilder.builder().intType());
@@ -319,7 +341,6 @@ public static Schema convertToSchema(LogicalType logicalType, String rowName) {
319341
.items(convertToSchema(arrayType.getElementType(), rowName));
320342
return nullable ? nullableSchema(array) : array;
321343
case RAW:
322-
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
323344
default:
324345
throw new UnsupportedOperationException(
325346
"Unsupported to derive Schema for type: " + logicalType);

hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import org.apache.flink.table.data.TimestampData;
3333
import org.apache.flink.table.types.logical.ArrayType;
3434
import org.apache.flink.table.types.logical.DecimalType;
35+
import org.apache.flink.table.types.logical.LocalZonedTimestampType;
3536
import org.apache.flink.table.types.logical.LogicalType;
3637
import org.apache.flink.table.types.logical.RowType;
3738
import org.apache.flink.table.types.logical.TimestampType;
@@ -127,6 +128,8 @@ public static AvroToRowDataConverter createConverter(LogicalType type) {
127128
return AvroToRowDataConverters::convertToDate;
128129
case TIME_WITHOUT_TIME_ZONE:
129130
return AvroToRowDataConverters::convertToTime;
131+
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
132+
return createTimestampConverter(((LocalZonedTimestampType) type).getPrecision());
130133
case TIMESTAMP_WITHOUT_TIME_ZONE:
131134
return createTimestampConverter(((TimestampType) type).getPrecision());
132135
case CHAR:

hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java

Lines changed: 8 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,8 @@
1818

1919
package org.apache.hudi.util;
2020

21-
import org.apache.hudi.common.util.ValidationUtils;
22-
2321
import org.apache.flink.table.types.DataType;
22+
import org.apache.flink.table.types.logical.LocalZonedTimestampType;
2423
import org.apache.flink.table.types.logical.LogicalType;
2524
import org.apache.flink.table.types.logical.LogicalTypeFamily;
2625
import org.apache.flink.table.types.logical.LogicalTypeRoot;
@@ -47,9 +46,13 @@ public static boolean isTimestampType(DataType type) {
4746
* Returns the precision of the given TIMESTAMP type.
4847
*/
4948
public static int precision(LogicalType logicalType) {
50-
ValidationUtils.checkArgument(logicalType instanceof TimestampType);
51-
TimestampType timestampType = (TimestampType) logicalType;
52-
return timestampType.getPrecision();
49+
if (logicalType instanceof TimestampType) {
50+
return ((TimestampType) logicalType).getPrecision();
51+
} else if (logicalType instanceof LocalZonedTimestampType) {
52+
return ((LocalZonedTimestampType) logicalType).getPrecision();
53+
} else {
54+
throw new AssertionError("Unexpected type: " + logicalType);
55+
}
5356
}
5457

5558
/**

hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@
3232
import org.apache.flink.table.types.logical.ArrayType;
3333
import org.apache.flink.table.types.logical.LogicalType;
3434
import org.apache.flink.table.types.logical.RowType;
35-
import org.apache.flink.table.types.logical.TimestampType;
3635

3736
import java.io.Serializable;
3837
import java.math.BigDecimal;
@@ -157,8 +156,9 @@ public Object convert(Schema schema, Object object) {
157156
}
158157
};
159158
break;
159+
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
160160
case TIMESTAMP_WITHOUT_TIME_ZONE:
161-
final int precision = ((TimestampType) type).getPrecision();
161+
final int precision = DataTypeUtils.precision(type);
162162
if (precision <= 3) {
163163
converter =
164164
new RowDataToAvroConverter() {
@@ -231,7 +231,7 @@ public Object convert(Schema schema, Object object) {
231231
actualSchema = types.get(1);
232232
} else {
233233
throw new IllegalArgumentException(
234-
"The Avro schema is not a nullable type: " + schema.toString());
234+
"The Avro schema is not a nullable type: " + schema);
235235
}
236236
} else {
237237
actualSchema = schema;

hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,7 @@
5656
import org.junit.jupiter.params.provider.ValueSource;
5757

5858
import java.io.File;
59+
import java.time.ZoneId;
5960
import java.util.Collection;
6061
import java.util.HashMap;
6162
import java.util.List;
@@ -1438,6 +1439,37 @@ void testWriteReadWithComputedColumns() {
14381439
assertRowsEquals(result2, "[+I[3]]");
14391440
}
14401441

1442+
@ParameterizedTest
1443+
@EnumSource(value = HoodieTableType.class)
1444+
void testWriteReadWithLocalTimestamp(HoodieTableType tableType) {
1445+
TableEnvironment tableEnv = batchTableEnv;
1446+
tableEnv.getConfig().setLocalTimeZone(ZoneId.of("Asia/Shanghai"));
1447+
String createTable = sql("t1")
1448+
.field("f0 int")
1449+
.field("f1 varchar(10)")
1450+
.field("f2 TIMESTAMP_LTZ(3)")
1451+
.field("f4 TIMESTAMP_LTZ(6)")
1452+
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
1453+
.option(FlinkOptions.PRECOMBINE_FIELD, "f1")
1454+
.option(FlinkOptions.TABLE_TYPE, tableType)
1455+
.pkField("f0")
1456+
.noPartition()
1457+
.end();
1458+
tableEnv.executeSql(createTable);
1459+
1460+
String insertInto = "insert into t1 values\n"
1461+
+ "(1, 'abc', TIMESTAMP '1970-01-01 08:00:01', TIMESTAMP '1970-01-01 08:00:02'),\n"
1462+
+ "(2, 'def', TIMESTAMP '1970-01-01 08:00:03', TIMESTAMP '1970-01-01 08:00:04')";
1463+
execInsertSql(tableEnv, insertInto);
1464+
1465+
List<Row> result = CollectionUtil.iterableToList(
1466+
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
1467+
final String expected = "["
1468+
+ "+I[1, abc, 1970-01-01T00:00:01Z, 1970-01-01T00:00:02Z], "
1469+
+ "+I[2, def, 1970-01-01T00:00:03Z, 1970-01-01T00:00:04Z]]";
1470+
assertRowsEquals(result, expected);
1471+
}
1472+
14411473
// -------------------------------------------------------------------------
14421474
// Utilities
14431475
// -------------------------------------------------------------------------

hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestAvroSchemaConverter.java

Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import org.apache.hudi.util.AvroSchemaConverter;
2424

2525
import org.apache.avro.Schema;
26+
import org.apache.flink.table.api.DataTypes;
2627
import org.apache.flink.table.types.DataType;
2728
import org.junit.jupiter.api.Test;
2829

@@ -50,4 +51,41 @@ void testUnionSchemaWithMultipleRecordTypes() {
5051
+ "`isDeleted` BOOLEAN NOT NULL>";
5152
assertThat(dataType.getChildren().get(pos).toString(), is(expected));
5253
}
54+
55+
@Test
56+
void testLocalTimestampType() {
57+
DataType dataType = DataTypes.ROW(
58+
DataTypes.FIELD("f_localtimestamp_millis", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)),
59+
DataTypes.FIELD("f_localtimestamp_micros", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6))
60+
);
61+
// convert to avro schema
62+
Schema schema = AvroSchemaConverter.convertToSchema(dataType.getLogicalType());
63+
final String expectedSchema = ""
64+
+ "[ \"null\", {\n"
65+
+ " \"type\" : \"record\",\n"
66+
+ " \"name\" : \"record\",\n"
67+
+ " \"fields\" : [ {\n"
68+
+ " \"name\" : \"f_localtimestamp_millis\",\n"
69+
+ " \"type\" : [ \"null\", {\n"
70+
+ " \"type\" : \"long\",\n"
71+
+ " \"logicalType\" : \"local-timestamp-millis\"\n"
72+
+ " } ],\n"
73+
+ " \"default\" : null\n"
74+
+ " }, {\n"
75+
+ " \"name\" : \"f_localtimestamp_micros\",\n"
76+
+ " \"type\" : [ \"null\", {\n"
77+
+ " \"type\" : \"long\",\n"
78+
+ " \"logicalType\" : \"local-timestamp-micros\"\n"
79+
+ " } ],\n"
80+
+ " \"default\" : null\n"
81+
+ " } ]\n"
82+
+ "} ]";
83+
assertThat(schema.toString(true), is(expectedSchema));
84+
// convert it back
85+
DataType convertedDataType = AvroSchemaConverter.convertToDataType(schema);
86+
final String expectedDataType = "ROW<"
87+
+ "`f_localtimestamp_millis` TIMESTAMP_LTZ(3), "
88+
+ "`f_localtimestamp_micros` TIMESTAMP_LTZ(6)>";
89+
assertThat(convertedDataType.toString(), is(expectedDataType));
90+
}
5391
}

hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,7 @@
5959
import org.apache.flink.table.types.logical.ArrayType;
6060
import org.apache.flink.table.types.logical.DecimalType;
6161
import org.apache.flink.table.types.logical.IntType;
62+
import org.apache.flink.table.types.logical.LocalZonedTimestampType;
6263
import org.apache.flink.table.types.logical.LogicalType;
6364
import org.apache.flink.table.types.logical.MapType;
6465
import org.apache.flink.table.types.logical.RowType;
@@ -334,7 +335,10 @@ private static ColumnReader createColumnReader(
334335
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
335336
switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) {
336337
case INT64:
337-
return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, ((TimestampType) fieldType).getPrecision());
338+
int precision = fieldType instanceof TimestampType
339+
? ((TimestampType) fieldType).getPrecision()
340+
: ((LocalZonedTimestampType) fieldType).getPrecision();
341+
return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, precision);
338342
case INT96:
339343
return new TimestampColumnReader(utcTimestamp, descriptor, pageReader);
340344
default:

hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,7 @@
5959
import org.apache.flink.table.types.logical.ArrayType;
6060
import org.apache.flink.table.types.logical.DecimalType;
6161
import org.apache.flink.table.types.logical.IntType;
62+
import org.apache.flink.table.types.logical.LocalZonedTimestampType;
6263
import org.apache.flink.table.types.logical.LogicalType;
6364
import org.apache.flink.table.types.logical.MapType;
6465
import org.apache.flink.table.types.logical.RowType;
@@ -334,7 +335,10 @@ private static ColumnReader createColumnReader(
334335
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
335336
switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) {
336337
case INT64:
337-
return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, ((TimestampType) fieldType).getPrecision());
338+
int precision = fieldType instanceof TimestampType
339+
? ((TimestampType) fieldType).getPrecision()
340+
: ((LocalZonedTimestampType) fieldType).getPrecision();
341+
return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, precision);
338342
case INT96:
339343
return new TimestampColumnReader(utcTimestamp, descriptor, pageReader);
340344
default:

hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,7 @@
5959
import org.apache.flink.table.types.logical.ArrayType;
6060
import org.apache.flink.table.types.logical.DecimalType;
6161
import org.apache.flink.table.types.logical.IntType;
62+
import org.apache.flink.table.types.logical.LocalZonedTimestampType;
6263
import org.apache.flink.table.types.logical.LogicalType;
6364
import org.apache.flink.table.types.logical.MapType;
6465
import org.apache.flink.table.types.logical.RowType;
@@ -334,7 +335,10 @@ private static ColumnReader createColumnReader(
334335
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
335336
switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) {
336337
case INT64:
337-
return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, ((TimestampType) fieldType).getPrecision());
338+
int precision = fieldType instanceof TimestampType
339+
? ((TimestampType) fieldType).getPrecision()
340+
: ((LocalZonedTimestampType) fieldType).getPrecision();
341+
return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, precision);
338342
case INT96:
339343
return new TimestampColumnReader(utcTimestamp, descriptor, pageReader);
340344
default:

0 commit comments

Comments
 (0)