Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,8 @@
import static com.facebook.presto.plugin.jdbc.JdbcWarningCode.USE_OF_DEPRECATED_CONFIGURATION_PROPERTY;
import static com.facebook.presto.plugin.jdbc.mapping.StandardColumnMappings.jdbcTypeToReadMapping;
import static com.facebook.presto.plugin.jdbc.mapping.StandardColumnMappings.prestoTypeToWriteMapping;
import static com.facebook.presto.plugin.jdbc.mapping.StandardColumnMappings.timestampReadMapping;
import static com.facebook.presto.plugin.jdbc.mapping.StandardColumnMappings.timestampReadMappingLegacy;
import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.google.common.base.MoreObjects.firstNonNull;
Expand Down Expand Up @@ -276,6 +278,10 @@ public List<JdbcColumnHandle> getColumns(ConnectorSession session, JdbcTableHand
@Override
public Optional<ReadMapping> toPrestoType(ConnectorSession session, JdbcTypeHandle typeHandle)
{
if (typeHandle.getJdbcType() == java.sql.Types.TIMESTAMP) {
boolean legacyTimestamp = session.getSqlFunctionProperties().isLegacyTimestamp();
return Optional.of(legacyTimestamp ? timestampReadMappingLegacy() : timestampReadMapping());
}
return jdbcTypeToReadMapping(typeHandle);
}

Expand Down Expand Up @@ -794,9 +800,9 @@ protected String toSqlType(Type type)
throw new PrestoException(NOT_SUPPORTED, "Unsupported column type: " + type.getDisplayName());
}

