Skip to content

Commit

Permalink
Map MySQL TIMESTAMP types to TIMESTAMP WITH TIME ZONE
Browse files Browse the repository at this point in the history
Before this change MySQL TIMESTAMP type was mapped to Trino TIMESTAMP
type. However MySQL TIMESTAMP type has point-in-time semantics while
Trino TIMESTAMP does not.

Until we have an INSTANT type in Trino the closest matching type is
TIMESTAMP WITH TIME ZONE so this commit changes the MySQL type mappings
for MySQL's TIMESTAMP type to Trino's TIMESTAMP WITH TIME ZONE type.

Note that MySQL DATETIME types are read as TIMESTAMP types and Trino
TIMESTAMP types are inserted into MySQL as DATETIME types -- this
behavior is unchanged.

In other words, for reads and writes:

    MySQL Type    Trino Type
    DATETIME      TIMESTAMP
    TIMESTAMP     TIMESTAMP WITH TIME ZONE
  • Loading branch information
adamjshook authored Sep 13, 2023
1 parent cc82dec commit fd5b247
Show file tree
Hide file tree
Showing 7 changed files with 1,054 additions and 46 deletions.
7 changes: 5 additions & 2 deletions docs/src/main/sphinx/connector/mysql.md
Original file line number Diff line number Diff line change
Expand Up @@ -205,10 +205,10 @@ this table:
- ``TIME(n)``
-
* - ``DATETIME(n)``
- ``DATETIME(n)``
- ``TIMESTAMP(n)``
-
* - ``TIMESTAMP(n)``
- ``TIMESTAMP(n)``
- ``TIMESTAMP(n) WITH TIME ZONE``
-
```

Expand Down Expand Up @@ -267,6 +267,9 @@ this table:
- ``TIME(n)``
-
* - ``TIMESTAMP(n)``
- ``DATETIME(n)``
-
* - ``TIMESTAMP(n) WITH TIME ZONE``
- ``TIMESTAMP(n)``
-
```
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@
import io.trino.plugin.jdbc.JdbcTypeHandle;
import io.trino.plugin.jdbc.LongReadFunction;
import io.trino.plugin.jdbc.LongWriteFunction;
import io.trino.plugin.jdbc.ObjectReadFunction;
import io.trino.plugin.jdbc.ObjectWriteFunction;
import io.trino.plugin.jdbc.PredicatePushdownController;
import io.trino.plugin.jdbc.PreparedQuery;
import io.trino.plugin.jdbc.QueryBuilder;
Expand All @@ -63,6 +65,7 @@
import io.trino.spi.TrinoException;
import io.trino.spi.connector.AggregateFunction;
import io.trino.spi.connector.ColumnHandle;
import io.trino.spi.connector.ColumnMetadata;
import io.trino.spi.connector.ConnectorSession;
import io.trino.spi.connector.ConnectorTableMetadata;
import io.trino.spi.connector.JoinCondition;
Expand All @@ -79,9 +82,11 @@
import io.trino.spi.type.CharType;
import io.trino.spi.type.DecimalType;
import io.trino.spi.type.Decimals;
import io.trino.spi.type.LongTimestampWithTimeZone;
import io.trino.spi.type.StandardTypes;
import io.trino.spi.type.TimeType;
import io.trino.spi.type.TimestampType;
import io.trino.spi.type.TimestampWithTimeZoneType;
import io.trino.spi.type.Type;
import io.trino.spi.type.TypeManager;
import io.trino.spi.type.TypeSignature;
Expand All @@ -97,9 +102,12 @@
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
import java.sql.SQLSyntaxErrorException;
import java.sql.Timestamp;
import java.sql.Types;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.OffsetDateTime;
import java.util.AbstractMap.SimpleEntry;
import java.util.Collection;
import java.util.List;
Expand Down Expand Up @@ -172,19 +180,28 @@
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.BooleanType.BOOLEAN;
import static io.trino.spi.type.CharType.createCharType;
import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone;
import static io.trino.spi.type.DateTimeEncoding.unpackMillisUtc;
import static io.trino.spi.type.DateType.DATE;
import static io.trino.spi.type.DecimalType.createDecimalType;
import static io.trino.spi.type.DoubleType.DOUBLE;
import static io.trino.spi.type.IntegerType.INTEGER;
import static io.trino.spi.type.RealType.REAL;
import static io.trino.spi.type.SmallintType.SMALLINT;
import static io.trino.spi.type.TimeType.createTimeType;
import static io.trino.spi.type.TimeZoneKey.UTC_KEY;
import static io.trino.spi.type.TimestampType.createTimestampType;
import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType;
import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_SECOND;
import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MILLISECOND;
import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_NANOSECOND;
import static io.trino.spi.type.TinyintType.TINYINT;
import static io.trino.spi.type.VarbinaryType.VARBINARY;
import static io.trino.spi.type.VarcharType.createUnboundedVarcharType;
import static io.trino.spi.type.VarcharType.createVarcharType;
import static java.lang.Float.floatToRawIntBits;
import static java.lang.Math.floorDiv;
import static java.lang.Math.floorMod;
import static java.lang.Math.max;
import static java.lang.Math.min;
import static java.lang.String.format;
Expand All @@ -208,6 +225,10 @@ public class MySqlClient
// MySQL driver returns width of time types instead of precision, same as the above timestamp type.
private static final int ZERO_PRECISION_TIME_COLUMN_SIZE = 8;

