From aa96a7752b42bf8fdd4e493da24753c917c5135b Mon Sep 17 00:00:00 2001 From: wuzexian Date: Fri, 18 Jul 2025 17:12:52 +0800 Subject: [PATCH] Add Pipeline support for DateType and TimeType --- .../flink/cdc/common/data/DateData.java | 78 +++++++ .../flink/cdc/common/data/RecordData.java | 12 +- .../flink/cdc/common/data/TimeData.java | 90 ++++++++ .../common/data/binary/BinaryRecordData.java | 14 ++ .../cdc/common/types/utils/DataTypeUtils.java | 6 +- .../flink/cdc/common/utils/DateTimeUtils.java | 17 +- .../cdc/common/utils/SchemaMergingUtils.java | 208 ++++++++++++++++-- .../common/utils/SchemaMergingUtilsTest.java | 15 +- .../FlinkParallelizedPipelineITCase.java | 6 +- .../flink/FlinkPipelineTransformITCase.java | 119 +++++++++- .../source/DistributedSourceFunction.java | 3 +- .../test/resources/ref-output/distributed.txt | 64 +++--- .../src/test/resources/ref-output/regular.txt | 64 +++--- .../doris/sink/DorisRowConverter.java | 7 +- .../doris/sink/DorisEventSerializerTest.java | 7 +- .../doris/sink/DorisRowConverterTest.java | 10 +- .../serializer/ElasticsearchRowConverter.java | 3 +- .../ElasticsearchEventSerializerTest.java | 3 +- .../FlussEventSerializationSchemaTest.java | 5 +- .../iceberg/sink/utils/IcebergTypeUtils.java | 6 +- .../sink/v2/CompactionOperatorTest.java | 3 +- .../iceberg/sink/v2/IcebergWriterTest.java | 35 ++- .../kafka/json/TableSchemaInfo.java | 6 +- .../kafka/json/TableSchemaInfoTest.java | 8 +- .../DebeziumJsonSerializationSchemaTest.java | 8 +- .../maxcompute/utils/TypeConvertUtils.java | 5 +- .../utils/TypeConvertUtilsTest.java | 8 +- .../source/MySqlAncientDateAndTimeITCase.java | 56 ++--- .../mysql/source/MySqlFullTypesITCase.java | 50 ++--- .../oceanbase/sink/OceanBaseRowConvert.java | 7 +- ...OceanBaseEventSerializationSchemaTest.java | 5 +- .../paimon/sink/v2/PaimonWriterHelper.java | 6 +- .../sink/v2/PaimonWriterHelperTest.java | 8 +- .../starrocks/sink/StarRocksUtils.java | 5 +- .../EventRecordSerializationSchemaTest.java | 5 +- .../DebeziumEventDeserializationSchema.java | 12 +- .../DebeziumSchemaDataTypeInference.java | 4 +- .../pipeline/tests/TransformE2eITCase.java | 30 +-- .../functions/SystemFunctionUtils.java | 12 +- .../serializer/InternalSerializers.java | 6 +- .../serializer/data/DateDataSerializer.java | 97 ++++++++ .../serializer/data/TimeDataSerializer.java | 97 ++++++++ .../data/writer/AbstractBinaryWriter.java | 12 + .../serializer/data/writer/BinaryWriter.java | 25 ++- .../runtime/typeutils/DataTypeConverter.java | 31 ++- .../transform/PostTransformOperatorTest.java | 24 +- .../data/DateDataSerializerTest.java | 56 +++++ .../data/TimeDataSerializerTest.java | 62 ++++++ .../BinaryRecordDataExtractorTest.java | 30 +-- .../BinaryRecordDataGeneratorTest.java | 43 ++-- 50 files changed, 1190 insertions(+), 303 deletions(-) create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/DateData.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/TimeData.java create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DateDataSerializer.java create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimeDataSerializer.java create mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/data/DateDataSerializerTest.java create mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/data/TimeDataSerializerTest.java diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/DateData.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/DateData.java new file mode 100644 index 00000000000..085f6a975c3 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/DateData.java @@ -0,0 +1,78 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.common.data; + +import java.time.LocalDate; +import java.util.Objects; + +/** + * An internal data structure representing data of {@link + * org.apache.flink.cdc.common.types.DateType}. + */ +public class DateData implements Comparable { + + private final int epochDay; + + private DateData(int epochDay) { + this.epochDay = epochDay; + } + + public static DateData fromEpochDay(int epochOfDay) { + return new DateData(epochOfDay); + } + + public static DateData fromLocalDate(LocalDate date) { + return fromEpochDay((int) date.toEpochDay()); + } + + public static DateData fromIsoLocalDateString(String dateString) { + return fromLocalDate(LocalDate.parse(dateString)); + } + + public int toEpochDay() { + return epochDay; + } + + public LocalDate toLocalDate() { + return LocalDate.ofEpochDay(epochDay); + } + + public String toString() { + return toLocalDate().toString(); + } + + @Override + public final boolean equals(Object o) { + if (!(o instanceof DateData)) { + return false; + } + + DateData dateData = (DateData) o; + return epochDay == dateData.epochDay; + } + + @Override + public int compareTo(DateData other) { + return Long.compare(epochDay, other.epochDay); + } + + @Override + public int hashCode() { + return Objects.hash(epochDay); + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/RecordData.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/RecordData.java index c3aa97b65bb..4bfd20e2e47 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/RecordData.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/RecordData.java @@ -163,6 +163,12 @@ public interface RecordData { */ RecordData getRow(int pos, int numFields); + /** Returns the Date data at the given position. */ + DateData getDate(int pos); + + /** Returns the Time data at the given position. */ + TimeData getTime(int pos); + /** * Creates an accessor for getting elements in an internal RecordData structure at the given * position. @@ -197,9 +203,13 @@ static RecordData.FieldGetter createFieldGetter(DataType fieldType, int fieldPos fieldGetter = record -> record.getShort(fieldPos); break; case INTEGER: + fieldGetter = record -> record.getInt(fieldPos); + break; case DATE: + fieldGetter = record -> record.getDate(fieldPos); + break; case TIME_WITHOUT_TIME_ZONE: - fieldGetter = record -> record.getInt(fieldPos); + fieldGetter = record -> record.getTime(fieldPos); break; case BIGINT: fieldGetter = record -> record.getLong(fieldPos); diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/TimeData.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/TimeData.java new file mode 100644 index 00000000000..5570864bdc0 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/TimeData.java @@ -0,0 +1,90 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.common.data; + +import java.time.LocalTime; +import java.util.Objects; + +/** + * An internal data structure representing data of {@link + * org.apache.flink.cdc.common.types.TimeType}. + */ +public class TimeData implements Comparable { + + private static final int SECONDS_TO_MILLIS = 1000; + private static final int MILLIS_TO_NANO = 1_000_000; + + private final int millisOfDay; + + private TimeData(int millisOfDay) { + this.millisOfDay = millisOfDay; + } + + public static TimeData fromSecondOfDay(int secondOfDay) { + return new TimeData(secondOfDay * SECONDS_TO_MILLIS); + } + + public static TimeData fromMillisOfDay(int millisOfDay) { + return new TimeData(millisOfDay); + } + + public static TimeData fromNanoOfDay(long nanoOfDay) { + // millisOfDay should not exceed 86400000, which is safe to fit into INT. + return new TimeData((int) (nanoOfDay / MILLIS_TO_NANO)); + } + + public static TimeData fromLocalTime(LocalTime localTime) { + return fromNanoOfDay(localTime.toNanoOfDay()); + } + + public static TimeData fromIsoLocalTimeString(String timeString) { + return fromLocalTime(LocalTime.parse(timeString)); + } + + public int toMillisOfDay() { + return millisOfDay; + } + + public LocalTime toLocalTime() { + return LocalTime.ofNanoOfDay((long) millisOfDay * MILLIS_TO_NANO); + } + + public String toString() { + return toLocalTime().toString(); + } + + @Override + public final boolean equals(Object o) { + if (!(o instanceof TimeData)) { + return false; + } + + TimeData timeData = (TimeData) o; + return millisOfDay == timeData.millisOfDay; + } + + @Override + public int compareTo(TimeData other) { + return Long.compare(millisOfDay, other.millisOfDay); + } + + @Override + public int hashCode() { + return Objects.hash(millisOfDay); + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/binary/BinaryRecordData.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/binary/BinaryRecordData.java index 2cd4bc1f54b..5fe100f2263 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/binary/BinaryRecordData.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/data/binary/BinaryRecordData.java @@ -19,11 +19,13 @@ import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.data.ArrayData; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.MapData; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.data.StringData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.utils.Preconditions; @@ -216,6 +218,18 @@ public RecordData getRow(int pos, int numFields) { return BinarySegmentUtils.readRecordData(segments, numFields, offset, getLong(pos)); } + @Override + public DateData getDate(int pos) { + assertIndexIsValid(pos); + return DateData.fromEpochDay(getInt(pos)); + } + + @Override + public TimeData getTime(int pos) { + assertIndexIsValid(pos); + return TimeData.fromMillisOfDay(getInt(pos)); + } + /** The bit is 1 when the field is null. Default is 0. */ @Override public boolean anyNull() { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/types/utils/DataTypeUtils.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/types/utils/DataTypeUtils.java index 4db049baf34..ef2a8087861 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/types/utils/DataTypeUtils.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/types/utils/DataTypeUtils.java @@ -18,10 +18,12 @@ package org.apache.flink.cdc.common.types.utils; import org.apache.flink.cdc.common.data.ArrayData; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.MapData; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.data.StringData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.types.DataField; @@ -58,9 +60,11 @@ public static Class toInternalConversionClass(DataType type) { case SMALLINT: return Short.class; case INTEGER: + return Integer.class; case DATE: + return DateData.class; case TIME_WITHOUT_TIME_ZONE: - return Integer.class; + return TimeData.class; case BIGINT: return Long.class; case FLOAT: diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/DateTimeUtils.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/DateTimeUtils.java index bb01e4a8e8b..5cc65de44b7 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/DateTimeUtils.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/DateTimeUtils.java @@ -17,6 +17,9 @@ package org.apache.flink.cdc.common.utils; +import org.apache.flink.cdc.common.data.DateData; +import org.apache.flink.cdc.common.data.TimeData; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,12 +74,12 @@ public class DateTimeUtils { * @param ts the timestamp in milliseconds. * @return the date in days. */ - public static int timestampMillisToDate(long ts) { - int days = (int) (ts / MILLIS_PER_DAY); + public static DateData timestampMillisToDate(long ts) { + long days = ts / MILLIS_PER_DAY; if (days < 0) { days = days - 1; } - return days; + return DateData.fromEpochDay((int) days); } /** @@ -85,8 +88,8 @@ public static int timestampMillisToDate(long ts) { * @param ts the timestamp in milliseconds. * @return the time in milliseconds. */ - public static int timestampMillisToTime(long ts) { - return (int) (ts % MILLIS_PER_DAY); + public static TimeData timestampMillisToTime(long ts) { + return TimeData.fromMillisOfDay((int) (ts % MILLIS_PER_DAY)); } // -------------------------------------------------------------------------------------------- @@ -103,12 +106,12 @@ public static int parseDate(String dateStr, String fromFormat) { return ymdToUnixDate(zdt.getYear(), zdt.getMonthValue(), zdt.getDayOfMonth()); } - public static int parseDate(String dateStr, String fromFormat, String timezone) { + public static DateData parseDate(String dateStr, String fromFormat, String timezone) { long ts = internalParseTimestampMillis(dateStr, fromFormat, TimeZone.getTimeZone(timezone)); ZoneId zoneId = ZoneId.of(timezone); Instant instant = Instant.ofEpochMilli(ts); ZonedDateTime zdt = ZonedDateTime.ofInstant(instant, zoneId); - return ymdToUnixDate(zdt.getYear(), zdt.getMonthValue(), zdt.getDayOfMonth()); + return DateData.fromLocalDate(zdt.toLocalDate()); } private static long internalParseTimestampMillis(String dateStr, String format, TimeZone tz) { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaMergingUtils.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaMergingUtils.java index 1851f893522..6bb35e5fe5b 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaMergingUtils.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaMergingUtils.java @@ -20,9 +20,15 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.annotation.PublicEvolving; import org.apache.flink.cdc.common.annotation.VisibleForTesting; +import org.apache.flink.cdc.common.data.ArrayData; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; +import org.apache.flink.cdc.common.data.GenericArrayData; +import org.apache.flink.cdc.common.data.GenericMapData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.MapData; import org.apache.flink.cdc.common.data.StringData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; @@ -68,10 +74,12 @@ import java.math.BigDecimal; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.LocalTime; import java.time.ZoneId; import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -170,11 +178,11 @@ public static Schema getLeastCommonSchema( } /** Merge compatible schemas. */ - public static Schema getCommonSchema(List schemas) { + public static Schema getCommonSchema(Collection schemas) { if (schemas.isEmpty()) { return null; } else if (schemas.size() == 1) { - return schemas.get(0); + return schemas.iterator().next(); } else { Schema outputSchema = null; for (Schema schema : schemas) { @@ -184,6 +192,13 @@ public static Schema getCommonSchema(List schemas) { } } + /** Merge {@link DataType}s. */ + public static DataType getCommonDataTypes(List dataTypes) { + return dataTypes.stream() + .reduce(SchemaMergingUtils::getLeastCommonType) + .orElse(DataTypes.STRING()); + } + /** * Generating what schema change events we need to do by converting compatible {@code * beforeSchema} to {@code afterSchema}. @@ -250,7 +265,7 @@ public static Object[] coerceRow( String timezone, Schema currentSchema, Schema upcomingSchema, - List upcomingRow) { + Collection upcomingRow) { return coerceRow(timezone, currentSchema, upcomingSchema, upcomingRow, true); } @@ -263,7 +278,7 @@ public static Object[] coerceRow( String timezone, Schema currentSchema, Schema upcomingSchema, - List upcomingRow, + Collection upcomingRow, boolean toleranceMode) { List currentColumns = currentSchema.getColumns(); Map upcomingColumnTypes = @@ -454,6 +469,22 @@ static boolean isDataTypeCompatible(@Nullable DataType currentType, DataType upc return false; } + // For nested types, we check their inner types covariantly. + if (currentType instanceof ArrayType && upcomingType instanceof ArrayType) { + return isDataTypeCompatible( + ((ArrayType) currentType).getElementType(), + ((ArrayType) upcomingType).getElementType()); + } + + if (currentType instanceof MapType && upcomingType instanceof MapType) { + return isDataTypeCompatible( + ((MapType) currentType).getKeyType(), + ((MapType) upcomingType).getValueType()) + && isDataTypeCompatible( + ((MapType) currentType).getValueType(), + ((MapType) upcomingType).getValueType()); + } + // Or, check if upcomingType is presented in the type merging tree. return TYPE_MERGING_TREE.get(upcomingType.getClass()).contains(currentType); } @@ -478,6 +509,23 @@ static DataType getLeastCommonType(DataType currentType, DataType targetType) { return mergeDecimalType(currentType, targetType).copy(nullable); } + if (currentType instanceof ArrayType && targetType instanceof ArrayType) { + return DataTypes.ARRAY( + getLeastCommonType( + ((ArrayType) currentType).getElementType(), + ((ArrayType) targetType).getElementType())); + } + + if (currentType instanceof MapType && targetType instanceof MapType) { + return DataTypes.MAP( + getLeastCommonType( + ((MapType) currentType).getKeyType(), + ((MapType) targetType).getKeyType()), + getLeastCommonType( + ((MapType) currentType).getValueType(), + ((MapType) targetType).getValueType())); + } + List currentTypeTree = TYPE_MERGING_TREE.get(currentType.getClass()); List targetTypeTree = TYPE_MERGING_TREE.get(targetType.getClass()); @@ -664,12 +712,11 @@ static Object coerceObject( } if (destinationType instanceof DateType) { - try { - return coerceToLong(originalField); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException( - String.format("Cannot fit \"%s\" into a DATE column.", originalField)); - } + return coerceToDate(originalField); + } + + if (destinationType instanceof TimeType) { + return coerceToTime(originalField); } if (destinationType.is(DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) @@ -702,6 +749,19 @@ static Object coerceObject( return coerceToZonedTimestamp(originalField, timezone); } + if (originalType instanceof ArrayType && destinationType instanceof ArrayType) { + return coerceToArray( + timezone, + originalField, + ((ArrayType) originalType), + ((ArrayType) destinationType)); + } + + if (originalType instanceof MapType && destinationType instanceof MapType) { + return coerceToMap( + timezone, originalField, ((MapType) originalType), (MapType) destinationType); + } + throw new IllegalArgumentException( String.format( "Column type \"%s\" doesn't support type coercion to \"%s\"", @@ -713,15 +773,14 @@ private static Object coerceToString(Object originalField, DataType originalType return BinaryStringData.fromString("null"); } - if (originalType instanceof DateType) { - long epochOfDay = coerceToLong(originalField); - return BinaryStringData.fromString(LocalDate.ofEpochDay(epochOfDay).toString()); - } - if (originalField instanceof StringData) { return originalField; } + if (originalType instanceof DateType || originalType instanceof TimeType) { + return BinaryStringData.fromString(originalField.toString()); + } + if (originalField instanceof byte[]) { return BinaryStringData.fromString(hexlify((byte[]) originalField)); } @@ -864,6 +923,52 @@ private static double coerceToDouble(Object o) { } } + private static DateData coerceToDate(Object o) { + if (o == null) { + return null; + } + if (o instanceof DateData) { + return (DateData) o; + } + if (o instanceof Number) { + return DateData.fromEpochDay(((Number) o).intValue()); + } + if (o instanceof String) { + return DateData.fromIsoLocalDateString((String) o); + } + if (o instanceof LocalDate) { + return DateData.fromLocalDate((LocalDate) o); + } + if (o instanceof LocalDateTime) { + return DateData.fromLocalDate(((LocalDateTime) o).toLocalDate()); + } + throw new IllegalArgumentException( + String.format("Cannot fit type \"%s\" into a DATE column. ", o.getClass())); + } + + private static TimeData coerceToTime(Object o) { + if (o == null) { + return null; + } + if (o instanceof TimeData) { + return (TimeData) o; + } + if (o instanceof Number) { + return TimeData.fromNanoOfDay(((Number) o).longValue()); + } + if (o instanceof String) { + return TimeData.fromIsoLocalTimeString((String) o); + } + if (o instanceof LocalTime) { + return TimeData.fromLocalTime((LocalTime) o); + } + if (o instanceof LocalDateTime) { + return TimeData.fromLocalTime(((LocalDateTime) o).toLocalTime()); + } + throw new IllegalArgumentException( + String.format("Cannot fit type \"%s\" into a TIME column. ", o.getClass())); + } + private static TimestampData coerceToTimestamp(Object object, String timezone) { if (object == null) { return null; @@ -881,6 +986,9 @@ private static TimestampData coerceToTimestamp(Object object, String timezone) { ((ZonedTimestampData) object).toInstant(), ZoneId.of(timezone))); } else if (object instanceof TimestampData) { return (TimestampData) object; + } else if (object instanceof DateData) { + return TimestampData.fromLocalDateTime( + ((DateData) object).toLocalDate().atStartOfDay()); } else { throw new IllegalArgumentException( String.format( @@ -911,6 +1019,70 @@ private static ZonedTimestampData coerceToZonedTimestamp(Object object, String t ZoneId.of(timezone))); } + private static GenericArrayData coerceToArray( + String timezone, Object object, ArrayType originalType, ArrayType targetType) { + DataType originalElementType = originalType.getElementType(); + DataType targetElementType = targetType.getElementType(); + + if (!(object instanceof ArrayData)) { + throw new IllegalArgumentException( + "Unable to coerce given object: " + object + " to " + targetType); + } + ArrayData arrayData = (ArrayData) object; + ArrayData.ElementGetter elementGetter = ArrayData.createElementGetter(originalElementType); + + Object[] coerced = new Object[arrayData.size()]; + for (int i = 0; i < arrayData.size(); i++) { + coerced[i] = + coerceObject( + timezone, + elementGetter.getElementOrNull(arrayData, i), + originalElementType, + targetElementType); + } + + return new GenericArrayData(coerced); + } + + private static GenericMapData coerceToMap( + String timezone, Object object, MapType originalType, MapType targetType) { + DataType originalKeyType = originalType.getKeyType(); + DataType originalValueType = originalType.getValueType(); + + DataType targetKeyType = targetType.getKeyType(); + DataType targetValueType = targetType.getValueType(); + + if (!(object instanceof MapData)) { + throw new IllegalArgumentException( + "Unable to coerce given object: " + object + " to " + targetType); + } + MapData mapData = (MapData) object; + + Object[] keyArray = + coerceToArray( + timezone, + mapData.keyArray(), + DataTypes.ARRAY(originalKeyType), + DataTypes.ARRAY(targetKeyType)) + .toObjectArray(); + Object[] valueArray = + coerceToArray( + timezone, + mapData.valueArray(), + DataTypes.ARRAY(originalValueType), + DataTypes.ARRAY(targetValueType)) + .toObjectArray(); + + Preconditions.checkArgument(keyArray.length == valueArray.length); + + Map genericMapObjects = new HashMap<>(keyArray.length); + for (int i = 0; i < keyArray.length; i++) { + genericMapObjects.put(keyArray[i], valueArray[i]); + } + + return new GenericMapData(genericMapObjects); + } + private static String hexlify(byte[] bytes) { return BaseEncoding.base64().encode(bytes); } @@ -932,6 +1104,8 @@ private static Map, List> getTypeMergingTree DataType timestampLtzType = DataTypes.TIMESTAMP_LTZ(LocalZonedTimestampType.MAX_PRECISION); DataType timestampType = DataTypes.TIMESTAMP(TimestampType.MAX_PRECISION); DataType dateType = DataTypes.DATE(); + DataType arrayType = DataTypes.ARRAY(stringType); + DataType mapType = DataTypes.MAP(stringType, stringType); Map, List> mergingTree = new HashMap<>(); @@ -988,8 +1162,8 @@ private static Map, List> getTypeMergingTree // Complex types mergingTree.put(RowType.class, ImmutableList.of(stringType)); - mergingTree.put(ArrayType.class, ImmutableList.of(stringType)); - mergingTree.put(MapType.class, ImmutableList.of(stringType)); + mergingTree.put(ArrayType.class, ImmutableList.of(arrayType, stringType)); + mergingTree.put(MapType.class, ImmutableList.of(mapType, stringType)); return mergingTree; } } diff --git a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaMergingUtilsTest.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaMergingUtilsTest.java index 567d5f130ec..de73579c1c2 100644 --- a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaMergingUtilsTest.java +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaMergingUtilsTest.java @@ -19,8 +19,10 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; @@ -49,6 +51,7 @@ import java.sql.Timestamp; import java.time.Instant; import java.time.LocalDate; +import java.time.LocalTime; import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.ArrayList; @@ -743,6 +746,10 @@ BINARY, binOf("notre dame"), STRING, binStrOf("bm90cmUgZGFtZQ==")), DATE, dateOf(2020, 4, 4), TIMESTAMP_TZ, zTsOf("2020", "04", "04")), Tuple4.of(DATE, dateOf(2021, 5, 5), STRING, binStrOf("2021-05-05")), + // From TIME + Tuple4.of(TIME, timeOf(21, 48, 25), TIME, timeOf(21, 48, 25)), + Tuple4.of(TIME, timeOf(21, 48, 25), STRING, binStrOf("21:48:25")), + // From TIMESTAMP Tuple4.of( TIMESTAMP, @@ -1145,8 +1152,12 @@ private static Schema of(Object... args) { return builder.build(); } - private static long dateOf(int year, int month, int dayOfMonth) { - return LocalDate.of(year, month, dayOfMonth).toEpochDay(); + private static DateData dateOf(int year, int month, int dayOfMonth) { + return DateData.fromLocalDate(LocalDate.of(year, month, dayOfMonth)); + } + + private static TimeData timeOf(int hour, int minute, int second) { + return TimeData.fromLocalTime(LocalTime.of(hour, minute, second)); } private static TimestampData tsOf(String year, String month, String dayOfMonth) { diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkParallelizedPipelineITCase.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkParallelizedPipelineITCase.java index 90b2e52331c..533e59b21fc 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkParallelizedPipelineITCase.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkParallelizedPipelineITCase.java @@ -387,7 +387,7 @@ void testDistributedTablesSourceInMultipleParallelismWithLenientBehavior( "Bob", "Q2ljYWRh", "RGVycmlkYQ==", - "64800000", + "18:00:01", "2019-12-31T18:00", "2020-07-17T18:00", "1970-01-05T05:20:00.000123456+08:00" @@ -686,7 +686,7 @@ void testRegularTablesSourceInMultipleParallelism(ValuesDataSink.SinkApi sinkApi "Bob", "Q2ljYWRh", "RGVycmlkYQ==", - "64800000", + "18:00:01", "2019-12-31T18:00", "2020-07-17T18:00", "1970-01-05T05:20:00.000123456+08:00" @@ -936,7 +936,7 @@ void testRegularTablesSourceMergedInMultipleParallelism(ValuesDataSink.SinkApi s "Bob", "Q2ljYWRh", "RGVycmlkYQ==", - "64800000", + "18:00:01", "2019-12-31T18:00", "2020-07-17T18:00", "1970-01-05T05:20:00.000123456+08:00" diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java index 08c6da71d78..e7289606d50 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineTransformITCase.java @@ -18,8 +18,10 @@ package org.apache.flink.cdc.composer.flink; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; @@ -74,7 +76,9 @@ import java.math.BigDecimal; import java.sql.Timestamp; import java.time.Instant; +import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.LocalTime; import java.time.ZoneId; import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; @@ -2725,6 +2729,103 @@ void testShadeOriginalColumnsWithDifferentType() throws Exception { "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2.5, ], after=[2.5, x], op=UPDATE, meta=({op_ts=5})}"); } + @Test + void testDateAndTimeCastingFunctions() throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.CUSTOM_SOURCE_EVENTS); + + TableId tableId = TableId.tableId("default_namespace", "default_schema", "my_table"); + Schema tableSchema = + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("date_0", DataTypes.DATE()) + .physicalColumn("time_0", DataTypes.TIME(0)) + .physicalColumn("time_3", DataTypes.TIME(3)) + .physicalColumn("time_6", DataTypes.TIME(6)) + .physicalColumn("time_9", DataTypes.TIME(9)) + .primaryKey("id") + .build(); + BinaryRecordDataGenerator generator = + new BinaryRecordDataGenerator( + tableSchema.getColumnDataTypes().toArray(new DataType[0])); + + List events = + Arrays.asList( + new CreateTableEvent(tableId, tableSchema), + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] { + 1, + DateData.fromLocalDate(LocalDate.of(1999, 12, 31)), + TimeData.fromLocalTime(LocalTime.of(21, 48, 25)), + TimeData.fromLocalTime( + LocalTime.of(21, 48, 25, 123000000)), + TimeData.fromLocalTime( + LocalTime.of(21, 48, 25, 123456000)), + TimeData.fromLocalTime( + LocalTime.of(21, 48, 25, 123456789)) + })), + DataChangeEvent.insertEvent( + tableId, + generator.generate( + new Object[] {2, null, null, null, null, null}))); + + TransformDef transformDef = + new TransformDef( + tableId.toString(), + "*," + + "CAST(date_0 AS VARCHAR) AS date_0_str," + + "CAST(time_0 AS VARCHAR) AS time_0_str," + + "CAST(time_3 AS VARCHAR) AS time_3_str," + + "CAST(time_6 AS VARCHAR) AS time_6_str," + + "CAST(time_9 AS VARCHAR) AS time_9_str", + null, + "id", + null, + null, + null, + null); + + ValuesDataSourceHelper.setSourceEvents(Collections.singletonList(events)); + + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + sinkConfig.set(ValuesDataSinkOptions.SINK_API, ValuesDataSink.SinkApi.SINK_V2); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, + sinkDef, + Collections.emptyList(), + Collections.singletonList(transformDef), + Collections.emptyList(), + pipelineConfig); + + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + + assertThat(outputEvents) + .containsExactlyInAnyOrder( + "CreateTableEvent{tableId=default_namespace.default_schema.my_table, schema=columns={`id` INT NOT NULL,`date_0` DATE,`time_0` TIME(0),`time_3` TIME(3),`time_6` TIME(6),`time_9` TIME(9),`date_0_str` STRING,`time_0_str` STRING,`time_3_str` STRING,`time_6_str` STRING,`time_9_str` STRING}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.my_table, before=[], after=[1, 1999-12-31, 21:48:25, 21:48:25.123, 21:48:25.123, 21:48:25.123, 1999-12-31, 21:48:25, 21:48:25.123, 21:48:25.123, 21:48:25.123], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.my_table, before=[], after=[2, null, null, null, null, null, null, null, null, null, null], op=INSERT, meta=()}"); + } + private List generateFloorCeilAndRoundEvents(TableId tableId) { List events = new ArrayList<>(); Schema schema = @@ -3131,14 +3232,16 @@ void verifyDataRecord(String recordLine) { .toInstant(ZoneOffset.UTC); long milliSecondsInOneDay = 24 * 60 * 60 * 1000; - - assertThat(instant.toEpochMilli() % milliSecondsInOneDay) - .isEqualTo(Long.parseLong(localTime)); - - String currentDate = tokens.get(5); - - assertThat(instant.toEpochMilli() / milliSecondsInOneDay) - .isEqualTo(Long.parseLong(currentDate)); + assertThat(TimeData.fromIsoLocalTimeString(localTime)) + .isEqualTo( + TimeData.fromMillisOfDay( + (int) (instant.toEpochMilli() % milliSecondsInOneDay))); + + String localDate = tokens.get(5); + assertThat(DateData.fromIsoLocalDateString(localDate)) + .isEqualTo( + DateData.fromEpochDay( + (int) (instant.toEpochMilli() / milliSecondsInOneDay))); } BinaryRecordData generate(Schema schema, Object... fields) { diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/testsource/source/DistributedSourceFunction.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/testsource/source/DistributedSourceFunction.java index 4192a4f64b3..58bbe3f42e1 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/testsource/source/DistributedSourceFunction.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/testsource/source/DistributedSourceFunction.java @@ -19,6 +19,7 @@ import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; @@ -116,7 +117,7 @@ public void open(Configuration parameters) throws Exception { dummyDataTypes.put(DataTypes.VARCHAR(17), BinaryStringData.fromString("Bob")); dummyDataTypes.put(DataTypes.BINARY(17), "Cicada".getBytes()); dummyDataTypes.put(DataTypes.VARBINARY(17), "Derrida".getBytes()); - dummyDataTypes.put(DataTypes.TIME(9), 64800000); + dummyDataTypes.put(DataTypes.TIME(9), TimeData.fromMillisOfDay(64801000)); dummyDataTypes.put( DataTypes.TIMESTAMP(9), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:00"))); diff --git a/flink-cdc-composer/src/test/resources/ref-output/distributed.txt b/flink-cdc-composer/src/test/resources/ref-output/distributed.txt index 5cecea29c86..691e11dea44 100644 --- a/flink-cdc-composer/src/test/resources/ref-output/distributed.txt +++ b/flink-cdc-composer/src/test/resources/ref-output/distributed.txt @@ -235,66 +235,66 @@ DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], a AddColumnEvent{tableId=default_namespace.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_varbinarytype` VARBINARY(17), position=LAST, existedColumnName=null}]} DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$135$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_0, addedColumns=[ColumnWithPosition{column=`col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$136$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$136$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_1, addedColumns=[ColumnWithPosition{column=`col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$137$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$137$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_2, addedColumns=[ColumnWithPosition{column=`col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$138$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$138$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_3, addedColumns=[ColumnWithPosition{column=`col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$139$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$139$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_0, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$140$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$140$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_1, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$141$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$141$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_2, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$142$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$142$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$143$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$143$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_0, addedColumns=[ColumnWithPosition{column=`col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$144$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$144$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_1, addedColumns=[ColumnWithPosition{column=`col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$145$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$145$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_2, addedColumns=[ColumnWithPosition{column=`col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$146$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$146$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_3, addedColumns=[ColumnWithPosition{column=`col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$147$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$147$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_0, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$148$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$148$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_1, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$149$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$149$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_2, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$150$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$150$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$151$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$151$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_0, addedColumns=[ColumnWithPosition{column=`col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$152$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$152$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_1, addedColumns=[ColumnWithPosition{column=`col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$153$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$153$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_2, addedColumns=[ColumnWithPosition{column=`col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$154$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$154$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_3, addedColumns=[ColumnWithPosition{column=`col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$155$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$155$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_0, addedColumns=[ColumnWithPosition{column=`subtask_0_col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$156$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$156$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_1, addedColumns=[ColumnWithPosition{column=`subtask_0_col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$157$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$157$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_2, addedColumns=[ColumnWithPosition{column=`subtask_0_col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$158$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$158$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$159$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$159$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_0, addedColumns=[ColumnWithPosition{column=`col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$160$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$160$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_1, addedColumns=[ColumnWithPosition{column=`col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$161$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$161$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_2, addedColumns=[ColumnWithPosition{column=`col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$162$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$162$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_3, addedColumns=[ColumnWithPosition{column=`col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$163$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$163$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_0, addedColumns=[ColumnWithPosition{column=`subtask_0_col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$164$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_0, before=[], after=[__$0$164$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_1, addedColumns=[ColumnWithPosition{column=`subtask_0_col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$165$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_1, before=[], after=[__$0$165$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_2, addedColumns=[ColumnWithPosition{column=`subtask_0_col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$166$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_2, before=[], after=[__$0$166$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$167$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace.default_database.table_3, before=[], after=[__$0$167$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} diff --git a/flink-cdc-composer/src/test/resources/ref-output/regular.txt b/flink-cdc-composer/src/test/resources/ref-output/regular.txt index 2781e76e728..0e092cfcc19 100644 --- a/flink-cdc-composer/src/test/resources/ref-output/regular.txt +++ b/flink-cdc-composer/src/test/resources/ref-output/regular.txt @@ -235,66 +235,66 @@ DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, be AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_varbinarytype` VARBINARY(17), position=LAST, existedColumnName=null}]} DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$135$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_0, addedColumns=[ColumnWithPosition{column=`col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$136$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$136$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_1, addedColumns=[ColumnWithPosition{column=`col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$137$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$137$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_2, addedColumns=[ColumnWithPosition{column=`col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$138$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$138$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_3, addedColumns=[ColumnWithPosition{column=`col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$139$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$139$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_0, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$140$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$140$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_1, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$141$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$141$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_2, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$142$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$142$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timetype` TIME(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$143$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$143$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_0, addedColumns=[ColumnWithPosition{column=`col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$144$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$144$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_1, addedColumns=[ColumnWithPosition{column=`col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$145$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$145$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_2, addedColumns=[ColumnWithPosition{column=`col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$146$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$146$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_3, addedColumns=[ColumnWithPosition{column=`col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$147$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$147$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_0, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$148$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$148$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_1, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$149$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$149$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_2, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$150$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$150$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_timestamptype` TIMESTAMP(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$151$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$151$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_0, addedColumns=[ColumnWithPosition{column=`col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$152$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$152$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_1, addedColumns=[ColumnWithPosition{column=`col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$153$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$153$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_2, addedColumns=[ColumnWithPosition{column=`col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$154$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$154$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_3, addedColumns=[ColumnWithPosition{column=`col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$155$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$155$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_0, addedColumns=[ColumnWithPosition{column=`subtask_0_col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$156$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$156$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_1, addedColumns=[ColumnWithPosition{column=`subtask_0_col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$157$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$157$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_2, addedColumns=[ColumnWithPosition{column=`subtask_0_col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$158$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$158$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_zonedtimestamptype` TIMESTAMP(9) WITH TIME ZONE, position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$159$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$159$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_0, addedColumns=[ColumnWithPosition{column=`col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$160$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$160$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_1, addedColumns=[ColumnWithPosition{column=`col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$161$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$161$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_2, addedColumns=[ColumnWithPosition{column=`col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$162$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$162$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_3, addedColumns=[ColumnWithPosition{column=`col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$163$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$163$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_0, addedColumns=[ColumnWithPosition{column=`subtask_0_col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$164$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_0, before=[], after=[__$0$164$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_1, addedColumns=[ColumnWithPosition{column=`subtask_0_col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$165$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_1, before=[], after=[__$0$165$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_2, addedColumns=[ColumnWithPosition{column=`subtask_0_col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$166$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_2, before=[], after=[__$0$166$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} AddColumnEvent{tableId=default_namespace_subtask_0.default_database.table_3, addedColumns=[ColumnWithPosition{column=`subtask_0_col_localzonedtimestamptype` TIMESTAMP_LTZ(9), position=LAST, existedColumnName=null}]} -DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$167$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 64800000, 64800000, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} +DataChangeEvent{tableId=default_namespace_subtask_0.default_database.table_3, before=[], after=[__$0$167$__, true, true, 17, 17, 34, 34, 68, 68, 136, 136, 272.0, 272.0, 544.0, 544.0, 1088.00000000000, 1088.00000000000, Alice, Alice, Bob, Bob, Q2ljYWRh, Q2ljYWRh, RGVycmlkYQ==, RGVycmlkYQ==, 18:00:01, 18:00:01, 2020-07-17T18:00, 2020-07-17T18:00, 1970-01-05T05:20:00.000123456+08:00, 1970-01-05T05:20:00.000123456+08:00, 2019-12-31T18:00, 2019-12-31T18:00], op=INSERT, meta=()} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisRowConverter.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisRowConverter.java index 4624c560471..b1b67a1e29a 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisRowConverter.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisRowConverter.java @@ -33,8 +33,6 @@ import java.io.IOException; import java.io.Serializable; -import java.time.LocalDate; -import java.time.LocalTime; import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.Arrays; @@ -98,7 +96,8 @@ static SerializationConverter createExternalConverter(DataType type, ZoneId pipe return (index, val) -> val.getDouble(index); case DATE: return (index, val) -> - LocalDate.ofEpochDay(val.getInt(index)) + val.getDate(index) + .toLocalDate() .format(DorisEventSerializer.DATE_FORMATTER); case TIMESTAMP_WITHOUT_TIME_ZONE: return (index, val) -> @@ -118,7 +117,7 @@ static SerializationConverter createExternalConverter(DataType type, ZoneId pipe final int zonedP = ((ZonedTimestampType) type).getPrecision(); return (index, val) -> val.getTimestamp(index, zonedP).toTimestamp(); case TIME_WITHOUT_TIME_ZONE: - return (index, val) -> LocalTime.ofNanoOfDay(val.getLong(index) * 1_000_000L); + return (index, val) -> val.getTime(index).toLocalTime(); case ARRAY: return (index, val) -> convertArrayData(val.getArray(index), type); case MAP: diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisEventSerializerTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisEventSerializerTest.java index 8823e66ef56..e64643226d9 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisEventSerializerTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisEventSerializerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.connectors.doris.sink; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; import org.apache.flink.cdc.common.event.CreateTableEvent; @@ -88,7 +89,7 @@ public void testDataChangeEventWithDateTimePartitionColumn() throws IOException new Object[] { new BinaryStringData("1"), new BinaryStringData("flink"), - (int) LocalDate.of(2025, 1, 16).toEpochDay(), + DateData.fromLocalDate(LocalDate.of(2025, 1, 16)), TimestampData.fromLocalDateTime(localDateTime), })); @@ -118,7 +119,7 @@ public void testDataChangeEventIfDatetimePartitionColumnIsNull() throws IOExcept new Object[] { new BinaryStringData("1"), new BinaryStringData("flink"), - (int) LocalDate.of(2025, 1, 16).toEpochDay(), + DateData.fromLocalDate(LocalDate.of(2025, 1, 16)), null, })); @@ -148,7 +149,7 @@ public void testDataChangeEventWithDatePartitionColumn() throws IOException { new Object[] { new BinaryStringData("1"), new BinaryStringData("flink"), - (int) LocalDate.of(2025, 1, 16).toEpochDay(), + DateData.fromLocalDate(LocalDate.of(2025, 1, 16)), null, })); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisRowConverterTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisRowConverterTest.java index 816a12cc507..aa9dc66afd9 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisRowConverterTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisRowConverterTest.java @@ -17,7 +17,9 @@ package org.apache.flink.cdc.connectors.doris.sink; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; @@ -98,7 +100,7 @@ void testExternalConvert() { 64, 128L, TimestampData.fromLocalDateTime(time1), - (int) date1.toEpochDay(), + DateData.fromLocalDate(date1), BinaryStringData.fromString("a"), BinaryStringData.fromString("doris"), TimestampData.fromLocalDateTime(f17), @@ -107,9 +109,9 @@ void testExternalConvert() { LocalZonedTimestampData.fromInstant(f20), LocalZonedTimestampData.fromInstant(f21), LocalZonedTimestampData.fromInstant(f22), - 3661000, - 3661123, - 3661123 + TimeData.fromNanoOfDay(3661000_000000L), + TimeData.fromNanoOfDay(3661123_000000L), + TimeData.fromNanoOfDay(3661123_000000L) }); List row = new ArrayList<>(); for (int i = 0; i < recordData.getArity(); i++) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/serializer/ElasticsearchRowConverter.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/serializer/ElasticsearchRowConverter.java index f3c77d7ba5a..792f6e22fcc 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/serializer/ElasticsearchRowConverter.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/serializer/ElasticsearchRowConverter.java @@ -31,7 +31,6 @@ import org.apache.flink.elasticsearch6.shaded.com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; -import java.time.LocalDate; import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.util.Arrays; @@ -118,7 +117,7 @@ static ElasticsearchRowConverter.SerializationConverter createExternalConverter( case DOUBLE: return (pos, data) -> data.getDouble(pos); case DATE: - return (pos, data) -> LocalDate.ofEpochDay(data.getInt(pos)).format(DATE_FORMATTER); + return (pos, data) -> data.getDate(pos).toLocalDate().format(DATE_FORMATTER); case TIMESTAMP_WITHOUT_TIME_ZONE: return (pos, data) -> data.getTimestamp(pos, DataTypeChecks.getPrecision(columnType)) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchEventSerializerTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchEventSerializerTest.java index 43241ca4b4c..10335972435 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchEventSerializerTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchEventSerializerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.connectors.elasticsearch.sink; import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; import org.apache.flink.cdc.common.event.CreateTableEvent; @@ -116,7 +117,7 @@ private String getShardingString(Map shardingKey, String shardi ZonedTimestampData.fromZonedDateTime( LocalDateTime.of(2023, 11, 11, 11, 11, 11, 11) .atZone(ZoneId.systemDefault())), - (int) LocalDate.of(2025, 1, 1).toEpochDay() + DateData.fromLocalDate(LocalDate.of(2025, 1, 1)) })); ElasticsearchEventSerializer serializer = new ElasticsearchEventSerializer(ZoneId.of("UTC"), shardingKey, shardingSeparator); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/test/java/org/apache/flink/cdc/connectors/fluss/sink/FlussEventSerializationSchemaTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/test/java/org/apache/flink/cdc/connectors/fluss/sink/FlussEventSerializationSchemaTest.java index 4d7e4065723..7097a72ee45 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/test/java/org/apache/flink/cdc/connectors/fluss/sink/FlussEventSerializationSchemaTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/test/java/org/apache/flink/cdc/connectors/fluss/sink/FlussEventSerializationSchemaTest.java @@ -17,6 +17,7 @@ package org.apache.flink.cdc.connectors.fluss.sink; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.RecordData; @@ -189,7 +190,7 @@ record = record = generator2.generate( new Object[] { - (int) LocalDate.of(2023, 11, 27).toEpochDay(), + DateData.fromLocalDate(LocalDate.of(2023, 11, 27)), 3.4f, BinaryStringData.fromString("insert table2"), DecimalData.fromBigDecimal(new BigDecimal("83.23"), 20, 5), @@ -206,7 +207,7 @@ record = record = generator2.generate( new Object[] { - 4, + DateData.fromEpochDay(4), 3.4f, BinaryStringData.fromString("insert table2"), DecimalData.fromBigDecimal(new BigDecimal("83.23"), 20, 5), diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/main/java/org/apache/flink/cdc/connectors/iceberg/sink/utils/IcebergTypeUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/main/java/org/apache/flink/cdc/connectors/iceberg/sink/utils/IcebergTypeUtils.java index 470e7b15a77..2f820b3a26a 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/main/java/org/apache/flink/cdc/connectors/iceberg/sink/utils/IcebergTypeUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/main/java/org/apache/flink/cdc/connectors/iceberg/sink/utils/IcebergTypeUtils.java @@ -156,9 +156,13 @@ public static RecordData.FieldGetter createFieldGetter( fieldGetter = row -> row.getDouble(fieldPos); break; case INTEGER: + fieldGetter = (row) -> row.getInt(fieldPos); + break; case DATE: + fieldGetter = (row) -> (int) row.getDate(fieldPos).toEpochDay(); + break; case TIME_WITHOUT_TIME_ZONE: - fieldGetter = (row) -> row.getInt(fieldPos); + fieldGetter = (row) -> (int) row.getTime(fieldPos).toMillisOfDay(); break; case TIMESTAMP_WITHOUT_TIME_ZONE: fieldGetter = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/CompactionOperatorTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/CompactionOperatorTest.java index 1935335b9aa..a150e7188fa 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/CompactionOperatorTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/CompactionOperatorTest.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.connectors.iceberg.sink.v2; import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; @@ -138,7 +139,7 @@ public void testCompationOperator() throws IOException, InterruptedException { 1.0f, 1.0d, DecimalData.fromBigDecimal(new BigDecimal(1.0), 10, 2), - 9 + DateData.fromEpochDay(9) }); icebergWriter.write(DataChangeEvent.insertEvent(tableId, recordData), null); Collection> collection = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/IcebergWriterTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/IcebergWriterTest.java index 6b3305ff1ba..4d970e7615f 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/IcebergWriterTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/IcebergWriterTest.java @@ -18,9 +18,11 @@ package org.apache.flink.cdc.connectors.iceberg.sink.v2; import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; @@ -152,7 +154,7 @@ public void testWriteWithSchemaEvolution() throws Exception { 1.0f, 1.0d, DecimalData.fromBigDecimal(new BigDecimal("1.0"), 10, 2), - 9 + DateData.fromEpochDay(9) }); DataChangeEvent dataChangeEvent = DataChangeEvent.insertEvent(tableId, recordData); icebergWriter.write(dataChangeEvent, null); @@ -167,7 +169,7 @@ public void testWriteWithSchemaEvolution() throws Exception { 1.0f, 1.0d, DecimalData.fromBigDecimal(new BigDecimal("1.0"), 10, 2), - 9 + DateData.fromEpochDay(9) }); DataChangeEvent dataChangeEvent2 = DataChangeEvent.insertEvent(tableId, recordData2); icebergWriter.write(dataChangeEvent2, null); @@ -194,7 +196,7 @@ public void testWriteWithSchemaEvolution() throws Exception { 1.0f, 1.0d, DecimalData.fromBigDecimal(new BigDecimal("1.0"), 10, 2), - 9 + DateData.fromEpochDay(9) }); DataChangeEvent dataChangeEvent3 = DataChangeEvent.insertEvent(tableId, recordData3); icebergWriter.write(dataChangeEvent3, null); @@ -209,7 +211,7 @@ public void testWriteWithSchemaEvolution() throws Exception { 1.0f, 1.0d, DecimalData.fromBigDecimal(new BigDecimal("1.0"), 10, 2), - 9 + DateData.fromEpochDay(9) }); DataChangeEvent dataChangeEvent4 = DataChangeEvent.insertEvent(tableId, recordData4); icebergWriter.write(dataChangeEvent4, null); @@ -243,7 +245,7 @@ public void testWriteWithSchemaEvolution() throws Exception { 1.0f, 1.0d, DecimalData.fromBigDecimal(new BigDecimal("1.0"), 10, 2), - 9, + DateData.fromEpochDay(9), BinaryStringData.fromString("newStringColumn"), }); DataChangeEvent dataChangeEvent5 = DataChangeEvent.insertEvent(tableId, recordData5); @@ -324,8 +326,8 @@ public void testWriteWithAllSupportedTypes() throws Exception { 12345L, 123.456f, 123456.789d, - 12345, - 12345, + TimeData.fromNanoOfDay(12345_000_000L), + DateData.fromEpochDay(12345), TimestampData.fromTimestamp(Timestamp.valueOf("1970-01-01 00:00:00")), LocalZonedTimestampData.fromInstant(Instant.ofEpochSecond(0)), ZonedTimestampData.fromZonedDateTime( @@ -340,9 +342,22 @@ public void testWriteWithAllSupportedTypes() throws Exception { writeResults.stream().map(MockCommitRequestImpl::new).collect(Collectors.toList()); icebergCommitter.commit(collection); List result = fetchTableContent(catalog, tableId); - Assertions.assertThat(result) - .containsExactlyInAnyOrder( - "char, varchar, string, false, [1,2,3,4,5,], [1,2,3,4,5,6,7,8,9,10,], 0.00, true, 2, 12345, 12345, 123.456, 123456.789, 00:00:12.345, 2003-10-20, 1970-01-01T00:00, 1970-01-01T00:00Z, 1970-01-01T08:00Z"); + String expectedTimestampTz = + Instant.EPOCH + .atZone(ZoneId.systemDefault()) // 例如 +11 + .toLocalDateTime() // 1970-01-01T11:00 + .toString() + + "Z"; // 1970-01-01T11:00Z + + String expected = + String.format( + "char, varchar, string, false, [1,2,3,4,5,], " + + "[1,2,3,4,5,6,7,8,9,10,], 0.00, true, 2, 12345, 12345, " + + "123.456, 123456.789, 00:00:12.345, 2003-10-20, " + + "1970-01-01T00:00, 1970-01-01T00:00Z, %s", + expectedTimestampTz); + + Assertions.assertThat(result).containsExactlyInAnyOrder(expected); } /** Mock CommitRequestImpl. */ diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfo.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfo.java index 5be8e28d811..10be2ff07f8 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfo.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfo.java @@ -139,9 +139,13 @@ record -> fieldGetter = record -> record.getShort(fieldPos); break; case INTEGER: + fieldGetter = record -> record.getInt(fieldPos); + break; case DATE: + fieldGetter = record -> (int) record.getDate(fieldPos).toEpochDay(); + break; case TIME_WITHOUT_TIME_ZONE: - fieldGetter = record -> record.getInt(fieldPos); + fieldGetter = record -> (int) record.getTime(fieldPos).toMillisOfDay(); break; case BIGINT: fieldGetter = record -> record.getLong(fieldPos); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfoTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfoTest.java index 3ab57e472b6..ffae9044233 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfoTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/TableSchemaInfoTest.java @@ -17,8 +17,10 @@ package org.apache.flink.cdc.connectors.kafka.json; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; @@ -116,9 +118,9 @@ void testGetRowDataFromRecordData() { BinaryStringData.fromString("test1"), BinaryStringData.fromString("test2"), BinaryStringData.fromString("test3"), - 100, - 200, - 300, + DateData.fromEpochDay(100), + TimeData.fromNanoOfDay(200_000_000L), + TimeData.fromNanoOfDay(300_000_000L), TimestampData.fromTimestamp( java.sql.Timestamp.valueOf("2023-01-01 00:00:00.000")), TimestampData.fromTimestamp(java.sql.Timestamp.valueOf("2023-01-01 00:00:00")), diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchemaTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchemaTest.java index d4ef2be7d58..f63677c6cbe 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchemaTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchemaTest.java @@ -18,8 +18,10 @@ package org.apache.flink.cdc.connectors.kafka.json.debezium; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; import org.apache.flink.cdc.common.event.CreateTableEvent; @@ -226,9 +228,9 @@ public void testSerializeWithSchemaAllDataTypes() throws Exception { BinaryStringData.fromString("test1"), BinaryStringData.fromString("test2"), BinaryStringData.fromString("test3"), - 100, - 200, - 300, + DateData.fromEpochDay(100), + TimeData.fromNanoOfDay(200_000_000L), + TimeData.fromNanoOfDay(300_000_000L), TimestampData.fromTimestamp( java.sql.Timestamp.valueOf("2023-01-01 00:00:00.000")), TimestampData.fromTimestamp( diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/utils/TypeConvertUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/utils/TypeConvertUtils.java index e87ad27e7ca..41d303a3b9b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/utils/TypeConvertUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/utils/TypeConvertUtils.java @@ -257,11 +257,10 @@ record -> fieldGetter = record -> record.getInt(fieldPos); break; case DATE: - fieldGetter = record -> LocalDate.ofEpochDay(record.getInt(fieldPos)); + fieldGetter = record -> record.getDate(fieldPos).toLocalDate(); break; case TIME_WITHOUT_TIME_ZONE: - fieldGetter = - record -> LocalTime.ofNanoOfDay(record.getInt(fieldPos) * 1000L).toString(); + fieldGetter = record -> record.getTime(fieldPos).toString(); break; case BIGINT: fieldGetter = record -> record.getLong(fieldPos); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/test/java/org/apache/flink/cdc/connectors/maxcompute/utils/TypeConvertUtilsTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/test/java/org/apache/flink/cdc/connectors/maxcompute/utils/TypeConvertUtilsTest.java index 1bbb0cabb42..ad6dd684ca2 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/test/java/org/apache/flink/cdc/connectors/maxcompute/utils/TypeConvertUtilsTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/test/java/org/apache/flink/cdc/connectors/maxcompute/utils/TypeConvertUtilsTest.java @@ -18,8 +18,10 @@ package org.apache.flink.cdc.connectors.maxcompute.utils; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; @@ -165,8 +167,8 @@ void testRecordConvert() { 12345L, 123.456f, 123456.789d, - 12345, - 12345, + TimeData.fromMillisOfDay(1234567), + DateData.fromEpochDay(12345), TimestampData.fromTimestamp(Timestamp.valueOf("1970-01-01 00:00:00")), LocalZonedTimestampData.fromInstant(Instant.ofEpochSecond(0)), ZonedTimestampData.fromZonedDateTime( @@ -179,7 +181,7 @@ void testRecordConvert() { TypeConvertUtils.toMaxComputeRecord(schemaWithoutComplexType, record1, arrayRecord); String expect = - "char,varchar,string,false,=01=02=03=04=05,=01=02=03=04=05=06=07=08=09=0A,0.00,1,2,12345,12345,123.456,123456.789,00:00:00.012345,2003-10-20,1970-01-01T00:00,1970-01-01T00:00:00Z,1970-01-01T00:00:00Z"; + "char,varchar,string,false,=01=02=03=04=05,=01=02=03=04=05=06=07=08=09=0A,0.00,1,2,12345,12345,123.456,123456.789,00:20:34.567,2003-10-20,1970-01-01T00:00,1970-01-01T00:00:00Z,1970-01-01T00:00:00Z"; assertThat(arrayRecord).hasToString(expect); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlAncientDateAndTimeITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlAncientDateAndTimeITCase.java index 3bc76d60e44..dccc41ca817 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlAncientDateAndTimeITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlAncientDateAndTimeITCase.java @@ -118,23 +118,23 @@ public void testAncientDateAndTimeWithTimeAdjuster() throws Exception { ancientDatabase, true, Arrays.asList( - "[1, 17390, 2016-07-13T17:17:17, 2015-06-14T17:17:17.100, 2014-05-15T17:17:17.120, 2013-04-16T17:17:17.123, 2012-03-17T17:17:17.123400, 2011-02-18T17:17:17.123450, 2010-01-19T17:17:17.123456]", + "[1, 2017-08-12, 2016-07-13T17:17:17, 2015-06-14T17:17:17.100, 2014-05-15T17:17:17.120, 2013-04-16T17:17:17.123, 2012-03-17T17:17:17.123400, 2011-02-18T17:17:17.123450, 2010-01-19T17:17:17.123456]", "[2, null, null, null, null, null, null, null, null]", - "[3, 11323, 2001-01-01T16:16:16, 2001-01-01T16:16:16.100, 2001-01-01T16:16:16.120, 2001-01-01T16:16:16.123, 2001-01-01T16:16:16.123400, 2001-01-01T16:16:16.123450, 2001-01-01T16:16:16.123456]", - "[4, 11720, 2002-02-02T15:15:15, 2002-02-02T15:15:15.100, 2002-02-02T15:15:15.120, 2002-02-02T15:15:15.123, 2002-02-02T15:15:15.123400, 2002-02-02T15:15:15.123450, 2002-02-02T15:15:15.123456]", - "[5, 23072, 2033-03-03T14:14:14, 2033-03-03T14:14:14.100, 2033-03-03T14:14:14.120, 2033-03-03T14:14:14.123, 2033-03-03T14:14:14.123400, 2033-03-03T14:14:14.123450, 2033-03-03T14:14:14.123456]", - "[6, -557266, 0444-04-04T13:13:13, 0444-04-04T13:13:13.100, 0444-04-04T13:13:13.120, 0444-04-04T13:13:13.123, 0444-04-04T13:13:13.123400, 0444-04-04T13:13:13.123450, 0444-04-04T13:13:13.123456]", - "[7, -1, 1969-12-31T12:12:12, 1969-12-31T12:12:12.100, 1969-12-31T12:12:12.120, 1969-12-31T12:12:12.123, 1969-12-31T12:12:12.123400, 1969-12-31T12:12:12.123450, 1969-12-31T12:12:12.123456]", - "[8, 18261, 2019-12-31T23:11:11, 2019-12-31T23:11:11.100, 2019-12-31T23:11:11.120, 2019-12-31T23:11:11.123, 2019-12-31T23:11:11.123400, 2019-12-31T23:11:11.123450, 2019-12-31T23:11:11.123456]"), + "[3, 2001-01-01, 2001-01-01T16:16:16, 2001-01-01T16:16:16.100, 2001-01-01T16:16:16.120, 2001-01-01T16:16:16.123, 2001-01-01T16:16:16.123400, 2001-01-01T16:16:16.123450, 2001-01-01T16:16:16.123456]", + "[4, 2002-02-02, 2002-02-02T15:15:15, 2002-02-02T15:15:15.100, 2002-02-02T15:15:15.120, 2002-02-02T15:15:15.123, 2002-02-02T15:15:15.123400, 2002-02-02T15:15:15.123450, 2002-02-02T15:15:15.123456]", + "[5, 2033-03-03, 2033-03-03T14:14:14, 2033-03-03T14:14:14.100, 2033-03-03T14:14:14.120, 2033-03-03T14:14:14.123, 2033-03-03T14:14:14.123400, 2033-03-03T14:14:14.123450, 2033-03-03T14:14:14.123456]", + "[6, 0444-04-04, 0444-04-04T13:13:13, 0444-04-04T13:13:13.100, 0444-04-04T13:13:13.120, 0444-04-04T13:13:13.123, 0444-04-04T13:13:13.123400, 0444-04-04T13:13:13.123450, 0444-04-04T13:13:13.123456]", + "[7, 1969-12-31, 1969-12-31T12:12:12, 1969-12-31T12:12:12.100, 1969-12-31T12:12:12.120, 1969-12-31T12:12:12.123, 1969-12-31T12:12:12.123400, 1969-12-31T12:12:12.123450, 1969-12-31T12:12:12.123456]", + "[8, 2019-12-31, 2019-12-31T23:11:11, 2019-12-31T23:11:11.100, 2019-12-31T23:11:11.120, 2019-12-31T23:11:11.123, 2019-12-31T23:11:11.123400, 2019-12-31T23:11:11.123450, 2019-12-31T23:11:11.123456]"), Arrays.asList( - "[9, 17390, 2016-07-13T17:17:17, 2015-06-14T17:17:17.100, 2014-05-15T17:17:17.120, 2013-04-16T17:17:17.123, 2012-03-17T17:17:17.123400, 2011-02-18T17:17:17.123450, 2010-01-19T17:17:17.123456]", + "[9, 2017-08-12, 2016-07-13T17:17:17, 2015-06-14T17:17:17.100, 2014-05-15T17:17:17.120, 2013-04-16T17:17:17.123, 2012-03-17T17:17:17.123400, 2011-02-18T17:17:17.123450, 2010-01-19T17:17:17.123456]", "[10, null, null, null, null, null, null, null, null]", - "[11, 11323, 2001-01-01T16:16:16, 2001-01-01T16:16:16.100, 2001-01-01T16:16:16.120, 2001-01-01T16:16:16.123, 2001-01-01T16:16:16.123400, 2001-01-01T16:16:16.123450, 2001-01-01T16:16:16.123456]", - "[12, 11720, 2002-02-02T15:15:15, 2002-02-02T15:15:15.100, 2002-02-02T15:15:15.120, 2002-02-02T15:15:15.123, 2002-02-02T15:15:15.123400, 2002-02-02T15:15:15.123450, 2002-02-02T15:15:15.123456]", - "[13, 23072, 2033-03-03T14:14:14, 2033-03-03T14:14:14.100, 2033-03-03T14:14:14.120, 2033-03-03T14:14:14.123, 2033-03-03T14:14:14.123400, 2033-03-03T14:14:14.123450, 2033-03-03T14:14:14.123456]", - "[14, -557266, 0444-04-04T13:13:13, 0444-04-04T13:13:13.100, 0444-04-04T13:13:13.120, 0444-04-04T13:13:13.123, 0444-04-04T13:13:13.123400, 0444-04-04T13:13:13.123450, 0444-04-04T13:13:13.123456]", - "[15, -1, 1969-12-31T12:12:12, 1969-12-31T12:12:12.100, 1969-12-31T12:12:12.120, 1969-12-31T12:12:12.123, 1969-12-31T12:12:12.123400, 1969-12-31T12:12:12.123450, 1969-12-31T12:12:12.123456]", - "[16, 18261, 2019-12-31T23:11:11, 2019-12-31T23:11:11.100, 2019-12-31T23:11:11.120, 2019-12-31T23:11:11.123, 2019-12-31T23:11:11.123400, 2019-12-31T23:11:11.123450, 2019-12-31T23:11:11.123456]")); + "[11, 2001-01-01, 2001-01-01T16:16:16, 2001-01-01T16:16:16.100, 2001-01-01T16:16:16.120, 2001-01-01T16:16:16.123, 2001-01-01T16:16:16.123400, 2001-01-01T16:16:16.123450, 2001-01-01T16:16:16.123456]", + "[12, 2002-02-02, 2002-02-02T15:15:15, 2002-02-02T15:15:15.100, 2002-02-02T15:15:15.120, 2002-02-02T15:15:15.123, 2002-02-02T15:15:15.123400, 2002-02-02T15:15:15.123450, 2002-02-02T15:15:15.123456]", + "[13, 2033-03-03, 2033-03-03T14:14:14, 2033-03-03T14:14:14.100, 2033-03-03T14:14:14.120, 2033-03-03T14:14:14.123, 2033-03-03T14:14:14.123400, 2033-03-03T14:14:14.123450, 2033-03-03T14:14:14.123456]", + "[14, 0444-04-04, 0444-04-04T13:13:13, 0444-04-04T13:13:13.100, 0444-04-04T13:13:13.120, 0444-04-04T13:13:13.123, 0444-04-04T13:13:13.123400, 0444-04-04T13:13:13.123450, 0444-04-04T13:13:13.123456]", + "[15, 1969-12-31, 1969-12-31T12:12:12, 1969-12-31T12:12:12.100, 1969-12-31T12:12:12.120, 1969-12-31T12:12:12.123, 1969-12-31T12:12:12.123400, 1969-12-31T12:12:12.123450, 1969-12-31T12:12:12.123456]", + "[16, 2019-12-31, 2019-12-31T23:11:11, 2019-12-31T23:11:11.100, 2019-12-31T23:11:11.120, 2019-12-31T23:11:11.123, 2019-12-31T23:11:11.123400, 2019-12-31T23:11:11.123450, 2019-12-31T23:11:11.123456]")); } @Test @@ -148,23 +148,23 @@ public void testAncientDateAndTimeWithoutTimeAdjuster() throws Exception { ancientDatabase, false, Arrays.asList( - "[1, -713095, 0016-07-13T17:17:17, 0015-06-14T17:17:17.100, 0014-05-15T17:17:17.120, 0013-04-16T17:17:17.123, 0012-03-17T17:17:17.123400, 0011-02-18T17:17:17.123450, 0010-01-19T17:17:17.123456]", + "[1, 0017-08-12, 0016-07-13T17:17:17, 0015-06-14T17:17:17.100, 0014-05-15T17:17:17.120, 0013-04-16T17:17:17.123, 0012-03-17T17:17:17.123400, 0011-02-18T17:17:17.123450, 0010-01-19T17:17:17.123456]", "[2, null, null, null, null, null, null, null, null]", - "[3, -719162, 0001-01-01T16:16:16, 0001-01-01T16:16:16.100, 0001-01-01T16:16:16.120, 0001-01-01T16:16:16.123, 0001-01-01T16:16:16.123400, 0001-01-01T16:16:16.123450, 0001-01-01T16:16:16.123456]", - "[4, -718765, 0002-02-02T15:15:15, 0002-02-02T15:15:15.100, 0002-02-02T15:15:15.120, 0002-02-02T15:15:15.123, 0002-02-02T15:15:15.123400, 0002-02-02T15:15:15.123450, 0002-02-02T15:15:15.123456]", - "[5, -707413, 0033-03-03T14:14:14, 0033-03-03T14:14:14.100, 0033-03-03T14:14:14.120, 0033-03-03T14:14:14.123, 0033-03-03T14:14:14.123400, 0033-03-03T14:14:14.123450, 0033-03-03T14:14:14.123456]", - "[6, -557266, 0444-04-04T13:13:13, 0444-04-04T13:13:13.100, 0444-04-04T13:13:13.120, 0444-04-04T13:13:13.123, 0444-04-04T13:13:13.123400, 0444-04-04T13:13:13.123450, 0444-04-04T13:13:13.123456]", - "[7, -1, 1969-12-31T12:12:12, 1969-12-31T12:12:12.100, 1969-12-31T12:12:12.120, 1969-12-31T12:12:12.123, 1969-12-31T12:12:12.123400, 1969-12-31T12:12:12.123450, 1969-12-31T12:12:12.123456]", - "[8, 18261, 2019-12-31T23:11:11, 2019-12-31T23:11:11.100, 2019-12-31T23:11:11.120, 2019-12-31T23:11:11.123, 2019-12-31T23:11:11.123400, 2019-12-31T23:11:11.123450, 2019-12-31T23:11:11.123456]"), + "[3, 0001-01-01, 0001-01-01T16:16:16, 0001-01-01T16:16:16.100, 0001-01-01T16:16:16.120, 0001-01-01T16:16:16.123, 0001-01-01T16:16:16.123400, 0001-01-01T16:16:16.123450, 0001-01-01T16:16:16.123456]", + "[4, 0002-02-02, 0002-02-02T15:15:15, 0002-02-02T15:15:15.100, 0002-02-02T15:15:15.120, 0002-02-02T15:15:15.123, 0002-02-02T15:15:15.123400, 0002-02-02T15:15:15.123450, 0002-02-02T15:15:15.123456]", + "[5, 0033-03-03, 0033-03-03T14:14:14, 0033-03-03T14:14:14.100, 0033-03-03T14:14:14.120, 0033-03-03T14:14:14.123, 0033-03-03T14:14:14.123400, 0033-03-03T14:14:14.123450, 0033-03-03T14:14:14.123456]", + "[6, 0444-04-04, 0444-04-04T13:13:13, 0444-04-04T13:13:13.100, 0444-04-04T13:13:13.120, 0444-04-04T13:13:13.123, 0444-04-04T13:13:13.123400, 0444-04-04T13:13:13.123450, 0444-04-04T13:13:13.123456]", + "[7, 1969-12-31, 1969-12-31T12:12:12, 1969-12-31T12:12:12.100, 1969-12-31T12:12:12.120, 1969-12-31T12:12:12.123, 1969-12-31T12:12:12.123400, 1969-12-31T12:12:12.123450, 1969-12-31T12:12:12.123456]", + "[8, 2019-12-31, 2019-12-31T23:11:11, 2019-12-31T23:11:11.100, 2019-12-31T23:11:11.120, 2019-12-31T23:11:11.123, 2019-12-31T23:11:11.123400, 2019-12-31T23:11:11.123450, 2019-12-31T23:11:11.123456]"), Arrays.asList( - "[9, -713095, 0016-07-13T17:17:17, 0015-06-14T17:17:17.100, 0014-05-15T17:17:17.120, 0013-04-16T17:17:17.123, 0012-03-17T17:17:17.123400, 0011-02-18T17:17:17.123450, 0010-01-19T17:17:17.123456]", + "[9, 0017-08-12, 0016-07-13T17:17:17, 0015-06-14T17:17:17.100, 0014-05-15T17:17:17.120, 0013-04-16T17:17:17.123, 0012-03-17T17:17:17.123400, 0011-02-18T17:17:17.123450, 0010-01-19T17:17:17.123456]", "[10, null, null, null, null, null, null, null, null]", - "[11, -719162, 0001-01-01T16:16:16, 0001-01-01T16:16:16.100, 0001-01-01T16:16:16.120, 0001-01-01T16:16:16.123, 0001-01-01T16:16:16.123400, 0001-01-01T16:16:16.123450, 0001-01-01T16:16:16.123456]", - "[12, -718765, 0002-02-02T15:15:15, 0002-02-02T15:15:15.100, 0002-02-02T15:15:15.120, 0002-02-02T15:15:15.123, 0002-02-02T15:15:15.123400, 0002-02-02T15:15:15.123450, 0002-02-02T15:15:15.123456]", - "[13, -707413, 0033-03-03T14:14:14, 0033-03-03T14:14:14.100, 0033-03-03T14:14:14.120, 0033-03-03T14:14:14.123, 0033-03-03T14:14:14.123400, 0033-03-03T14:14:14.123450, 0033-03-03T14:14:14.123456]", - "[14, -557266, 0444-04-04T13:13:13, 0444-04-04T13:13:13.100, 0444-04-04T13:13:13.120, 0444-04-04T13:13:13.123, 0444-04-04T13:13:13.123400, 0444-04-04T13:13:13.123450, 0444-04-04T13:13:13.123456]", - "[15, -1, 1969-12-31T12:12:12, 1969-12-31T12:12:12.100, 1969-12-31T12:12:12.120, 1969-12-31T12:12:12.123, 1969-12-31T12:12:12.123400, 1969-12-31T12:12:12.123450, 1969-12-31T12:12:12.123456]", - "[16, 18261, 2019-12-31T23:11:11, 2019-12-31T23:11:11.100, 2019-12-31T23:11:11.120, 2019-12-31T23:11:11.123, 2019-12-31T23:11:11.123400, 2019-12-31T23:11:11.123450, 2019-12-31T23:11:11.123456]")); + "[11, 0001-01-01, 0001-01-01T16:16:16, 0001-01-01T16:16:16.100, 0001-01-01T16:16:16.120, 0001-01-01T16:16:16.123, 0001-01-01T16:16:16.123400, 0001-01-01T16:16:16.123450, 0001-01-01T16:16:16.123456]", + "[12, 0002-02-02, 0002-02-02T15:15:15, 0002-02-02T15:15:15.100, 0002-02-02T15:15:15.120, 0002-02-02T15:15:15.123, 0002-02-02T15:15:15.123400, 0002-02-02T15:15:15.123450, 0002-02-02T15:15:15.123456]", + "[13, 0033-03-03, 0033-03-03T14:14:14, 0033-03-03T14:14:14.100, 0033-03-03T14:14:14.120, 0033-03-03T14:14:14.123, 0033-03-03T14:14:14.123400, 0033-03-03T14:14:14.123450, 0033-03-03T14:14:14.123456]", + "[14, 0444-04-04, 0444-04-04T13:13:13, 0444-04-04T13:13:13.100, 0444-04-04T13:13:13.120, 0444-04-04T13:13:13.123, 0444-04-04T13:13:13.123400, 0444-04-04T13:13:13.123450, 0444-04-04T13:13:13.123456]", + "[15, 1969-12-31, 1969-12-31T12:12:12, 1969-12-31T12:12:12.100, 1969-12-31T12:12:12.120, 1969-12-31T12:12:12.123, 1969-12-31T12:12:12.123400, 1969-12-31T12:12:12.123450, 1969-12-31T12:12:12.123456]", + "[16, 2019-12-31, 2019-12-31T23:11:11, 2019-12-31T23:11:11.100, 2019-12-31T23:11:11.120, 2019-12-31T23:11:11.123, 2019-12-31T23:11:11.123400, 2019-12-31T23:11:11.123450, 2019-12-31T23:11:11.123456]")); } private void runGenericAncientDateAndTimeTest( diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlFullTypesITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlFullTypesITCase.java index c8546a3dbf7..eb28aaeebcc 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlFullTypesITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlFullTypesITCase.java @@ -19,9 +19,11 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; import org.apache.flink.cdc.common.event.DataChangeEvent; @@ -159,13 +161,10 @@ void testMysql57TimeDataTypes() throws Throwable { new Object[] { DecimalData.fromBigDecimal(new BigDecimal("1"), 20, 0), 2021, - 18460, - 64822000, - 64822123, - // TIME(6) will lose precision for microseconds. - // Because Flink's BinaryWriter force write int value for TIME(6). - // See BinaryWriter#write for detail. - 64822123, + DateData.fromEpochDay(18460), + TimeData.fromNanoOfDay(64822000_000_000L), + TimeData.fromNanoOfDay(64822123_000_000L), + TimeData.fromNanoOfDay(64822123_456_000L), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123")), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123456")), @@ -177,9 +176,9 @@ void testMysql57TimeDataTypes() throws Throwable { new Object[] { DecimalData.fromBigDecimal(new BigDecimal("1"), 20, 0), 2021, - 18460, - 64822000, - 64822123, + DateData.fromEpochDay(18460), + TimeData.fromNanoOfDay(64822000_000_000L), + TimeData.fromNanoOfDay(64822123_000_000L), null, TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123")), @@ -194,6 +193,7 @@ void testMysql57TimeDataTypes() throws Throwable { @Test void testMysql8TimeDataTypes() throws Throwable { + UniqueDatabase usedDd = fullTypesMySql8Database; RowType recordType = RowType.of( DataTypes.DECIMAL(20, 0).notNull(), @@ -214,13 +214,10 @@ void testMysql8TimeDataTypes() throws Throwable { new Object[] { DecimalData.fromBigDecimal(new BigDecimal("1"), 20, 0), 2021, - 18460, - 64822000, - 64822123, - // TIME(6) will lose precision for microseconds. - // Because Flink's BinaryWriter force write int value for TIME(6). - // See BinaryWriter#write for detail. - 64822123, + DateData.fromEpochDay(18460), + TimeData.fromNanoOfDay(64822000_000_000L), + TimeData.fromNanoOfDay(64822123_000_000L), + TimeData.fromNanoOfDay(64822123_456_000L), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123")), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123456")), @@ -234,9 +231,9 @@ void testMysql8TimeDataTypes() throws Throwable { new Object[] { DecimalData.fromBigDecimal(new BigDecimal("1"), 20, 0), 2021, - 18460, - 64822000, - 64822123, + DateData.fromEpochDay(18460), + TimeData.fromNanoOfDay(64822000_000_000L), + TimeData.fromNanoOfDay(64822123_000_000L), null, TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123")), @@ -247,8 +244,7 @@ void testMysql8TimeDataTypes() throws Throwable { LocalZonedTimestampData.fromInstant(toInstant("2000-01-01 00:00:00")) }; - testTimeDataTypes( - fullTypesMySql8Database, recordType, expectedSnapshot, expectedStreamRecord); + testTimeDataTypes(usedDd, recordType, expectedSnapshot, expectedStreamRecord); } @Test @@ -296,9 +292,9 @@ void testMysqlPrecisionTypes(UniqueDatabase database) throws Throwable { DecimalData.fromBigDecimal(new BigDecimal("123.4"), 6, 2), DecimalData.fromBigDecimal(new BigDecimal("1234.5"), 9, 4), DecimalData.fromBigDecimal(new BigDecimal("1234.56"), 20, 4), - 64800000, - 64822100, - 64822100, + TimeData.fromNanoOfDay(64800000_000_000L), + TimeData.fromNanoOfDay(64822100_000_000L), + TimeData.fromNanoOfDay(64822100_000_000L), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:00")), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), @@ -325,8 +321,8 @@ void testMysqlPrecisionTypes(UniqueDatabase database) throws Throwable { DecimalData.fromBigDecimal(new BigDecimal("123.4"), 6, 2), DecimalData.fromBigDecimal(new BigDecimal("1234.5"), 9, 4), DecimalData.fromBigDecimal(new BigDecimal("1234.56"), 20, 4), - 64800000, - 64822100, + TimeData.fromNanoOfDay(64800000_000_000L), + TimeData.fromNanoOfDay(64822100_000_000L), null, TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:00")), TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java index 18adbb1c81d..ff583f7ea8c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/main/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseRowConvert.java @@ -36,8 +36,6 @@ import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalTime; import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.HashMap; @@ -101,10 +99,9 @@ static SerializationConverter createExternalConverter(DataType type, ZoneId pipe case DOUBLE: return (index, val) -> val.getDouble(index); case DATE: - return (index, val) -> Date.valueOf(LocalDate.ofEpochDay(val.getInt(index))); + return (index, val) -> Date.valueOf(val.getDate(index).toLocalDate()); case TIME_WITHOUT_TIME_ZONE: - return (index, val) -> - Time.valueOf(LocalTime.ofSecondOfDay(val.getInt(index) / 1000)); + return (index, val) -> Time.valueOf(val.getTime(index).toLocalTime()); case TIMESTAMP_WITHOUT_TIME_ZONE: return (index, val) -> val.getTimestamp(index, DataTypeChecks.getPrecision(type)).toTimestamp(); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchemaTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchemaTest.java index 9fcac44ec57..addeabda952 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchemaTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oceanbase/src/test/java/org/apache/flink/cdc/connectors/oceanbase/sink/OceanBaseEventSerializationSchemaTest.java @@ -17,6 +17,7 @@ package org.apache.flink.cdc.connectors.oceanbase.sink; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.TimestampData; @@ -144,7 +145,7 @@ void testMixedSchemaAndDataChanges() throws Exception { table2, generator2.generate( new Object[] { - (int) LocalDate.of(2023, 11, 27).toEpochDay(), + DateData.fromLocalDate(LocalDate.of(2023, 11, 27)), 3.4f, BinaryStringData.fromString("insert table2") })); @@ -202,7 +203,7 @@ void testMixedSchemaAndDataChanges() throws Exception { DataChangeEvent.insertEvent( table2, newGenerator2.generate( - new Object[] {(int) LocalDate.of(2023, 11, 28).toEpochDay()})); + new Object[] {DateData.fromLocalDate(LocalDate.of(2023, 11, 28))})); verifySerializeResult( table2, "[2023-11-28]", Objects.requireNonNull(serializer.serialize(insertEvent3))); } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterHelper.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterHelper.java index 36bb9fbbc5b..c64860fc73c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterHelper.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterHelper.java @@ -104,9 +104,13 @@ private static RecordData.FieldGetter createFieldGetter( fieldGetter = row -> row.getDouble(fieldPos); break; case INTEGER: + fieldGetter = row -> row.getInt(fieldPos); + break; case DATE: + fieldGetter = row -> (int) row.getDate(fieldPos).toEpochDay(); + break; case TIME_WITHOUT_TIME_ZONE: - fieldGetter = row -> row.getInt(fieldPos); + fieldGetter = row -> (int) row.getTime(fieldPos).toMillisOfDay(); break; case TIMESTAMP_WITHOUT_TIME_ZONE: fieldGetter = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterHelperTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterHelperTest.java index 3e3ebdb9d34..843ff13456f 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterHelperTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriterHelperTest.java @@ -17,10 +17,12 @@ package org.apache.flink.cdc.connectors.paimon.sink.v2; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.GenericMapData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryMapData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; @@ -95,9 +97,9 @@ void testConvertEventToGenericRowOfAllDataTypes() { BinaryStringData.fromString("test1"), BinaryStringData.fromString("test2"), BinaryStringData.fromString("test3"), - 100, - 200, - 300, + DateData.fromEpochDay(100), + TimeData.fromNanoOfDay(200_000_000L), + TimeData.fromNanoOfDay(300_000_000L), TimestampData.fromTimestamp( java.sql.Timestamp.valueOf("2023-01-01 00:00:00.000")), TimestampData.fromTimestamp(java.sql.Timestamp.valueOf("2023-01-01 00:00:00")), diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksUtils.java index 50dab2ac452..4c75d10e016 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksUtils.java @@ -40,7 +40,6 @@ import com.starrocks.connector.flink.catalog.StarRocksColumn; import com.starrocks.connector.flink.catalog.StarRocksTable; -import java.time.LocalDate; import java.time.ZoneId; import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; @@ -172,9 +171,7 @@ record -> break; case DATE: fieldGetter = - record -> - LocalDate.ofEpochDay(record.getInt(fieldPos)) - .format(DATE_FORMATTER); + record -> record.getDate(fieldPos).toLocalDate().format(DATE_FORMATTER); break; case TIMESTAMP_WITHOUT_TIME_ZONE: fieldGetter = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchemaTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchemaTest.java index 7b0daec7e35..5d830b9352c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchemaTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchemaTest.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.TimestampData; @@ -192,7 +193,7 @@ void testMixedSchemaAndDataChanges() throws Exception { table2, generator2.generate( new Object[] { - (int) LocalDate.of(2023, 11, 27).toEpochDay(), + DateData.fromLocalDate(LocalDate.of(2023, 11, 27)), 3.4f, BinaryStringData.fromString("insert table2") })); @@ -252,7 +253,7 @@ void testMixedSchemaAndDataChanges() throws Exception { DataChangeEvent.insertEvent( table2, newGenerator2.generate( - new Object[] {(int) LocalDate.of(2023, 11, 28).toEpochDay()})); + new Object[] {DateData.fromLocalDate(LocalDate.of(2023, 11, 28))})); verifySerializeResult( table2, "{\"col1\":\"2023-11-28\",\"__op\":0}", diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumEventDeserializationSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumEventDeserializationSchema.java index ad7ec458e53..5c6b9f5ed1e 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumEventDeserializationSchema.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumEventDeserializationSchema.java @@ -19,9 +19,11 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; import org.apache.flink.cdc.common.event.DataChangeEvent; @@ -284,22 +286,22 @@ protected Object convertToFloat(Object dbzObj, Schema schema) { } protected Object convertToDate(Object dbzObj, Schema schema) { - return (int) TemporalConversions.toLocalDate(dbzObj).toEpochDay(); + return DateData.fromLocalDate(TemporalConversions.toLocalDate(dbzObj)); } protected Object convertToTime(Object dbzObj, Schema schema) { if (dbzObj instanceof Long) { switch (schema.name()) { case MicroTime.SCHEMA_NAME: - return (int) ((long) dbzObj / 1000); + return TimeData.fromNanoOfDay((long) dbzObj * 1000); case NanoTime.SCHEMA_NAME: - return (int) ((long) dbzObj / 1000_000); + return TimeData.fromNanoOfDay((long) dbzObj); } } else if (dbzObj instanceof Integer) { - return dbzObj; + return TimeData.fromNanoOfDay((int) dbzObj * 1_000_000L); } // get number of milliseconds of the day - return TemporalConversions.toLocalTime(dbzObj).toSecondOfDay() * 1000; + return TimeData.fromLocalTime(TemporalConversions.toLocalTime(dbzObj)); } protected Object convertToTimestamp(Object dbzObj, Schema schema) { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumSchemaDataTypeInference.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumSchemaDataTypeInference.java index 3afa7128d13..f06a297ffd7 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumSchemaDataTypeInference.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/org/apache/flink/cdc/debezium/event/DebeziumSchemaDataTypeInference.java @@ -25,6 +25,7 @@ import io.debezium.data.SpecialValueDecimal; import io.debezium.data.VariableScaleDecimal; +import io.debezium.time.Date; import io.debezium.time.MicroTime; import io.debezium.time.MicroTimestamp; import io.debezium.time.NanoTime; @@ -32,7 +33,6 @@ import io.debezium.time.Time; import io.debezium.time.Timestamp; import io.debezium.time.ZonedTimestamp; -import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; @@ -104,7 +104,7 @@ protected DataType inferInt16(Object value, Schema schema) { } protected DataType inferInt32(Object value, Schema schema) { - if (Date.LOGICAL_NAME.equals(schema.name())) { + if (Date.SCHEMA_NAME.equals(schema.name())) { return DataTypes.DATE(); } if (Time.SCHEMA_NAME.equals(schema.name())) { diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java index d685d283b05..5a1b8c97c0c 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java @@ -17,12 +17,13 @@ package org.apache.flink.cdc.pipeline.tests; +import org.apache.flink.cdc.common.data.DateData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; import org.apache.flink.cdc.runtime.operators.transform.PostTransformOperator; import org.apache.flink.cdc.runtime.operators.transform.PreTransformOperator; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -47,6 +48,8 @@ import java.util.regex.Pattern; import java.util.stream.Stream; +import static org.assertj.core.api.Assertions.assertThat; + /** E2e tests for the {@link PreTransformOperator} and {@link PostTransformOperator}. */ class TransformE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(TransformE2eITCase.class); @@ -1179,20 +1182,19 @@ boolean extractDataLines(String line) { } void verifyDataRecord(String recordLine) { - LOG.info("Verifying data line {}", recordLine); List tokens = Arrays.asList(recordLine.split(", ")); - Assertions.assertThat(tokens).hasSizeGreaterThanOrEqualTo(6); + assertThat(tokens).hasSizeGreaterThanOrEqualTo(6); tokens = tokens.subList(tokens.size() - 6, tokens.size()); String localTime = tokens.get(0); String currentTime = tokens.get(1); - Assertions.assertThat(currentTime).isEqualTo(localTime); + assertThat(localTime).isEqualTo(currentTime); String currentTimestamp = tokens.get(2); String nowTimestamp = tokens.get(3); String localTimestamp = tokens.get(4); - Assertions.assertThat(currentTimestamp).isEqualTo(nowTimestamp).isEqualTo(localTimestamp); + assertThat(currentTimestamp).isEqualTo(nowTimestamp).isEqualTo(localTimestamp); // If timestamp millisecond part is .000, it will be truncated to yyyy-MM-dd'T'HH:mm:ss // format. Manually append this for the following checks. @@ -1207,13 +1209,15 @@ void verifyDataRecord(String recordLine) { .toInstant(ZoneOffset.UTC); long milliSecondsInOneDay = 24 * 60 * 60 * 1000; - - Assertions.assertThat(Long.parseLong(localTime)) - .isEqualTo(instant.toEpochMilli() % milliSecondsInOneDay); - - String currentDate = tokens.get(5); - - Assertions.assertThat(Long.parseLong(currentDate)) - .isEqualTo(instant.toEpochMilli() / milliSecondsInOneDay); + assertThat(TimeData.fromIsoLocalTimeString(localTime)) + .isEqualTo( + TimeData.fromMillisOfDay( + (int) (instant.toEpochMilli() % milliSecondsInOneDay))); + + String localDate = tokens.get(5); + assertThat(DateData.fromIsoLocalDateString(localDate)) + .isEqualTo( + DateData.fromEpochDay( + (int) (instant.toEpochMilli() / milliSecondsInOneDay))); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/functions/SystemFunctionUtils.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/functions/SystemFunctionUtils.java index bdf7674858f..eb9f593c34c 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/functions/SystemFunctionUtils.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/functions/SystemFunctionUtils.java @@ -17,8 +17,10 @@ package org.apache.flink.cdc.runtime.functions; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.utils.DateTimeUtils; @@ -65,16 +67,16 @@ public static TimestampData localtimestamp(long epochTime, String timezone) { Instant.ofEpochMilli(epochTime).atZone(ZoneId.of(timezone)).toLocalDateTime()); } - public static int localtime(long epochTime, String timezone) { + public static TimeData localtime(long epochTime, String timezone) { return timestampMillisToTime(localtimestamp(epochTime, timezone).getMillisecond()); } - public static int currentTime(long epochTime, String timezone) { + public static TimeData currentTime(long epochTime, String timezone) { // the time value of currentTimestamp under given session time zone return timestampMillisToTime(localtimestamp(epochTime, timezone).getMillisecond()); } - public static int currentDate(long epochTime, String timezone) { + public static DateData currentDate(long epochTime, String timezone) { // the date value of currentTimestamp under given session time zone return timestampMillisToDate(localtimestamp(epochTime, timezone).getMillisecond()); } @@ -118,11 +120,11 @@ public static String dateFormat( timestamp.getEpochMillisecond(), format, TimeZone.getTimeZone(timezone)); } - public static int toDate(String str, String timezone) { + public static DateData toDate(String str, String timezone) { return toDate(str, "yyyy-MM-dd", timezone); } - public static int toDate(String str, String format, String timezone) { + public static DateData toDate(String str, String format, String timezone) { return DateTimeUtils.parseDate(str, format, timezone); } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/InternalSerializers.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/InternalSerializers.java index cc15223031d..c0c9f3ec416 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/InternalSerializers.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/InternalSerializers.java @@ -22,11 +22,13 @@ import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.common.types.MapType; import org.apache.flink.cdc.runtime.serializer.data.ArrayDataSerializer; +import org.apache.flink.cdc.runtime.serializer.data.DateDataSerializer; import org.apache.flink.cdc.runtime.serializer.data.DecimalDataSerializer; import org.apache.flink.cdc.runtime.serializer.data.LocalZonedTimestampDataSerializer; import org.apache.flink.cdc.runtime.serializer.data.MapDataSerializer; import org.apache.flink.cdc.runtime.serializer.data.RecordDataSerializer; import org.apache.flink.cdc.runtime.serializer.data.StringDataSerializer; +import org.apache.flink.cdc.runtime.serializer.data.TimeDataSerializer; import org.apache.flink.cdc.runtime.serializer.data.TimestampDataSerializer; import org.apache.flink.cdc.runtime.serializer.data.ZonedTimestampDataSerializer; @@ -61,9 +63,11 @@ private static TypeSerializer createInternal(DataType type) { case SMALLINT: return ShortSerializer.INSTANCE; case INTEGER: + return IntSerializer.INSTANCE; case DATE: + return DateDataSerializer.INSTANCE; case TIME_WITHOUT_TIME_ZONE: - return IntSerializer.INSTANCE; + return TimeDataSerializer.INSTANCE; case BIGINT: return LongSerializer.INSTANCE; case FLOAT: diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DateDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DateDataSerializer.java new file mode 100644 index 00000000000..8c59a6db1f6 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DateDataSerializer.java @@ -0,0 +1,97 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.runtime.serializer.data; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.cdc.common.data.DateData; +import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** Serializer for {@link DateData}. */ +public final class DateDataSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + public static final DateDataSerializer INSTANCE = new DateDataSerializer(); + + private DateDataSerializer() {} + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public DateData createInstance() { + return DateData.fromEpochDay(0); + } + + @Override + public DateData copy(DateData from) { + return DateData.fromEpochDay(from.toEpochDay()); + } + + @Override + public DateData copy(DateData from, DateData reuse) { + return DateData.fromEpochDay(from.toEpochDay()); + } + + @Override + public int getLength() { + return 4; + } + + @Override + public void serialize(DateData record, DataOutputView target) throws IOException { + target.writeInt(record.toEpochDay()); + } + + @Override + public DateData deserialize(DataInputView source) throws IOException { + return DateData.fromEpochDay(source.readInt()); + } + + @Override + public DateData deserialize(DateData record, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + target.writeInt(source.readInt()); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DateDataSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class DateDataSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public DateDataSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimeDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimeDataSerializer.java new file mode 100644 index 00000000000..414a7d964b7 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimeDataSerializer.java @@ -0,0 +1,97 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.runtime.serializer.data; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.cdc.common.data.TimeData; +import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** Serializer for {@link TimeData}. */ +public final class TimeDataSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + public static final TimeDataSerializer INSTANCE = new TimeDataSerializer(); + + private TimeDataSerializer() {} + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public TimeData createInstance() { + return TimeData.fromNanoOfDay(0); + } + + @Override + public TimeData copy(TimeData from) { + return TimeData.fromMillisOfDay(from.toMillisOfDay()); + } + + @Override + public TimeData copy(TimeData from, TimeData reuse) { + return TimeData.fromMillisOfDay(from.toMillisOfDay()); + } + + @Override + public int getLength() { + return 4; + } + + @Override + public void serialize(TimeData record, DataOutputView target) throws IOException { + target.writeInt(record.toMillisOfDay()); + } + + @Override + public TimeData deserialize(DataInputView source) throws IOException { + return TimeData.fromMillisOfDay(source.readInt()); + } + + @Override + public TimeData deserialize(TimeData record, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + target.writeInt(source.readInt()); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new TimeDataSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class TimeDataSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public TimeDataSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/AbstractBinaryWriter.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/AbstractBinaryWriter.java index dd75c2d12fa..8663f22291f 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/AbstractBinaryWriter.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/AbstractBinaryWriter.java @@ -20,11 +20,13 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.data.ArrayData; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.MapData; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.data.StringData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryArrayData; @@ -224,6 +226,16 @@ public void writeZonedTimestamp(int pos, ZonedTimestampData value, int precision writeString(pos, new BinaryStringData(timestampString)); } + @Override + public void writeDate(int pos, DateData value) { + writeInt(pos, value.toEpochDay()); + } + + @Override + public void writeTime(int pos, TimeData value, int precision) { + writeInt(pos, value.toMillisOfDay()); + } + private void zeroBytes(int offset, int size) { for (int i = offset; i < offset + size; i++) { segment.put(i, (byte) 0); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/BinaryWriter.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/BinaryWriter.java index f25626de9f6..bf003448b3a 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/BinaryWriter.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/writer/BinaryWriter.java @@ -20,16 +20,19 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.data.ArrayData; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.MapData; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.data.StringData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.common.types.DecimalType; import org.apache.flink.cdc.common.types.LocalZonedTimestampType; +import org.apache.flink.cdc.common.types.TimeType; import org.apache.flink.cdc.common.types.TimestampType; import org.apache.flink.cdc.common.types.ZonedTimestampType; import org.apache.flink.cdc.runtime.serializer.NullableSerializerWrapper; @@ -82,6 +85,10 @@ public interface BinaryWriter { void writeRecord(int pos, RecordData value, TypeSerializer serializer); + void writeDate(int pos, DateData value); + + void writeTime(int pos, TimeData value, int precision); + /** Finally, complete write to set real size to binary. */ void complete(); @@ -98,9 +105,25 @@ static void write( writer.writeShort(pos, (short) o); break; case INTEGER: + writer.writeInt(pos, (int) o); + break; case DATE: + if (o instanceof DateData) { + writer.writeDate(pos, (DateData) o); + } else { + // This path is kept for backward compatibility, as legacy data might represent + // dates as integers (days since epoch). + writer.writeInt(pos, (int) o); + } + break; case TIME_WITHOUT_TIME_ZONE: - writer.writeInt(pos, (int) o); + if (o instanceof TimeData) { + writer.writeTime(pos, (TimeData) o, ((TimeType) type).getPrecision()); + } else { + // This path is kept for backward compatibility, as legacy data might represent + // time as integers (milliseconds of the day). + writer.writeInt(pos, (int) o); + } break; case BIGINT: writer.writeLong(pos, (long) o); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/DataTypeConverter.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/DataTypeConverter.java index 4d6dffcaf77..8175a79ada0 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/DataTypeConverter.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/DataTypeConverter.java @@ -18,11 +18,13 @@ package org.apache.flink.cdc.runtime.typeutils; import org.apache.flink.cdc.common.data.ArrayData; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.GenericArrayData; import org.apache.flink.cdc.common.data.GenericMapData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.MapData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; @@ -94,9 +96,9 @@ public static Class convertOriginalClass(DataType dataType) { case BIGINT: return Long.class; case DATE: - return Integer.class; + return DateData.class; case TIME_WITHOUT_TIME_ZONE: - return Integer.class; + return TimeData.class; case TIMESTAMP_WITHOUT_TIME_ZONE: return TimestampData.class; case TIMESTAMP_WITH_TIME_ZONE: @@ -170,10 +172,7 @@ public static RelDataType convertCalciteRelDataType( case TIME_WITHOUT_TIME_ZONE: TimeType timeType = (TimeType) column.getType(); fieldInfoBuilder - .add( - column.getName(), - SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE, - timeType.getPrecision()) + .add(column.getName(), SqlTypeName.TIME, timeType.getPrecision()) .nullable(timeType.isNullable()); break; case TIMESTAMP_WITHOUT_TIME_ZONE: @@ -309,8 +308,7 @@ public static RelDataType convertCalciteType( return typeFactory.createSqlType(SqlTypeName.DATE); case TIME_WITHOUT_TIME_ZONE: TimeType timeType = (TimeType) dataType; - return typeFactory.createSqlType( - SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE, timeType.getPrecision()); + return typeFactory.createSqlType(SqlTypeName.TIME, timeType.getPrecision()); case TIMESTAMP_WITHOUT_TIME_ZONE: TimestampType timestampType = (TimestampType) dataType; return typeFactory.createSqlType( @@ -382,7 +380,6 @@ public static DataType convertCalciteRelDataTypeToDataType(RelDataType relDataTy case DATE: return DataTypes.DATE(); case TIME: - case TIME_WITH_LOCAL_TIME_ZONE: return DataTypes.TIME(relDataType.getPrecision()); case TIMESTAMP: return DataTypes.TIMESTAMP(relDataType.getPrecision()); @@ -573,8 +570,11 @@ private static Object convertToDouble(Object obj) { } } - private static Object convertToDate(Object obj) { - return (int) toLocalDate(obj).toEpochDay(); + private static DateData convertToDate(Object obj) { + if (obj instanceof DateData) { + return (DateData) obj; + } + return DateData.fromLocalDate(toLocalDate(obj)); } private static LocalDate toLocalDate(Object obj) { @@ -613,12 +613,11 @@ private static LocalDate toLocalDate(Object obj) { + obj.getClass().getName()); } - private static Object convertToTime(Object obj) { - if (obj instanceof Integer) { - return obj; + private static TimeData convertToTime(Object obj) { + if (obj instanceof TimeData) { + return (TimeData) obj; } - // get number of milliseconds of the day - return toLocalTime(obj).toSecondOfDay() * 1000; + return TimeData.fromLocalTime(toLocalTime(obj)); } private static Object convertToArray(Object obj, ArrayType arrayType) { diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorTest.java index b3c585ef766..ed49bb3e373 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorTest.java @@ -17,8 +17,10 @@ package org.apache.flink.cdc.runtime.operators.transform; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; import org.apache.flink.cdc.common.event.CreateTableEvent; @@ -601,18 +603,18 @@ void testDataChangeEventTransformProjectionDataTypeConvert() throws Exception { DATATYPE_TABLEID, recordDataGenerator.generate( new Object[] { - new BinaryStringData("3.14"), - new Boolean(true), - new Byte("1"), - new Short("1"), - new Integer(1), - new Long(1), - new Integer(1704471599), - new Integer(1704471599), + BinaryStringData.fromString("3.14"), + true, + (byte) 1, + (short) 1, + 1, + 1L, + DateData.fromEpochDay(1704471599), + TimeData.fromMillisOfDay(1704471), TimestampData.fromMillis(1704471599), - new Float(3.14f), - new Double(3.14d), - DecimalData.fromBigDecimal(new BigDecimal(3.14), 6, 2), + 3.14f, + 3.14d, + DecimalData.fromBigDecimal(new BigDecimal("3.14"), 6, 2), })); transform.processElement(new StreamRecord<>(createTableEvent)); Assertions.assertThat( diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/data/DateDataSerializerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/data/DateDataSerializerTest.java new file mode 100644 index 00000000000..de4904e1138 --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/data/DateDataSerializerTest.java @@ -0,0 +1,56 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.runtime.serializer.data; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.cdc.common.data.DateData; +import org.apache.flink.cdc.runtime.serializer.SerializerTestBase; + +import java.time.LocalDate; + +/** A test for the {@link TimestampDataSerializer}. */ +class DateDataSerializerTest extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return DateDataSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return 4; + } + + @Override + protected Class getTypeClass() { + return DateData.class; + } + + @Override + protected DateData[] getTestData() { + return new DateData[] { + DateData.fromEpochDay(10240), + DateData.fromEpochDay(20480), + DateData.fromEpochDay(40960), + DateData.fromIsoLocalDateString("2014-08-15"), + DateData.fromIsoLocalDateString("2001-03-15"), + DateData.fromIsoLocalDateString("2023-09-19"), + DateData.fromLocalDate(LocalDate.of(2012, 12, 22)), + DateData.fromLocalDate(LocalDate.of(1999, 12, 31)) + }; + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/data/TimeDataSerializerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/data/TimeDataSerializerTest.java new file mode 100644 index 00000000000..692ab444468 --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/data/TimeDataSerializerTest.java @@ -0,0 +1,62 @@ +/* + * 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. + */ + +package org.apache.flink.cdc.runtime.serializer.data; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.cdc.common.data.TimeData; +import org.apache.flink.cdc.runtime.serializer.SerializerTestBase; + +import java.time.LocalTime; + +/** A test for the {@link TimestampDataSerializer}. */ +class TimeDataSerializerTest extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return TimeDataSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return 4; + } + + @Override + protected Class getTypeClass() { + return TimeData.class; + } + + @Override + protected TimeData[] getTestData() { + return new TimeData[] { + TimeData.fromSecondOfDay(1024), + TimeData.fromSecondOfDay(2048), + TimeData.fromSecondOfDay(4096), + TimeData.fromMillisOfDay(10240), + TimeData.fromMillisOfDay(20480), + TimeData.fromMillisOfDay(40960), + TimeData.fromNanoOfDay(102400), + TimeData.fromNanoOfDay(204800), + TimeData.fromNanoOfDay(409600), + TimeData.fromIsoLocalTimeString("14:28:25"), + TimeData.fromIsoLocalTimeString("01:23:45"), + TimeData.fromIsoLocalTimeString("23:59:59"), + TimeData.fromLocalTime(LocalTime.MIDNIGHT), + TimeData.fromLocalTime(LocalTime.NOON) + }; + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/typeutils/BinaryRecordDataExtractorTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/typeutils/BinaryRecordDataExtractorTest.java index 98abd82e922..e73f38fa1f1 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/typeutils/BinaryRecordDataExtractorTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/typeutils/BinaryRecordDataExtractorTest.java @@ -17,11 +17,13 @@ package org.apache.flink.cdc.runtime.typeutils; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.GenericArrayData; import org.apache.flink.cdc.common.data.GenericMapData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryStringData; @@ -34,7 +36,9 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; +import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.LocalTime; import java.time.ZoneId; import java.util.ArrayList; import java.util.List; @@ -57,7 +61,7 @@ public class BinaryRecordDataExtractorTest { .physicalColumn("bin_col", DataTypes.BINARY(17)) .physicalColumn("varbin_col", DataTypes.VARBINARY(17)) .physicalColumn("date_col", DataTypes.DATE()) - .physicalColumn("time_col", DataTypes.TIME()) + .physicalColumn("time_col", DataTypes.TIME(9)) .physicalColumn("ts_col", DataTypes.TIMESTAMP(3)) .physicalColumn("ts_tz_col", DataTypes.TIMESTAMP_TZ(3)) .physicalColumn("ts_ltz_col", DataTypes.TIMESTAMP_LTZ(3)) @@ -88,8 +92,8 @@ public static List generateEventWithAllTypes() { BinaryStringData.fromString("Nine"), "Ten\1".getBytes(), "Eleven\2".getBytes(), - 19673, - (8 * 3600 + 30 * 60 + 15) * 1000, + DateData.fromLocalDate(LocalDate.of(2019, 12, 31)), + TimeData.fromLocalTime(LocalTime.of(8, 30, 17, 123456789)), TimestampData.fromLocalDateTime( LocalDateTime.of(2023, 11, 11, 11, 11, 11, 11)), ZonedTimestampData.fromZonedDateTime( @@ -131,8 +135,8 @@ public static List generateEventWithAllTypes() { BinaryStringData.fromString("-Nine"), "-Ten\1".getBytes(), "-Eleven\2".getBytes(), - 2000, - (8 * 3600 + 30 * 60 + 17) * 1000, + DateData.fromLocalDate(LocalDate.of(2019, 12, 31)), + TimeData.fromLocalTime(LocalTime.of(8, 30, 17, 123456789)), TimestampData.fromLocalDateTime( LocalDateTime.of(2021, 11, 11, 11, 11, 11, 11)), ZonedTimestampData.fromZonedDateTime( @@ -166,24 +170,24 @@ public static List generateEventWithAllTypes() { } @Test - void testConvertingBinaryRecordData() throws Exception { + void testConvertingBinaryRecordData() { Assertions.assertThat(generateEventWithAllTypes()) .map(e -> BinaryRecordDataExtractor.extractRecord(e, SCHEMA.toRowDataType())) .containsExactly( - "{id: INT NOT NULL -> 1, bool_col: BOOLEAN -> true, tinyint_col: TINYINT -> 2, smallint_col: SMALLINT -> 3, int_col: INT -> 4, bigint_col: BIGINT -> 5, float_col: FLOAT -> 6.0, double_col: DOUBLE -> 7.0, decimal_col: DECIMAL(17, 10) -> 0.1234567890, char_col: CHAR(17) -> Eight, varchar_col: VARCHAR(17) -> Nine, bin_col: BINARY(17) -> VGVuAQ==, varbin_col: VARBINARY(17) -> RWxldmVuAg==, date_col: DATE -> 19673, time_col: TIME(0) -> 30615000, ts_col: TIMESTAMP(3) -> 2023-11-11T11:11:11.000000011, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> 2023-11-11T11:11:11.000000011+05:00, ts_ltz_col: TIMESTAMP_LTZ(3) -> 2023-11-11T06:11:11.000000011, array_col: ARRAY -> [One, Two, Three], map_col: MAP -> {1 -> yi, 2 -> er, 3 -> san}, row_col: ROW<`f0` INT, `f1` DOUBLE> -> {f0: INT -> 3, f1: DOUBLE -> 0.1415926}}", - "{id: INT NOT NULL -> -1, bool_col: BOOLEAN -> false, tinyint_col: TINYINT -> -2, smallint_col: SMALLINT -> -3, int_col: INT -> -4, bigint_col: BIGINT -> -5, float_col: FLOAT -> -6.0, double_col: DOUBLE -> -7.0, decimal_col: DECIMAL(17, 10) -> -0.1234567890, char_col: CHAR(17) -> -Eight, varchar_col: VARCHAR(17) -> -Nine, bin_col: BINARY(17) -> LVRlbgE=, varbin_col: VARBINARY(17) -> LUVsZXZlbgI=, date_col: DATE -> 2000, time_col: TIME(0) -> 30617000, ts_col: TIMESTAMP(3) -> 2021-11-11T11:11:11.000000011, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> 2021-11-11T11:11:11.000000011+05:00, ts_ltz_col: TIMESTAMP_LTZ(3) -> 2021-11-11T06:11:11.000000011, array_col: ARRAY -> [Ninety, Eighty, Seventy], map_col: MAP -> {7 -> qi, 8 -> ba, 9 -> jiu}, row_col: ROW<`f0` INT, `f1` DOUBLE> -> {f0: INT -> 2, f1: DOUBLE -> 0.718281828}}", - "{id: INT NOT NULL -> 0, bool_col: BOOLEAN -> null, tinyint_col: TINYINT -> null, smallint_col: SMALLINT -> null, int_col: INT -> null, bigint_col: BIGINT -> null, float_col: FLOAT -> null, double_col: DOUBLE -> null, decimal_col: DECIMAL(17, 10) -> null, char_col: CHAR(17) -> null, varchar_col: VARCHAR(17) -> null, bin_col: BINARY(17) -> null, varbin_col: VARBINARY(17) -> null, date_col: DATE -> null, time_col: TIME(0) -> null, ts_col: TIMESTAMP(3) -> null, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> null, ts_ltz_col: TIMESTAMP_LTZ(3) -> null, array_col: ARRAY -> null, map_col: MAP -> null, row_col: ROW<`f0` INT, `f1` DOUBLE> -> null}", + "{id: INT NOT NULL -> 1, bool_col: BOOLEAN -> true, tinyint_col: TINYINT -> 2, smallint_col: SMALLINT -> 3, int_col: INT -> 4, bigint_col: BIGINT -> 5, float_col: FLOAT -> 6.0, double_col: DOUBLE -> 7.0, decimal_col: DECIMAL(17, 10) -> 0.1234567890, char_col: CHAR(17) -> Eight, varchar_col: VARCHAR(17) -> Nine, bin_col: BINARY(17) -> VGVuAQ==, varbin_col: VARBINARY(17) -> RWxldmVuAg==, date_col: DATE -> 2019-12-31, time_col: TIME(9) -> 08:30:17.123, ts_col: TIMESTAMP(3) -> 2023-11-11T11:11:11.000000011, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> 2023-11-11T11:11:11.000000011+05:00, ts_ltz_col: TIMESTAMP_LTZ(3) -> 2023-11-11T06:11:11.000000011, array_col: ARRAY -> [One, Two, Three], map_col: MAP -> {1 -> yi, 2 -> er, 3 -> san}, row_col: ROW<`f0` INT, `f1` DOUBLE> -> {f0: INT -> 3, f1: DOUBLE -> 0.1415926}}", + "{id: INT NOT NULL -> -1, bool_col: BOOLEAN -> false, tinyint_col: TINYINT -> -2, smallint_col: SMALLINT -> -3, int_col: INT -> -4, bigint_col: BIGINT -> -5, float_col: FLOAT -> -6.0, double_col: DOUBLE -> -7.0, decimal_col: DECIMAL(17, 10) -> -0.1234567890, char_col: CHAR(17) -> -Eight, varchar_col: VARCHAR(17) -> -Nine, bin_col: BINARY(17) -> LVRlbgE=, varbin_col: VARBINARY(17) -> LUVsZXZlbgI=, date_col: DATE -> 2019-12-31, time_col: TIME(9) -> 08:30:17.123, ts_col: TIMESTAMP(3) -> 2021-11-11T11:11:11.000000011, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> 2021-11-11T11:11:11.000000011+05:00, ts_ltz_col: TIMESTAMP_LTZ(3) -> 2021-11-11T06:11:11.000000011, array_col: ARRAY -> [Ninety, Eighty, Seventy], map_col: MAP -> {7 -> qi, 8 -> ba, 9 -> jiu}, row_col: ROW<`f0` INT, `f1` DOUBLE> -> {f0: INT -> 2, f1: DOUBLE -> 0.718281828}}", + "{id: INT NOT NULL -> 0, bool_col: BOOLEAN -> null, tinyint_col: TINYINT -> null, smallint_col: SMALLINT -> null, int_col: INT -> null, bigint_col: BIGINT -> null, float_col: FLOAT -> null, double_col: DOUBLE -> null, decimal_col: DECIMAL(17, 10) -> null, char_col: CHAR(17) -> null, varchar_col: VARCHAR(17) -> null, bin_col: BINARY(17) -> null, varbin_col: VARBINARY(17) -> null, date_col: DATE -> null, time_col: TIME(9) -> null, ts_col: TIMESTAMP(3) -> null, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> null, ts_ltz_col: TIMESTAMP_LTZ(3) -> null, array_col: ARRAY -> null, map_col: MAP -> null, row_col: ROW<`f0` INT, `f1` DOUBLE> -> null}", "null"); } @Test - void testConvertingBinaryRecordDataWithSchema() throws Exception { + void testConvertingBinaryRecordDataWithSchema() { Assertions.assertThat(generateEventWithAllTypes()) .map(e -> BinaryRecordDataExtractor.extractRecord(e, SCHEMA)) .containsExactly( - "{id: INT NOT NULL -> 1, bool_col: BOOLEAN -> true, tinyint_col: TINYINT -> 2, smallint_col: SMALLINT -> 3, int_col: INT -> 4, bigint_col: BIGINT -> 5, float_col: FLOAT -> 6.0, double_col: DOUBLE -> 7.0, decimal_col: DECIMAL(17, 10) -> 0.1234567890, char_col: CHAR(17) -> Eight, varchar_col: VARCHAR(17) -> Nine, bin_col: BINARY(17) -> VGVuAQ==, varbin_col: VARBINARY(17) -> RWxldmVuAg==, date_col: DATE -> 19673, time_col: TIME(0) -> 30615000, ts_col: TIMESTAMP(3) -> 2023-11-11T11:11:11.000000011, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> 2023-11-11T11:11:11.000000011+05:00, ts_ltz_col: TIMESTAMP_LTZ(3) -> 2023-11-11T06:11:11.000000011, array_col: ARRAY -> [One, Two, Three], map_col: MAP -> {1 -> yi, 2 -> er, 3 -> san}, row_col: ROW<`f0` INT, `f1` DOUBLE> -> {f0: INT -> 3, f1: DOUBLE -> 0.1415926}}", - "{id: INT NOT NULL -> -1, bool_col: BOOLEAN -> false, tinyint_col: TINYINT -> -2, smallint_col: SMALLINT -> -3, int_col: INT -> -4, bigint_col: BIGINT -> -5, float_col: FLOAT -> -6.0, double_col: DOUBLE -> -7.0, decimal_col: DECIMAL(17, 10) -> -0.1234567890, char_col: CHAR(17) -> -Eight, varchar_col: VARCHAR(17) -> -Nine, bin_col: BINARY(17) -> LVRlbgE=, varbin_col: VARBINARY(17) -> LUVsZXZlbgI=, date_col: DATE -> 2000, time_col: TIME(0) -> 30617000, ts_col: TIMESTAMP(3) -> 2021-11-11T11:11:11.000000011, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> 2021-11-11T11:11:11.000000011+05:00, ts_ltz_col: TIMESTAMP_LTZ(3) -> 2021-11-11T06:11:11.000000011, array_col: ARRAY -> [Ninety, Eighty, Seventy], map_col: MAP -> {7 -> qi, 8 -> ba, 9 -> jiu}, row_col: ROW<`f0` INT, `f1` DOUBLE> -> {f0: INT -> 2, f1: DOUBLE -> 0.718281828}}", - "{id: INT NOT NULL -> 0, bool_col: BOOLEAN -> null, tinyint_col: TINYINT -> null, smallint_col: SMALLINT -> null, int_col: INT -> null, bigint_col: BIGINT -> null, float_col: FLOAT -> null, double_col: DOUBLE -> null, decimal_col: DECIMAL(17, 10) -> null, char_col: CHAR(17) -> null, varchar_col: VARCHAR(17) -> null, bin_col: BINARY(17) -> null, varbin_col: VARBINARY(17) -> null, date_col: DATE -> null, time_col: TIME(0) -> null, ts_col: TIMESTAMP(3) -> null, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> null, ts_ltz_col: TIMESTAMP_LTZ(3) -> null, array_col: ARRAY -> null, map_col: MAP -> null, row_col: ROW<`f0` INT, `f1` DOUBLE> -> null}", + "{id: INT NOT NULL -> 1, bool_col: BOOLEAN -> true, tinyint_col: TINYINT -> 2, smallint_col: SMALLINT -> 3, int_col: INT -> 4, bigint_col: BIGINT -> 5, float_col: FLOAT -> 6.0, double_col: DOUBLE -> 7.0, decimal_col: DECIMAL(17, 10) -> 0.1234567890, char_col: CHAR(17) -> Eight, varchar_col: VARCHAR(17) -> Nine, bin_col: BINARY(17) -> VGVuAQ==, varbin_col: VARBINARY(17) -> RWxldmVuAg==, date_col: DATE -> 2019-12-31, time_col: TIME(9) -> 08:30:17.123, ts_col: TIMESTAMP(3) -> 2023-11-11T11:11:11.000000011, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> 2023-11-11T11:11:11.000000011+05:00, ts_ltz_col: TIMESTAMP_LTZ(3) -> 2023-11-11T06:11:11.000000011, array_col: ARRAY -> [One, Two, Three], map_col: MAP -> {1 -> yi, 2 -> er, 3 -> san}, row_col: ROW<`f0` INT, `f1` DOUBLE> -> {f0: INT -> 3, f1: DOUBLE -> 0.1415926}}", + "{id: INT NOT NULL -> -1, bool_col: BOOLEAN -> false, tinyint_col: TINYINT -> -2, smallint_col: SMALLINT -> -3, int_col: INT -> -4, bigint_col: BIGINT -> -5, float_col: FLOAT -> -6.0, double_col: DOUBLE -> -7.0, decimal_col: DECIMAL(17, 10) -> -0.1234567890, char_col: CHAR(17) -> -Eight, varchar_col: VARCHAR(17) -> -Nine, bin_col: BINARY(17) -> LVRlbgE=, varbin_col: VARBINARY(17) -> LUVsZXZlbgI=, date_col: DATE -> 2019-12-31, time_col: TIME(9) -> 08:30:17.123, ts_col: TIMESTAMP(3) -> 2021-11-11T11:11:11.000000011, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> 2021-11-11T11:11:11.000000011+05:00, ts_ltz_col: TIMESTAMP_LTZ(3) -> 2021-11-11T06:11:11.000000011, array_col: ARRAY -> [Ninety, Eighty, Seventy], map_col: MAP -> {7 -> qi, 8 -> ba, 9 -> jiu}, row_col: ROW<`f0` INT, `f1` DOUBLE> -> {f0: INT -> 2, f1: DOUBLE -> 0.718281828}}", + "{id: INT NOT NULL -> 0, bool_col: BOOLEAN -> null, tinyint_col: TINYINT -> null, smallint_col: SMALLINT -> null, int_col: INT -> null, bigint_col: BIGINT -> null, float_col: FLOAT -> null, double_col: DOUBLE -> null, decimal_col: DECIMAL(17, 10) -> null, char_col: CHAR(17) -> null, varchar_col: VARCHAR(17) -> null, bin_col: BINARY(17) -> null, varbin_col: VARBINARY(17) -> null, date_col: DATE -> null, time_col: TIME(9) -> null, ts_col: TIMESTAMP(3) -> null, ts_tz_col: TIMESTAMP(3) WITH TIME ZONE -> null, ts_ltz_col: TIMESTAMP_LTZ(3) -> null, array_col: ARRAY -> null, map_col: MAP -> null, row_col: ROW<`f0` INT, `f1` DOUBLE> -> null}", "null"); } } diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/typeutils/BinaryRecordDataGeneratorTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/typeutils/BinaryRecordDataGeneratorTest.java index 1b5fb746dfb..847d3613060 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/typeutils/BinaryRecordDataGeneratorTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/typeutils/BinaryRecordDataGeneratorTest.java @@ -17,8 +17,10 @@ package org.apache.flink.cdc.runtime.typeutils; +import org.apache.flink.cdc.common.data.DateData; import org.apache.flink.cdc.common.data.DecimalData; import org.apache.flink.cdc.common.data.LocalZonedTimestampData; +import org.apache.flink.cdc.common.data.TimeData; import org.apache.flink.cdc.common.data.TimestampData; import org.apache.flink.cdc.common.data.ZonedTimestampData; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; @@ -57,6 +59,7 @@ void testOf() { DataTypes.VARCHAR(10), DataTypes.STRING(), DataTypes.DATE(), + DataTypes.DATE(), DataTypes.TIME(), DataTypes.TIME(6), DataTypes.TIMESTAMP(), @@ -86,9 +89,10 @@ void testOf() { BinaryStringData.fromString("test1"), BinaryStringData.fromString("test2"), BinaryStringData.fromString("test3"), - 100, - 200, - 300, + DateData.fromEpochDay(1000), + DateData.fromEpochDay(1000).toEpochDay(), + TimeData.fromMillisOfDay(200), + TimeData.fromMillisOfDay(300).toMillisOfDay(), TimestampData.fromMillis(100, 1), TimestampData.fromMillis(200, 0), LocalZonedTimestampData.fromEpochMillis(300, 1), @@ -97,7 +101,7 @@ void testOf() { ZonedTimestampData.of(600, 0, "UTC"), new BinaryRecordDataGenerator( RowType.of(DataTypes.STRING(), DataTypes.BIGINT())) - .generate(new Object[] {BinaryStringData.fromString("test"), 23L}), + .generate(new Object[] {BinaryStringData.fromString("test"), 24L}), null }; BinaryRecordData actual = new BinaryRecordDataGenerator(rowType).generate(testData); @@ -120,23 +124,24 @@ void testOf() { assertThat(actual.getString(12)).isEqualTo(BinaryStringData.fromString("test2")); assertThat(actual.getString(13)).isEqualTo(BinaryStringData.fromString("test3")); - assertThat(actual.getInt(14)).isEqualTo(testData[14]); - assertThat(actual.getInt(15)).isEqualTo(testData[15]); - assertThat(actual.getInt(16)).isEqualTo(testData[16]); + assertThat(actual.getDate(14)).isEqualTo(testData[14]); + assertThat(actual.getDate(15).toEpochDay()).isEqualTo(testData[15]); + assertThat(actual.getTime(16)).isEqualTo(testData[16]); + assertThat(actual.getTime(17).toMillisOfDay()).isEqualTo(testData[17]); - assertThat(actual.getTimestamp(17, TimestampType.DEFAULT_PRECISION)) - .isEqualTo(testData[17]); - assertThat(actual.getTimestamp(18, 3)).isEqualTo(testData[18]); - assertThat(actual.getLocalZonedTimestampData(19, LocalZonedTimestampType.DEFAULT_PRECISION)) - .isEqualTo(testData[19]); - assertThat(actual.getLocalZonedTimestampData(20, 3)).isEqualTo(testData[20]); - assertThat(actual.getZonedTimestamp(21, ZonedTimestampType.DEFAULT_PRECISION)) - .isEqualTo(testData[21]); - assertThat(actual.getZonedTimestamp(22, 3)).isEqualTo(testData[22]); + assertThat(actual.getTimestamp(18, TimestampType.DEFAULT_PRECISION)) + .isEqualTo(testData[18]); + assertThat(actual.getTimestamp(19, 3)).isEqualTo(testData[19]); + assertThat(actual.getLocalZonedTimestampData(20, LocalZonedTimestampType.DEFAULT_PRECISION)) + .isEqualTo(testData[20]); + assertThat(actual.getLocalZonedTimestampData(21, 3)).isEqualTo(testData[21]); + assertThat(actual.getZonedTimestamp(22, ZonedTimestampType.DEFAULT_PRECISION)) + .isEqualTo(testData[22]); + assertThat(actual.getZonedTimestamp(23, 3)).isEqualTo(testData[23]); - assertThat(actual.getRow(23, 2).getString(0)) + assertThat(actual.getRow(24, 2).getString(0)) .isEqualTo(BinaryStringData.fromString("test")); - assertThat(actual.getRow(23, 2).getLong(1)).isEqualTo(23L); - assertThat(actual.isNullAt(24)).isTrue(); + assertThat(actual.getRow(24, 2).getLong(1)).isEqualTo(24L); + assertThat(actual.isNullAt(25)).isTrue(); } }