-
Notifications
You must be signed in to change notification settings - Fork 2.1k
[FLINK-38835][postgres] Fix timestamp conversion to LocalDateTime for dates before 1970-01-01. #4214
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
[FLINK-38835][postgres] Fix timestamp conversion to LocalDateTime for dates before 1970-01-01. #4214
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,99 @@ | ||
| /* | ||
| * 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 io.debezium.connector.postgresql; | ||
|
|
||
| import io.debezium.config.CommonConnectorConfig; | ||
| import io.debezium.jdbc.TemporalPrecisionMode; | ||
| import io.debezium.relational.Column; | ||
| import org.apache.kafka.connect.data.Field; | ||
|
|
||
| import java.nio.charset.Charset; | ||
| import java.sql.Timestamp; | ||
| import java.time.ZoneOffset; | ||
|
|
||
| /** | ||
| * A custom PostgresValueConverter that correctly handles timestamp conversion to LocalDateTime for | ||
| * dates before 1970-01-01. | ||
| */ | ||
| public class CustomPostgresValueConverter extends PostgresValueConverter { | ||
| protected CustomPostgresValueConverter( | ||
| Charset databaseCharset, | ||
| DecimalMode decimalMode, | ||
| TemporalPrecisionMode temporalPrecisionMode, | ||
| ZoneOffset defaultOffset, | ||
| BigIntUnsignedMode bigIntUnsignedMode, | ||
| boolean includeUnknownDatatypes, | ||
| TypeRegistry typeRegistry, | ||
| PostgresConnectorConfig.HStoreHandlingMode hStoreMode, | ||
| CommonConnectorConfig.BinaryHandlingMode binaryMode, | ||
| PostgresConnectorConfig.IntervalHandlingMode intervalMode, | ||
| byte[] toastPlaceholder, | ||
| int moneyFractionDigits) { | ||
| super( | ||
| databaseCharset, | ||
| decimalMode, | ||
| temporalPrecisionMode, | ||
| defaultOffset, | ||
| bigIntUnsignedMode, | ||
| includeUnknownDatatypes, | ||
| typeRegistry, | ||
| hStoreMode, | ||
| binaryMode, | ||
| intervalMode, | ||
| toastPlaceholder, | ||
| moneyFractionDigits); | ||
| } | ||
|
|
||
| public static CustomPostgresValueConverter of( | ||
| PostgresConnectorConfig connectorConfig, | ||
| Charset databaseCharset, | ||
| TypeRegistry typeRegistry) { | ||
| return new CustomPostgresValueConverter( | ||
| databaseCharset, | ||
| connectorConfig.getDecimalMode(), | ||
| connectorConfig.getTemporalPrecisionMode(), | ||
| ZoneOffset.UTC, | ||
| null, | ||
| connectorConfig.includeUnknownDatatypes(), | ||
| typeRegistry, | ||
| connectorConfig.hStoreHandlingMode(), | ||
| connectorConfig.binaryHandlingMode(), | ||
| connectorConfig.intervalHandlingMode(), | ||
| connectorConfig.getUnavailableValuePlaceholder(), | ||
| connectorConfig.moneyFractionDigits()); | ||
| } | ||
|
|
||
| @Override | ||
| protected Object convertTimestampToLocalDateTime(Column column, Field fieldDefn, Object data) { | ||
| if (data == null) { | ||
| return null; | ||
| } | ||
| if (!(data instanceof Timestamp)) { | ||
| return data; | ||
| } | ||
| final Timestamp timestamp = (Timestamp) data; | ||
|
|
||
| if (POSITIVE_INFINITY_TIMESTAMP.equals(timestamp)) { | ||
| return POSITIVE_INFINITY_LOCAL_DATE_TIME; | ||
| } else if (NEGATIVE_INFINITY_TIMESTAMP.equals(timestamp)) { | ||
| return NEGATIVE_INFINITY_LOCAL_DATE_TIME; | ||
| } | ||
|
|
||
| return timestamp.toLocalDateTime(); | ||
|
Comment on lines
+82
to
+97
|
||
| } | ||
|
Comment on lines
+81
to
+98
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -689,18 +689,21 @@ void testAllTypes(boolean parallelismSnapshot) throws Throwable { | |
| // generate WAL | ||
| try (Connection connection = getJdbcConnection(POSTGIS_CONTAINER); | ||
| Statement statement = connection.createStatement()) { | ||
| statement.execute("UPDATE inventory.full_types SET small_c=0 WHERE id=1;"); | ||
| statement.execute("UPDATE inventory.full_types SET small_c=0 WHERE id<=2;"); | ||
| } | ||
|
|
||
| waitForSinkSize("sink", 3); | ||
| waitForSinkSize("sink", 6); | ||
|
|
||
| List<String> expected = | ||
| Arrays.asList( | ||
| "+I(1,[50],32767,65535,2147483647,5.5,6.6,123.12345,404.4,true,Hello World,a,abc,abcd..xyz,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17,18:00:22,500,{\"hexewkb\":\"0105000020e610000001000000010200000002000000a779c7293a2465400b462575025a46c0c66d3480b7fc6440c3d32b65195246c0\",\"srid\":4326},{\"hexewkb\":\"0101000020730c00001c7c613255de6540787aa52c435c42c0\",\"srid\":3187})", | ||
| "+I(2,[50],32767,65535,2147483647,5.5,6.6,123.12345,404.4,true,Hello World,a,abc,abcd..xyz,1900-01-01T00:00:00.123,1900-01-01T00:00:00.123456,1900-01-01,18:00:22,500,{\"hexewkb\":\"0105000020e610000001000000010200000002000000a779c7293a2465400b462575025a46c0c66d3480b7fc6440c3d32b65195246c0\",\"srid\":4326},{\"hexewkb\":\"0101000020730c00001c7c613255de6540787aa52c435c42c0\",\"srid\":3187})", | ||
| "-D(1,[50],32767,65535,2147483647,5.5,6.6,123.12345,404.4,true,Hello World,a,abc,abcd..xyz,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17,18:00:22,500,{\"hexewkb\":\"0105000020e610000001000000010200000002000000a779c7293a2465400b462575025a46c0c66d3480b7fc6440c3d32b65195246c0\",\"srid\":4326},{\"hexewkb\":\"0101000020730c00001c7c613255de6540787aa52c435c42c0\",\"srid\":3187})", | ||
| "+I(1,[50],0,65535,2147483647,5.5,6.6,123.12345,404.4,true,Hello World,a,abc,abcd..xyz,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17,18:00:22,500,{\"hexewkb\":\"0105000020e610000001000000010200000002000000a779c7293a2465400b462575025a46c0c66d3480b7fc6440c3d32b65195246c0\",\"srid\":4326},{\"hexewkb\":\"0101000020730c00001c7c613255de6540787aa52c435c42c0\",\"srid\":3187})"); | ||
| "-D(2,[50],32767,65535,2147483647,5.5,6.6,123.12345,404.4,true,Hello World,a,abc,abcd..xyz,1900-01-01T00:00:00.123,1900-01-01T00:00:00.123456,1900-01-01,18:00:22,500,{\"hexewkb\":\"0105000020e610000001000000010200000002000000a779c7293a2465400b462575025a46c0c66d3480b7fc6440c3d32b65195246c0\",\"srid\":4326},{\"hexewkb\":\"0101000020730c00001c7c613255de6540787aa52c435c42c0\",\"srid\":3187})", | ||
| "+I(1,[50],0,65535,2147483647,5.5,6.6,123.12345,404.4,true,Hello World,a,abc,abcd..xyz,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17,18:00:22,500,{\"hexewkb\":\"0105000020e610000001000000010200000002000000a779c7293a2465400b462575025a46c0c66d3480b7fc6440c3d32b65195246c0\",\"srid\":4326},{\"hexewkb\":\"0101000020730c00001c7c613255de6540787aa52c435c42c0\",\"srid\":3187})", | ||
| "+I(2,[50],0,65535,2147483647,5.5,6.6,123.12345,404.4,true,Hello World,a,abc,abcd..xyz,1900-01-01T00:00:00.123,1900-01-01T00:00:00.123456,1900-01-01,18:00:22,500,{\"hexewkb\":\"0105000020e610000001000000010200000002000000a779c7293a2465400b462575025a46c0c66d3480b7fc6440c3d32b65195246c0\",\"srid\":4326},{\"hexewkb\":\"0101000020730c00001c7c613255de6540787aa52c435c42c0\",\"srid\":3187})"); | ||
| List<String> actual = TestValuesTableFactory.getRawResultsAsStrings("sink"); | ||
| Assertions.assertThat(actual).isEqualTo(expected); | ||
| Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); | ||
|
||
|
|
||
| result.getJobClient().get().cancel().get(); | ||
| } | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The overridden method convertTimestampToLocalDateTime should include a JavaDoc comment explaining why this override is necessary and how it differs from the parent implementation. Specifically, it should document that this method uses Timestamp.toLocalDateTime() to properly handle dates before 1970-01-01 by avoiding timezone offset inconsistencies.