// MySQL TIMESTAMP has a range of '1970-01-01 00:00:01' UTC to '2038-01-19 03:14:07' UTC
private static final Instant MYSQL_TIMESTAMP_MIN_VALUE = Instant.parse("1970-01-01T00:00:01.000000Z");
private static final Instant MYSQL_TIMESTAMP_MAX_VALUE = Instant.parse("2038-01-19T03:14:07.499999Z");

// An empty character means that the table doesn't have a comment in MySQL
private static final String NO_COMMENT = "";

Expand Down Expand Up @@ -439,6 +460,21 @@ protected String createTableSql(RemoteTableName remoteTableName, List<String> co
return format("CREATE TABLE %s (%s) COMMENT %s", quoted(remoteTableName), join(", ", columns), mysqlVarcharLiteral(tableMetadata.getComment().orElse(NO_COMMENT)));
}

// This is overridden to pass NULL to MySQL for TIMESTAMP column types
// Without it, an "Invalid default value" error is thrown
@Override
protected String getColumnDefinitionSql(ConnectorSession session, ColumnMetadata column, String columnName)
{
if (column.getComment() != null) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with column comment");
}

return "%s %s %s".formatted(
quoted(columnName),
toWriteMapping(session, column.getType()).getDataType(),
column.isNullable() ? "NULL" : "NOT NULL");
}

private static String mysqlVarcharLiteral(String value)
{
requireNonNull(value, "value is null");
Expand Down Expand Up @@ -469,6 +505,8 @@ public Optional<ColumnMapping> toColumnMapping(ConnectorSession session, Connect
return Optional.of(jsonColumnMapping());
case "enum":
return Optional.of(defaultVarcharColumnMapping(typeHandle.getRequiredColumnSize(), false));
case "datetime":
return mysqlDateTimeToTrinoTimestamp(typeHandle);
}

switch (typeHandle.getJdbcType()) {
Expand Down Expand Up @@ -545,12 +583,7 @@ public Optional<ColumnMapping> toColumnMapping(ConnectorSession session, Connect
timeWriteFunction(timeType.getPrecision())));

case Types.TIMESTAMP:
TimestampType timestampType = createTimestampType(getTimestampPrecision(typeHandle.getRequiredColumnSize()));
checkArgument(timestampType.getPrecision() <= TimestampType.MAX_SHORT_PRECISION, "Precision is out of range: %s", timestampType.getPrecision());
return Optional.of(ColumnMapping.longMapping(
timestampType,
mySqlTimestampReadFunction(timestampType),
timestampWriteFunction(timestampType)));
return mysqlTimestampToTrinoTimestampWithTz(typeHandle);
}

