[HUDI-3184] hudi-flink support timestamp-micros (#4548)
* support both avro and parquet code path * string rowdata conversion is also supported
This commit is contained in:
@@ -545,6 +545,42 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
assertRowsEquals(result2, expected);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = ExecMode.class)
|
||||
void testWriteAndReadWithTimestampMicros(ExecMode execMode) throws Exception {
|
||||
boolean streaming = execMode == ExecMode.STREAM;
|
||||
String hoodieTableDDL = sql("t1")
|
||||
.field("id int")
|
||||
.field("name varchar(10)")
|
||||
.field("ts timestamp(6)")
|
||||
.pkField("id")
|
||||
.noPartition()
|
||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||
.option(FlinkOptions.READ_AS_STREAMING, streaming)
|
||||
.end();
|
||||
streamTableEnv.executeSql(hoodieTableDDL);
|
||||
String insertInto = "insert into t1 values\n"
|
||||
+ "(1,'Danny',TIMESTAMP '2021-12-01 01:02:01.100001'),\n"
|
||||
+ "(2,'Stephen',TIMESTAMP '2021-12-02 03:04:02.200002'),\n"
|
||||
+ "(3,'Julian',TIMESTAMP '2021-12-03 13:14:03.300003'),\n"
|
||||
+ "(4,'Fabian',TIMESTAMP '2021-12-04 15:16:04.400004')";
|
||||
execInsertSql(streamTableEnv, insertInto);
|
||||
|
||||
final String expected = "["
|
||||
+ "+I[1, Danny, 2021-12-01T01:02:01.100001], "
|
||||
+ "+I[2, Stephen, 2021-12-02T03:04:02.200002], "
|
||||
+ "+I[3, Julian, 2021-12-03T13:14:03.300003], "
|
||||
+ "+I[4, Fabian, 2021-12-04T15:16:04.400004]]";
|
||||
|
||||
List<Row> result = execSelectSql(streamTableEnv, "select * from t1", execMode);
|
||||
assertRowsEquals(result, expected);
|
||||
|
||||
// insert another batch of data
|
||||
execInsertSql(streamTableEnv, insertInto);
|
||||
List<Row> result2 = execSelectSql(streamTableEnv, "select * from t1", execMode);
|
||||
assertRowsEquals(result2, expected);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = ExecMode.class)
|
||||
void testInsertOverwrite(ExecMode execMode) {
|
||||
|
||||
@@ -49,13 +49,14 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
public class TestStringToRowDataConverter {
|
||||
@Test
|
||||
void testConvert() {
|
||||
String[] fields = new String[] {"1.1", "3.4", "2021-03-30", "56669000", "1617119069000", "12345.67"};
|
||||
String[] fields = new String[] {"1.1", "3.4", "2021-03-30", "56669000", "1617119069000", "1617119069666111", "12345.67"};
|
||||
LogicalType[] fieldTypes = new LogicalType[] {
|
||||
DataTypes.FLOAT().getLogicalType(),
|
||||
DataTypes.DOUBLE().getLogicalType(),
|
||||
DataTypes.DATE().getLogicalType(),
|
||||
DataTypes.TIME(3).getLogicalType(),
|
||||
DataTypes.TIMESTAMP().getLogicalType(),
|
||||
DataTypes.TIMESTAMP(3).getLogicalType(),
|
||||
DataTypes.TIMESTAMP(6).getLogicalType(),
|
||||
DataTypes.DECIMAL(7, 2).getLogicalType()
|
||||
};
|
||||
StringToRowDataConverter converter = new StringToRowDataConverter(fieldTypes);
|
||||
@@ -63,7 +64,8 @@ public class TestStringToRowDataConverter {
|
||||
Object[] expected = new Object[] {
|
||||
1.1f, 3.4D, (int) LocalDate.parse("2021-03-30").toEpochDay(),
|
||||
LocalTime.parse("15:44:29").get(ChronoField.MILLI_OF_DAY),
|
||||
TimestampData.fromEpochMillis(Instant.parse("2021-03-30T15:44:29Z").toEpochMilli()),
|
||||
TimestampData.fromInstant(Instant.parse("2021-03-30T15:44:29Z")),
|
||||
TimestampData.fromInstant(Instant.parse("2021-03-30T15:44:29.666111Z")),
|
||||
DecimalData.fromBigDecimal(new BigDecimal("12345.67"), 7, 2)
|
||||
};
|
||||
assertArrayEquals(expected, converted);
|
||||
@@ -71,13 +73,14 @@ public class TestStringToRowDataConverter {
|
||||
|
||||
@Test
|
||||
void testRowDataToAvroStringToRowData() {
|
||||
GenericRowData rowData = new GenericRowData(6);
|
||||
GenericRowData rowData = new GenericRowData(7);
|
||||
rowData.setField(0, 1.1f);
|
||||
rowData.setField(1, 3.4D);
|
||||
rowData.setField(2, (int) LocalDate.parse("2021-03-30").toEpochDay());
|
||||
rowData.setField(3, LocalTime.parse("15:44:29").get(ChronoField.MILLI_OF_DAY));
|
||||
rowData.setField(4, TimestampData.fromEpochMillis(Instant.parse("2021-03-30T15:44:29Z").toEpochMilli()));
|
||||
rowData.setField(5, DecimalData.fromBigDecimal(new BigDecimal("12345.67"), 7, 2));
|
||||
rowData.setField(4, TimestampData.fromInstant(Instant.parse("2021-03-30T15:44:29Z")));
|
||||
rowData.setField(5, TimestampData.fromInstant(Instant.parse("2021-03-30T15:44:29.666111Z")));
|
||||
rowData.setField(6, DecimalData.fromBigDecimal(new BigDecimal("12345.67"), 7, 2));
|
||||
|
||||
DataType dataType = DataTypes.ROW(
|
||||
DataTypes.FIELD("f_float", DataTypes.FLOAT()),
|
||||
@@ -85,6 +88,7 @@ public class TestStringToRowDataConverter {
|
||||
DataTypes.FIELD("f_date", DataTypes.DATE()),
|
||||
DataTypes.FIELD("f_time", DataTypes.TIME(3)),
|
||||
DataTypes.FIELD("f_timestamp", DataTypes.TIMESTAMP(3)),
|
||||
DataTypes.FIELD("f_timestamp_micros", DataTypes.TIMESTAMP(6)),
|
||||
DataTypes.FIELD("f_decimal", DataTypes.DECIMAL(7, 2))
|
||||
);
|
||||
RowType rowType = (RowType) dataType.getLogicalType();
|
||||
@@ -98,8 +102,8 @@ public class TestStringToRowDataConverter {
|
||||
final String[] recordKeys = KeyGenUtils.extractRecordKeys(recordKey);
|
||||
Object[] convertedKeys = stringToRowDataConverter.convert(recordKeys);
|
||||
|
||||
GenericRowData converted = new GenericRowData(6);
|
||||
for (int i = 0; i < 6; i++) {
|
||||
GenericRowData converted = new GenericRowData(7);
|
||||
for (int i = 0; i < 7; i++) {
|
||||
converted.setField(i, convertedKeys[i]);
|
||||
}
|
||||
assertThat(converted, is(rowData));
|
||||
|
||||
Reference in New Issue
Block a user