Skip to content

Commit

Permalink
Support microsecond timestamp precision in MemSQL
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed Oct 26, 2021
1 parent 8622e91 commit b88cc93
Show file tree
Hide file tree
Showing 2 changed files with 263 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import io.trino.spi.type.CharType;
import io.trino.spi.type.Decimals;
import io.trino.spi.type.StandardTypes;
import io.trino.spi.type.TimestampType;
import io.trino.spi.type.Type;
import io.trino.spi.type.TypeManager;
import io.trino.spi.type.TypeSignature;
Expand All @@ -53,11 +54,13 @@

import java.sql.Connection;
import java.sql.DatabaseMetaData;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Types;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
Expand Down Expand Up @@ -88,7 +91,7 @@
import static io.trino.plugin.jdbc.StandardColumnMappings.realWriteFunction;
import static io.trino.plugin.jdbc.StandardColumnMappings.smallintColumnMapping;
import static io.trino.plugin.jdbc.StandardColumnMappings.timeColumnMappingUsingSqlTime;
import static io.trino.plugin.jdbc.StandardColumnMappings.timestampColumnMappingUsingSqlTimestampWithRounding;
import static io.trino.plugin.jdbc.StandardColumnMappings.timestampColumnMapping;
import static io.trino.plugin.jdbc.StandardColumnMappings.timestampWriteFunction;
import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintColumnMapping;
import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryColumnMapping;
Expand All @@ -100,7 +103,8 @@
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static io.trino.spi.type.DecimalType.createDecimalType;
import static io.trino.spi.type.RealType.REAL;
import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS;
import static io.trino.spi.type.TimestampType.TIMESTAMP_MICROS;
import static io.trino.spi.type.TimestampType.createTimestampType;
import static io.trino.spi.type.VarbinaryType.VARBINARY;
import static java.lang.Math.max;
import static java.lang.Math.min;
Expand All @@ -114,6 +118,7 @@ public class MemSqlClient
{
private static final Logger log = Logger.get(MemSqlClient.class);

static final int MEMSQL_DATE_TIME_MAX_PRECISION = 6;
static final int MEMSQL_VARCHAR_MAX_LENGTH = 21844;
static final int MEMSQL_TEXT_MAX_LENGTH = 65535;
static final int MEMSQL_MEDIUMTEXT_MAX_LENGTH = 16777215;
Expand Down Expand Up @@ -176,6 +181,7 @@ public List<JdbcColumnHandle> getColumns(ConnectorSession session, JdbcTableHand

try (Connection connection = connectionFactory.openConnection(session);
ResultSet resultSet = getColumns(tableHandle, connection.getMetaData())) {
Map<String, Integer> timestampPrecisions = getTimestampPrecisions(connection, tableHandle);
int allColumns = 0;
List<JdbcColumnHandle> columns = new ArrayList<>();
while (resultSet.next()) {
Expand All @@ -185,18 +191,22 @@ public List<JdbcColumnHandle> getColumns(ConnectorSession session, JdbcTableHand
}
allColumns++;
String columnName = resultSet.getString("COLUMN_NAME");
Optional<Integer> decimalDigits = getInteger(resultSet, "DECIMAL_DIGITS");
if (timestampPrecisions.containsKey(columnName)) {
decimalDigits = Optional.of(timestampPrecisions.get(columnName));
}

JdbcTypeHandle typeHandle = new JdbcTypeHandle(
getInteger(resultSet, "DATA_TYPE").orElseThrow(() -> new IllegalStateException("DATA_TYPE is null")),
Optional.ofNullable(resultSet.getString("TYPE_NAME")),
getInteger(resultSet, "COLUMN_SIZE"),
getInteger(resultSet, "DECIMAL_DIGITS"),
decimalDigits,
Optional.empty(),
Optional.empty());
Optional<ColumnMapping> columnMapping = toColumnMapping(session, connection, typeHandle);
log.debug("Mapping data type of '%s' column '%s': %s mapped to %s", schemaTableName, columnName, typeHandle, columnMapping);
// skip unsupported column types
boolean nullable = (resultSet.getInt("NULLABLE") != columnNoNulls);
// Note: some databases (e.g. SQL Server) do not return column remarks/comment here.
Optional<String> comment = Optional.ofNullable(emptyToNull(resultSet.getString("REMARKS")));
if (columnMapping.isPresent()) {
columns.add(JdbcColumnHandle.builder()
Expand All @@ -217,7 +227,6 @@ public List<JdbcColumnHandle> getColumns(ConnectorSession session, JdbcTableHand
}
}
if (columns.isEmpty()) {
// A table may have no supported columns. In rare cases (e.g. PostgreSQL) a table might have no columns at all.
throw new TableNotFoundException(
schemaTableName,
format("Table '%s' has no supported columns (all %s columns are not supported)", schemaTableName, allColumns));
Expand All @@ -238,6 +247,32 @@ private static RemoteTableName getRemoteTable(ResultSet resultSet)
resultSet.getString("TABLE_NAME"));
}

private static Map<String, Integer> getTimestampPrecisions(Connection connection, JdbcTableHandle tableHandle)
throws SQLException
{
// MariaDB JDBC driver doesn't expose timestamp precision when connecting to MemSQL cluster
String sql = "" +
"SELECT column_name, column_type " +
"FROM information_schema.columns " +
"WHERE table_schema = ? " +
"AND table_name = ? " +
"AND column_type IN ('datetime', 'datetime(6)', 'timestamp', 'timestamp(6)')";
try (PreparedStatement statement = connection.prepareStatement(sql)) {
statement.setString(1, tableHandle.getCatalogName());
statement.setString(2, tableHandle.getTableName());

Map<String, Integer> timestampColumnPrecisions = new HashMap<>();
try (ResultSet resultSet = statement.executeQuery()) {
while (resultSet.next()) {
String columnType = resultSet.getString("column_type");
int size = columnType.equals("datetime") || columnType.equals("timestamp") ? 0 : 6;
timestampColumnPrecisions.put(resultSet.getString("column_name"), size);
}
}
return timestampColumnPrecisions;
}
}

@Override
public Optional<ColumnMapping> toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle)
{
Expand Down Expand Up @@ -300,8 +335,9 @@ public Optional<ColumnMapping> toColumnMapping(ConnectorSession session, Connect
// TODO (https://github.com/trinodb/trino/issues/5450) Fix TIME type mapping
return Optional.of(timeColumnMappingUsingSqlTime());
case Types.TIMESTAMP:
// TODO (https://github.com/trinodb/trino/issues/5450) Fix Timestamp type mapping
return Optional.of(timestampColumnMappingUsingSqlTimestampWithRounding(TIMESTAMP_MILLIS));
// TODO (https://github.com/trinodb/trino/issues/5450) Fix DST handling
TimestampType timestampType = createTimestampType(typeHandle.getRequiredDecimalDigits());
return Optional.of(timestampColumnMapping(timestampType));
}

if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) {
Expand Down Expand Up @@ -425,9 +461,13 @@ else if (varcharType.getBoundedLength() <= MEMSQL_MEDIUMTEXT_MAX_LENGTH) {
return WriteMapping.longMapping("float", realWriteFunction());
}
// TODO implement TIME type
// TODO add support for other TIMESTAMP precisions
if (TIMESTAMP_MILLIS.equals(type)) {
return WriteMapping.longMapping("datetime", timestampWriteFunction(TIMESTAMP_MILLIS));
if (type instanceof TimestampType) {
TimestampType timestampType = (TimestampType) type;
checkArgument(timestampType.getPrecision() <= MEMSQL_DATE_TIME_MAX_PRECISION, "The max timestamp precision in MemSQL is 6");
if (timestampType.getPrecision() == 0) {
return WriteMapping.longMapping("datetime", timestampWriteFunction(timestampType));
}
return WriteMapping.longMapping(format("datetime(%s)", MEMSQL_DATE_TIME_MAX_PRECISION), timestampWriteFunction(TIMESTAMP_MICROS));
}

// TODO add explicit mappings
Expand Down
Loading

0 comments on commit b88cc93

Please sign in to comment.