public WriteMapping toWriteMapping(Type type)
public WriteMapping toWriteMapping(ConnectorSession session, Type type)
{
Optional<WriteMapping> writeMapping = prestoTypeToWriteMapping(type);
Optional<WriteMapping> writeMapping = prestoTypeToWriteMapping(session, type);
if (writeMapping.isPresent()) {
return writeMapping.get();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ default boolean schemaExists(ConnectorSession session, JdbcIdentity identity, St

Optional<ReadMapping> toPrestoType(ConnectorSession session, JdbcTypeHandle typeHandle);

WriteMapping toWriteMapping(Type type);
WriteMapping toWriteMapping(ConnectorSession session, Type type);

ConnectorSplitSource getSplits(ConnectorSession session, JdbcIdentity identity, JdbcTableLayoutHandle layoutHandle);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public JdbcPageSink(ConnectorSession session, JdbcOutputTableHandle handle, Jdbc

columnTypes = handle.getColumnTypes();
columnWriters = columnTypes.stream().map(type -> {
WriteFunction writeFunction = jdbcClient.toWriteMapping(type).getWriteFunction();
WriteFunction writeFunction = jdbcClient.toWriteMapping(session, type).getWriteFunction();
verify(type.getJavaType() == writeFunction.getJavaType(),
format("Presto type %s is not compatible with write function %s accepting %s", type, writeFunction, writeFunction.getJavaType()));
return writeFunction;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.facebook.presto.common.type.UuidType;
import com.facebook.presto.common.type.VarcharType;
import com.facebook.presto.plugin.jdbc.JdbcTypeHandle;
import com.facebook.presto.spi.ConnectorSession;
import com.google.common.base.CharMatcher;
import com.google.common.primitives.Shorts;
import com.google.common.primitives.SignedBytes;
Expand All @@ -38,7 +39,9 @@
import java.sql.Timestamp;
import java.sql.Types;
import java.time.Instant;
import java.util.Calendar;
import java.util.Optional;
import java.util.TimeZone;

import static com.facebook.presto.common.type.BigintType.BIGINT;
import static com.facebook.presto.common.type.BooleanType.BOOLEAN;
Expand Down Expand Up @@ -91,6 +94,7 @@ public final class StandardColumnMappings
private StandardColumnMappings() {}

private static final ISOChronology UTC_CHRONOLOGY = ISOChronology.getInstanceUTC();
private static final Calendar UTC_CALENDAR = Calendar.getInstance(TimeZone.getTimeZone("UTC"));

public static ReadMapping booleanReadMapping()
{
Expand Down Expand Up @@ -260,22 +264,37 @@ public static WriteMapping timeWriteMapping()
public static ReadMapping timestampReadMapping()
{
return createLongReadMapping(TIMESTAMP, (resultSet, columnIndex) -> {
/*
* TODO `resultSet.getTimestamp(columnIndex)` returns wrong value if JVM's zone had forward offset change and the local time
* corresponding to timestamp value being retrieved was not present (a 'gap'), this includes regular DST changes (e.g. Europe/Warsaw)
* and one-time policy changes (Asia/Kathmandu's shift by 15 minutes on January 1, 1986, 00:00:00).
* The problem can be averted by using `resultSet.getObject(columnIndex, LocalDateTime.class)` -- but this is not universally supported by JDBC drivers.
*/
Timestamp timestamp = resultSet.getTimestamp(columnIndex, UTC_CALENDAR);
return timestamp.getTime();
});
}

@Deprecated
public static ReadMapping timestampReadMappingLegacy()
{
return createLongReadMapping(TIMESTAMP, (resultSet, columnIndex) -> {
Timestamp timestamp = resultSet.getTimestamp(columnIndex);
return timestamp.getTime();
});
}

public static WriteMapping timestampWriteMapping(TimestampType timestampType)
{
return createLongWriteMapping((statement, index, value) -> statement.setTimestamp(index, Timestamp.from(Instant.ofEpochSecond(
timestampType.getEpochSecond(value),
timestampType.getNanos(value)))));
return createLongWriteMapping((statement, index, value) -> {
statement.setTimestamp(index, Timestamp.from(Instant.ofEpochSecond(
timestampType.getEpochSecond(value),
timestampType.getNanos(value))), UTC_CALENDAR);
});
}

@Deprecated
public static WriteMapping timestampWriteMappingLegacy(TimestampType timestampType)
{
return createLongWriteMapping((statement, index, value) -> {
statement.setTimestamp(index, Timestamp.from(Instant.ofEpochSecond(
timestampType.getEpochSecond(value),
timestampType.getNanos(value))));
});
}
public static WriteMapping uuidWriteMapping()
{
Expand Down Expand Up @@ -358,7 +377,7 @@ public static Optional<ReadMapping> jdbcTypeToReadMapping(JdbcTypeHandle type)
return Optional.empty();
}

public static Optional<WriteMapping> prestoTypeToWriteMapping(Type type)
public static Optional<WriteMapping> prestoTypeToWriteMapping(ConnectorSession session, Type type)
{
if (type.equals(BOOLEAN)) {
return Optional.of(booleanWriteMapping());
Expand Down Expand Up @@ -394,7 +413,8 @@ else if (type instanceof DateType) {
return Optional.of(dateWriteMapping());
}
else if (type instanceof TimestampType) {
return Optional.of(timestampWriteMapping((TimestampType) type));
boolean legacyTimestamp = session.getSqlFunctionProperties().isLegacyTimestamp();
return Optional.of(legacyTimestamp ? timestampWriteMappingLegacy((TimestampType) type) : timestampWriteMapping((TimestampType) type));
}
else if (type.equals(TIME)) {
return Optional.of(timeWriteMapping());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,9 +37,12 @@
import java.sql.Timestamp;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.ZoneOffset;
import java.util.Calendar;
import java.util.List;
import java.util.Locale;
import java.util.Optional;
import java.util.TimeZone;

import static com.facebook.airlift.testing.Assertions.assertContains;
import static com.facebook.presto.common.type.BigintType.BIGINT;
Expand Down Expand Up @@ -74,6 +77,7 @@
@Test(singleThreaded = true)
public class TestJdbcQueryBuilder
{
private static final Calendar UTC_CALENDAR = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
private TestingDatabase database;
private JdbcClient jdbcClient;
private ConnectorSession session;
Expand Down Expand Up @@ -348,7 +352,7 @@ public void testBuildSqlWithTimestamp()
ResultSet resultSet = preparedStatement.executeQuery()) {
ImmutableSet.Builder<Timestamp> builder = ImmutableSet.builder();
while (resultSet.next()) {
builder.add((Timestamp) resultSet.getObject("col_6"));
builder.add(resultSet.getTimestamp("col_6", UTC_CALENDAR));
}
assertEquals(builder.build(), ImmutableSet.of(
toTimestamp(2016, 6, 3, 0, 23, 37),
Expand Down Expand Up @@ -379,7 +383,7 @@ public void testEmptyBuildSql()

private static Timestamp toTimestamp(int year, int month, int day, int hour, int minute, int second)
{
return Timestamp.valueOf(LocalDateTime.of(year, month, day, hour, minute, second));
return Timestamp.from(LocalDateTime.of(year, month, day, hour, minute, second).toInstant(ZoneOffset.UTC));
}

private static long toDays(int year, int month, int day)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,10 @@
import org.testng.annotations.Test;

import java.math.BigDecimal;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.Statement;
import java.time.LocalDate;
import java.time.ZoneId;

Expand All @@ -53,6 +57,8 @@
import static com.google.common.base.Strings.repeat;
import static com.google.common.base.Verify.verify;
import static java.lang.String.format;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;

@Test
public class TestMySqlTypeMapping
Expand Down Expand Up @@ -254,15 +260,154 @@ public void testDate()
}

@Test
public void testDatetime()
public void testDatetimeUnderlyingStorageVerification()
throws Exception
{
// TODO MySQL datetime is not correctly read (see comment in StandardColumnMappings.timestampReadMapping), but testing this is hard because of #7122
String jdbcUrl = mysqlContainer.getJdbcUrl();
String jdbcUrlWithCredentials = format("%s%suser=%s&password=%s",
jdbcUrl,
jdbcUrl.contains("?") ? "&" : "?",
mysqlContainer.getUsername(),
mysqlContainer.getPassword());
JdbcSqlExecutor jdbcExecutor = new JdbcSqlExecutor(jdbcUrlWithCredentials);

try {
jdbcExecutor.execute("CREATE TABLE tpch.test_datetime_storage (" +
"id INT PRIMARY KEY, " +
"dt DATETIME(6), " +
"source VARCHAR(10))");

// MySQL insertion, MySQL retrieval, and Presto retrieval all agree on wall clock time
jdbcExecutor.execute("INSERT INTO tpch.test_datetime_storage VALUES (1, '1970-01-01 00:00:00.000000', 'jdbc')");

try (Connection conn = DriverManager.getConnection(jdbcUrlWithCredentials);
Statement stmt = conn.createStatement();
ResultSet rs = stmt.executeQuery("SELECT CAST(dt AS CHAR) FROM tpch.test_datetime_storage WHERE id = 1")) {
assertTrue(rs.next(), "Expected one row");
String dbValue1 = rs.getString(1);
assertEquals(dbValue1, "1970-01-01 00:00:00.000000", "JDBC insert should store wall clock time 1970-01-01 00:00:00 in DB");
}

Session session = Session.builder(getQueryRunner().getDefaultSession())
.setSystemProperty("legacy_timestamp", "false")
.build();
assertQuery(session,
"SELECT dt FROM mysql.tpch.test_datetime_storage WHERE id = 1",
"VALUES TIMESTAMP '1970-01-01 00:00:00.000000'");

// Presto insertion, retrieval via MySQL, and retrieval via Presto all agree on wall clock time
assertUpdate(session, "INSERT INTO mysql.tpch.test_datetime_storage VALUES (2, TIMESTAMP '2023-06-15 14:30:00.000000', 'presto')", 1);

try (Connection conn = DriverManager.getConnection(jdbcUrlWithCredentials);
Statement stmt = conn.createStatement();
ResultSet rs = stmt.executeQuery("SELECT CAST(dt AS CHAR) FROM tpch.test_datetime_storage WHERE id = 2")) {
assertTrue(rs.next(), "Expected one row");
String dbValue2 = rs.getString(1);
assertEquals(dbValue2, "2023-06-15 14:30:00.000000", "Presto insert should store wall clock time 2023-06-15 14:30:00 in DB");
}

assertQuery(session,
"SELECT dt FROM mysql.tpch.test_datetime_storage WHERE id = 2",
"VALUES TIMESTAMP '2023-06-15 14:30:00.000000'");

for (String timeZoneId : ImmutableList.of("UTC", "America/New_York", "Asia/Tokyo", "Europe/Warsaw")) {
Session sessionWithTimezone = Session.builder(getQueryRunner().getDefaultSession())
.setTimeZoneKey(TimeZoneKey.getTimeZoneKey(timeZoneId))
.setSystemProperty("legacy_timestamp", "false")
.build();

assertQuery(sessionWithTimezone,
"SELECT dt FROM mysql.tpch.test_datetime_storage WHERE id = 1",
"VALUES TIMESTAMP '1970-01-01 00:00:00.000000'");

assertQuery(sessionWithTimezone,
"SELECT dt FROM mysql.tpch.test_datetime_storage WHERE id = 2",
"VALUES TIMESTAMP '2023-06-15 14:30:00.000000'");
}
}
finally {
jdbcExecutor.execute("DROP TABLE IF EXISTS tpch.test_datetime_storage");
}
}

@Test
public void testTimestamp()
public void testDatetimeLegacyUnderlyingStorageVerification()
throws Exception
{
// TODO MySQL timestamp is not correctly read (see comment in StandardColumnMappings.timestampReadMapping), but testing this is hard because of #7122
String jdbcUrl = mysqlContainer.getJdbcUrl();
String jdbcUrlWithCredentials = format("%s%suser=%s&password=%s",
jdbcUrl,
jdbcUrl.contains("?") ? "&" : "?",
mysqlContainer.getUsername(),
mysqlContainer.getPassword());
JdbcSqlExecutor jdbcExecutor = new JdbcSqlExecutor(jdbcUrlWithCredentials);

try {
jdbcExecutor.execute("CREATE TABLE tpch.test_datetime_legacy_storage (" +
"id INT PRIMARY KEY, " +
"dt DATETIME(6), " +
"source VARCHAR(10))");

// MySQL insertion and MySQL retrieval agree, Presto incorrectly interprets DB value due to legacy mode
jdbcExecutor.execute("INSERT INTO tpch.test_datetime_legacy_storage VALUES (1, '1970-01-01 00:00:00.000000', 'jdbc')");

// Prove that the value is 1970-01-01 00:00:00 by reading directly from the DB via JDBC
try (Connection conn = DriverManager.getConnection(jdbcUrlWithCredentials);
Statement stmt = conn.createStatement();
ResultSet rs = stmt.executeQuery("SELECT CAST(dt AS CHAR) FROM tpch.test_datetime_legacy_storage WHERE id = 1")) {
assertTrue(rs.next(), "Expected one row");
String dbValue1 = rs.getString(1);
assertEquals(dbValue1, "1970-01-01 00:00:00.000000", "JDBC insert should store wall clock time 1970-01-01 00:00:00 in DB");
}

// In legacy mode, DB value 1970-01-01 00:00:00 is interpreted as if it's in JVM timezone (America/Bahia_Banderas UTC-7)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What happens if the user session and server are in different zone . what happens if we set session timezone key as "Asia/kolkata" for same scenario?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good . Thank you !

// and then converted to the session timezone. Since both are the same (America/Bahia_Banderas),
// the offset comes from treating the wall-clock DB time as UTC, resulting in 1969-12-31 20:00:00
Session legacySession = Session.builder(getQueryRunner().getDefaultSession())
.setSystemProperty("legacy_timestamp", "true")
.build();
assertQuery(legacySession,
"SELECT dt FROM mysql.tpch.test_datetime_legacy_storage WHERE id = 1",
"VALUES TIMESTAMP '1969-12-31 20:00:00.000000'");

// Presto insertion with legacy mode, verify DB storage via JDBC (should apply JVM timezone conversion during write)
assertUpdate(legacySession, "INSERT INTO mysql.tpch.test_datetime_legacy_storage VALUES (2, TIMESTAMP '2023-06-15 14:30:00.000000', 'presto')", 1);

try (Connection conn = DriverManager.getConnection(jdbcUrlWithCredentials);
Statement stmt = conn.createStatement();
ResultSet rs = stmt.executeQuery("SELECT CAST(dt AS CHAR) FROM tpch.test_datetime_legacy_storage WHERE id = 2")) {
assertTrue(rs.next(), "Expected one row");
String dbValue2 = rs.getString(1);
// JVM timezone is America/Bahia_Banderas (UTC-7), so 2023-06-15 14:30:00 becomes 2023-06-14 19:30:00.000000
assertEquals(dbValue2, "2023-06-14 19:30:00.000000", "Legacy mode applies timezone conversion during write, expected 2023-06-14 19:30:00.000000");
}

// Verify Presto reads it back correctly in legacy mode (round-trip should work)
assertQuery(legacySession,
"SELECT dt FROM mysql.tpch.test_datetime_legacy_storage WHERE id = 2",
"VALUES TIMESTAMP '2023-06-15 14:30:00.000000'");

// DB value 1970-01-01 00:00:00 is interpreted as JVM timezone (America/Bahia_Banderas UTC-7),
// then converted to the session timezone
Session legacyUtcSession = Session.builder(getQueryRunner().getDefaultSession())
.setTimeZoneKey(TimeZoneKey.getTimeZoneKey("UTC"))
.setSystemProperty("legacy_timestamp", "true")
.build();
assertQuery(legacyUtcSession,
"SELECT dt FROM mysql.tpch.test_datetime_legacy_storage WHERE id = 1",
"VALUES TIMESTAMP '1970-01-01 07:00:00.000000'");

Session legacyTokyoSession = Session.builder(getQueryRunner().getDefaultSession())
.setTimeZoneKey(TimeZoneKey.getTimeZoneKey("Asia/Tokyo"))
.setSystemProperty("legacy_timestamp", "true")
.build();
assertQuery(legacyTokyoSession,
"SELECT dt FROM mysql.tpch.test_datetime_legacy_storage WHERE id = 1",
"VALUES TIMESTAMP '1970-01-01 16:00:00.000000'");
}
finally {
jdbcExecutor.execute("DROP TABLE IF EXISTS tpch.test_datetime_legacy_storage");
}
}

private void testUnsupportedDataType(String databaseDataType)
Expand Down
Loading
Loading