if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) {
Expand Down Expand Up @@ -592,6 +625,73 @@ private static ColumnMapping mySqlCharColumnMapping(CharType charType, Optional<
return ColumnMapping.sliceMapping(charType, charReadFunction(charType), charWriteFunction(), pushdownController);
}

private Optional<ColumnMapping> mysqlDateTimeToTrinoTimestamp(JdbcTypeHandle typeHandle)
{
TimestampType timestampType = createTimestampType(getTimestampPrecision(typeHandle.getRequiredColumnSize()));
checkArgument(timestampType.getPrecision() <= TimestampType.MAX_SHORT_PRECISION, "Precision is out of range: %s", timestampType.getPrecision());
return Optional.of(ColumnMapping.longMapping(
timestampType,
mySqlTimestampReadFunction(timestampType),
timestampWriteFunction(timestampType)));
}

private static Optional<ColumnMapping> mysqlTimestampToTrinoTimestampWithTz(JdbcTypeHandle typeHandle)
{
TimestampWithTimeZoneType trinoType = createTimestampWithTimeZoneType(getTimestampPrecision(typeHandle.getRequiredColumnSize()));
if (trinoType.getPrecision() <= TimestampWithTimeZoneType.MAX_SHORT_PRECISION) {
return Optional.of(ColumnMapping.longMapping(
trinoType,
shortTimestampWithTimeZoneReadFunction(),
shortTimestampWithTimeZoneWriteFunction()));
}
return Optional.of(ColumnMapping.objectMapping(
trinoType,
longTimestampWithTimeZoneReadFunction(),
longTimestampWithTimeZoneWriteFunction()));
}

private static LongReadFunction shortTimestampWithTimeZoneReadFunction()
{
return (resultSet, columnIndex) -> {
Timestamp timestamp = resultSet.getTimestamp(columnIndex);
long millisUtc = timestamp.getTime();
return packDateTimeWithZone(millisUtc, UTC_KEY);
};
}

private static ObjectReadFunction longTimestampWithTimeZoneReadFunction()
{
return ObjectReadFunction.of(
LongTimestampWithTimeZone.class,
(resultSet, columnIndex) -> {
OffsetDateTime offsetDateTime = resultSet.getObject(columnIndex, OffsetDateTime.class);
return LongTimestampWithTimeZone.fromEpochSecondsAndFraction(
offsetDateTime.toEpochSecond(),
(long) offsetDateTime.getNano() * PICOSECONDS_PER_NANOSECOND,
UTC_KEY);
});
}

private static LongWriteFunction shortTimestampWithTimeZoneWriteFunction()
{
return (statement, index, value) -> {
Instant instantValue = Instant.ofEpochMilli(unpackMillisUtc(value));
statement.setObject(index, instantValue);
};
}

private static ObjectWriteFunction longTimestampWithTimeZoneWriteFunction()
{
return ObjectWriteFunction.of(
LongTimestampWithTimeZone.class,
(statement, index, value) -> {
long epochSeconds = floorDiv(value.getEpochMillis(), MILLISECONDS_PER_SECOND);
long nanosOfSecond = (long) floorMod(value.getEpochMillis(), MILLISECONDS_PER_SECOND) * NANOSECONDS_PER_MILLISECOND + value.getPicosOfMilli() / PICOSECONDS_PER_NANOSECOND;
Instant instantValue = Instant.ofEpochSecond(epochSeconds, nanosOfSecond);
statement.setObject(index, instantValue);
});
}

private LongWriteFunction mySqlDateWriteFunctionUsingLocalDate()
{
return new LongWriteFunction() {
Expand Down Expand Up @@ -730,6 +830,17 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type)
return WriteMapping.objectMapping(format("datetime(%s)", MAX_SUPPORTED_DATE_TIME_PRECISION), longTimestampWriteFunction(timestampType, MAX_SUPPORTED_DATE_TIME_PRECISION));
}

if (type instanceof TimestampWithTimeZoneType timestampWithTimeZoneType) {
if (timestampWithTimeZoneType.getPrecision() <= MAX_SUPPORTED_DATE_TIME_PRECISION) {
String dataType = format("timestamp(%d)", timestampWithTimeZoneType.getPrecision());
if (timestampWithTimeZoneType.getPrecision() <= TimestampWithTimeZoneType.MAX_SHORT_PRECISION) {
return WriteMapping.longMapping(dataType, shortTimestampWithTimeZoneWriteFunction());
}
return WriteMapping.objectMapping(dataType, longTimestampWithTimeZoneWriteFunction());
}
return WriteMapping.objectMapping(format("timestamp(%d)", MAX_SUPPORTED_DATE_TIME_PRECISION), longTimestampWithTimeZoneWriteFunction());
}

if (VARBINARY.equals(type)) {
return WriteMapping.sliceMapping("mediumblob", varbinaryWriteFunction());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,11 +76,11 @@ public static Properties getConnectionProperties(MySqlConfig mySqlConfig)
connectionProperties.setProperty("tinyInt1isBit", "false");
connectionProperties.setProperty("rewriteBatchedStatements", "true");

// Try to make MySQL timestamps work (See https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-time-instants.html)
// without relying on server time zone (which may be configured to be totally unusable).
// TODO (https://github.com/trinodb/trino/issues/15668) rethink how timestamps are mapped. Also, probably worth adding tests
// with MySQL server with a non-UTC system zone.
connectionProperties.setProperty("connectionTimeZone", "UTC");
// connectionTimeZone = LOCAL means the JDBC driver uses the JVM zone as the session zone
// forceConnectionTimeZoneToSession = true means that the server side connection zone is changed to match local JVM zone
// https://dev.mysql.com/doc/connector-j/8.1/en/connector-j-time-instants.html (Solution 2b)
connectionProperties.setProperty("connectionTimeZone", "LOCAL");
connectionProperties.setProperty("forceConnectionTimeZoneToSession", "true");

if (mySqlConfig.isAutoReconnect()) {
connectionProperties.setProperty("autoReconnect", String.valueOf(mySqlConfig.isAutoReconnect()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,9 +119,19 @@ protected boolean isColumnNameRejected(Exception exception, String columnName, b
protected Optional<DataMappingTestSetup> filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup)
{
String typeName = dataMappingTestSetup.getTrinoTypeName();
if (typeName.equals("timestamp(3) with time zone") ||
typeName.equals("timestamp(6) with time zone")) {
return Optional.of(dataMappingTestSetup.asUnsupported());

// MySQL TIMESTAMP has a range of '1970-01-01 00:00:01' UTC to '2038-01-19 03:14:07' UTC.
if (typeName.equals("timestamp(3) with time zone")) {
if (dataMappingTestSetup.getSampleValueLiteral().contains("1969")) {
return Optional.of(new DataMappingTestSetup("timestamp(3) with time zone", "TIMESTAMP '1970-01-01 15:03:00.123 +01:00'", "TIMESTAMP '1970-01-31 17:03:00.456 +01:00'"));
}
return Optional.of(new DataMappingTestSetup("timestamp(3) with time zone", "TIMESTAMP '2020-02-12 15:03:00 +01:00'", "TIMESTAMP '2038-01-19 03:14:07.000 UTC'"));
}
else if (typeName.equals("timestamp(6) with time zone")) {
if (dataMappingTestSetup.getSampleValueLiteral().contains("1969")) {
return Optional.of(new DataMappingTestSetup("timestamp(6) with time zone", "TIMESTAMP '1970-01-01 15:03:00.123456 +01:00'", "TIMESTAMP '1970-01-31 17:03:00.123456 +01:00'"));
}
return Optional.of(new DataMappingTestSetup("timestamp(6) with time zone", "TIMESTAMP '2020-02-12 15:03:00 +01:00'", "TIMESTAMP '2038-01-19 03:14:07.000 UTC'"));
}

if (typeName.equals("timestamp")) {
Expand Down
Loading

0 comments on commit fd5b247

Please sign in to comment.