Skip to content

Commit

Permalink
Make JdbcTypeHandle.decimalDigits optional
Browse files Browse the repository at this point in the history
The `DECIMAL_DIGITS` field is nullable and the distinction is sometimes
important.
  • Loading branch information
findepi committed Sep 22, 2020
1 parent b23d261 commit f6bf01b
Show file tree
Hide file tree
Showing 12 changed files with 61 additions and 35 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,7 @@ public List<JdbcColumnHandle> getColumns(ConnectorSession session, JdbcTableHand
resultSet.getInt("DATA_TYPE"),
Optional.ofNullable(resultSet.getString("TYPE_NAME")),
resultSet.getInt("COLUMN_SIZE"),
resultSet.getInt("DECIMAL_DIGITS"),
getInteger(resultSet, "DECIMAL_DIGITS"),
Optional.empty(),
Optional.empty());
Optional<ColumnMapping> columnMapping = toPrestoType(session, connection, typeHandle);
Expand Down Expand Up @@ -315,6 +315,16 @@ public List<JdbcColumnHandle> getColumns(ConnectorSession session, JdbcTableHand
}
}

protected static Optional<Integer> getInteger(ResultSet resultSet, String columnLabel)
throws SQLException
{
int value = resultSet.getInt(columnLabel);
if (resultSet.wasNull()) {
return Optional.empty();
}
return Optional.of(value);
}

protected ResultSet getColumns(JdbcTableHandle tableHandle, DatabaseMetaData metadata)
throws SQLException
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ public final class JdbcTypeHandle
private final int jdbcType;
private final Optional<String> jdbcTypeName;
private final int columnSize;
private final int decimalDigits;
private final Optional<Integer> decimalDigits;
private final Optional<Integer> arrayDimensions;
private final Optional<CaseSensitivity> caseSensitivity;

Expand All @@ -37,19 +37,31 @@ public JdbcTypeHandle(int jdbcType, Optional<String> jdbcTypeName, int columnSiz
this(jdbcType, jdbcTypeName, columnSize, decimalDigits, arrayDimensions, Optional.empty());
}

@JsonCreator
@Deprecated
public JdbcTypeHandle(
@JsonProperty("jdbcType") int jdbcType,
@JsonProperty("jdbcTypeName") Optional<String> jdbcTypeName,
@JsonProperty("columnSize") int columnSize,
@JsonProperty("decimalDigits") int decimalDigits,
@JsonProperty("arrayDimensions") Optional<Integer> arrayDimensions,
@JsonProperty("caseSensitivity") Optional<CaseSensitivity> caseSensitivity)
{
this(jdbcType, jdbcTypeName, columnSize, Optional.of(decimalDigits), arrayDimensions, caseSensitivity);
}

@JsonCreator
public JdbcTypeHandle(
@JsonProperty("jdbcType") int jdbcType,
@JsonProperty("jdbcTypeName") Optional<String> jdbcTypeName,
@JsonProperty("columnSize") int columnSize,
@JsonProperty("decimalDigits") Optional<Integer> decimalDigits,
@JsonProperty("arrayDimensions") Optional<Integer> arrayDimensions,
@JsonProperty("caseSensitivity") Optional<CaseSensitivity> caseSensitivity)
{
this.jdbcType = jdbcType;
this.jdbcTypeName = requireNonNull(jdbcTypeName, "jdbcTypeName is null");
this.columnSize = columnSize;
this.decimalDigits = decimalDigits;
this.decimalDigits = requireNonNull(decimalDigits, "decimalDigits is null");
this.arrayDimensions = requireNonNull(arrayDimensions, "arrayDimensions is null");
this.caseSensitivity = requireNonNull(caseSensitivity, "caseSensitivity is null");
}
Expand All @@ -73,7 +85,7 @@ public int getColumnSize()
}

@JsonProperty
public int getDecimalDigits()
public Optional<Integer> getDecimalDigits()
{
return decimalDigits;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -494,7 +494,7 @@ public static Optional<ColumnMapping> jdbcTypeToPrestoType(JdbcTypeHandle type)

case Types.NUMERIC:
case Types.DECIMAL:
int decimalDigits = type.getDecimalDigits();
int decimalDigits = type.getDecimalDigits().orElseThrow(() -> new IllegalStateException("decimal digits not present"));
int precision = columnSize + max(-decimalDigits, 0); // Map decimal(p, -s) (negative scale) to decimal(p+s, 0).
if (precision > Decimals.MAX_PRECISION) {
return Optional.empty();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
class TestingH2JdbcClient
extends BaseJdbcClient
{
private static final JdbcTypeHandle BIGINT_TYPE_HANDLE = new JdbcTypeHandle(Types.BIGINT, Optional.empty(), -1, -1, Optional.empty(), Optional.empty());
private static final JdbcTypeHandle BIGINT_TYPE_HANDLE = new JdbcTypeHandle(Types.BIGINT, Optional.empty(), -1, Optional.empty(), Optional.empty(), Optional.empty());

public TestingH2JdbcClient(BaseJdbcConfig config, ConnectionFactory connectionFactory)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,20 +20,20 @@ public final class TestingJdbcTypeHandle
{
private TestingJdbcTypeHandle() {}

public static final JdbcTypeHandle JDBC_BOOLEAN = new JdbcTypeHandle(Types.BOOLEAN, Optional.of("boolean"), 1, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_BOOLEAN = new JdbcTypeHandle(Types.BOOLEAN, Optional.of("boolean"), 1, Optional.empty(), Optional.empty(), Optional.empty());

public static final JdbcTypeHandle JDBC_SMALLINT = new JdbcTypeHandle(Types.SMALLINT, Optional.of("smallint"), 1, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_TINYINT = new JdbcTypeHandle(Types.TINYINT, Optional.of("tinyint"), 2, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_INTEGER = new JdbcTypeHandle(Types.INTEGER, Optional.of("integer"), 4, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_BIGINT = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), 8, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_SMALLINT = new JdbcTypeHandle(Types.SMALLINT, Optional.of("smallint"), 1, Optional.empty(), Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_TINYINT = new JdbcTypeHandle(Types.TINYINT, Optional.of("tinyint"), 2, Optional.empty(), Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_INTEGER = new JdbcTypeHandle(Types.INTEGER, Optional.of("integer"), 4, Optional.empty(), Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_BIGINT = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), 8, Optional.empty(), Optional.empty(), Optional.empty());

public static final JdbcTypeHandle JDBC_REAL = new JdbcTypeHandle(Types.REAL, Optional.of("real"), 8, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_DOUBLE = new JdbcTypeHandle(Types.DOUBLE, Optional.of("double precision"), 8, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_REAL = new JdbcTypeHandle(Types.REAL, Optional.of("real"), 8, Optional.empty(), Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_DOUBLE = new JdbcTypeHandle(Types.DOUBLE, Optional.of("double precision"), 8, Optional.empty(), Optional.empty(), Optional.empty());

public static final JdbcTypeHandle JDBC_CHAR = new JdbcTypeHandle(Types.CHAR, Optional.of("char"), 10, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_VARCHAR = new JdbcTypeHandle(Types.VARCHAR, Optional.of("varchar"), 10, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_CHAR = new JdbcTypeHandle(Types.CHAR, Optional.of("char"), 10, Optional.empty(), Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_VARCHAR = new JdbcTypeHandle(Types.VARCHAR, Optional.of("varchar"), 10, Optional.empty(), Optional.empty(), Optional.empty());

public static final JdbcTypeHandle JDBC_DATE = new JdbcTypeHandle(Types.DATE, Optional.of("date"), 8, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_TIME = new JdbcTypeHandle(Types.TIME, Optional.of("time"), 4, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_TIMESTAMP = new JdbcTypeHandle(Types.TIMESTAMP, Optional.of("timestamp"), 8, 0, Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_DATE = new JdbcTypeHandle(Types.DATE, Optional.of("date"), 8, Optional.empty(), Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_TIME = new JdbcTypeHandle(Types.TIME, Optional.of("time"), 4, Optional.empty(), Optional.empty(), Optional.empty());
public static final JdbcTypeHandle JDBC_TIMESTAMP = new JdbcTypeHandle(Types.TIMESTAMP, Optional.of("timestamp"), 8, Optional.empty(), Optional.empty(), Optional.empty());
}
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,6 @@ public Optional<JdbcExpression> rewrite(AggregateFunction aggregateFunction, Cap

return Optional.of(new JdbcExpression(
format("avg((%s * 1.0))", columnHandle.toSqlExpression(context.getIdentifierQuote())),
new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), 0, 0, Optional.empty(), Optional.empty())));
new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), 0, Optional.empty(), Optional.empty(), Optional.empty())));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ public MySqlClient(BaseJdbcConfig config, ConnectionFactory connectionFactory, T
super(config, "`", connectionFactory);
this.jsonType = typeManager.getType(new TypeSignature(StandardTypes.JSON));

JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), 0, 0, Optional.empty(), Optional.empty());
JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), 0, Optional.empty(), Optional.empty(), Optional.empty());
this.aggregateFunctionRewriter = new AggregateFunctionRewriter(
this::quoted,
ImmutableSet.<AggregateFunctionRule>builder()
Expand All @@ -135,7 +135,7 @@ public Optional<JdbcExpression> implementAggregation(ConnectorSession session, A

private static Optional<JdbcTypeHandle> toTypeHandle(DecimalType decimalType)
{
return Optional.of(new JdbcTypeHandle(Types.NUMERIC, Optional.of("decimal"), decimalType.getPrecision(), decimalType.getScale(), Optional.empty(), Optional.empty()));
return Optional.of(new JdbcTypeHandle(Types.NUMERIC, Optional.of("decimal"), decimalType.getPrecision(), Optional.of(decimalType.getScale()), Optional.empty(), Optional.empty()));
}

@Override
Expand Down Expand Up @@ -241,8 +241,9 @@ public Optional<ColumnMapping> toPrestoType(ConnectorSession session, Connection

case Types.DECIMAL:
int precision = columnSize;
int decimalDigits = typeHandle.getDecimalDigits().orElseThrow(() -> new IllegalStateException("decimal digits not present"));
if (getDecimalRounding(session) == ALLOW_OVERFLOW && precision > Decimals.MAX_PRECISION) {
int scale = min(typeHandle.getDecimalDigits(), getDecimalDefaultScale(session));
int scale = min(decimalDigits, getDecimalDefaultScale(session));
return Optional.of(decimalColumnMapping(createDecimalType(Decimals.MAX_PRECISION, scale), getDecimalRoundingMode(session)));
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,14 +49,14 @@ public class TestMySqlClient
JdbcColumnHandle.builder()
.setColumnName("c_bigint")
.setColumnType(BIGINT)
.setJdbcTypeHandle(new JdbcTypeHandle(Types.BIGINT, Optional.of("int8"), 0, 0, Optional.empty(), Optional.empty()))
.setJdbcTypeHandle(new JdbcTypeHandle(Types.BIGINT, Optional.of("int8"), 0, Optional.empty(), Optional.empty(), Optional.empty()))
.build();

private static final JdbcColumnHandle DOUBLE_COLUMN =
JdbcColumnHandle.builder()
.setColumnName("c_double")
.setColumnType(DOUBLE)
.setJdbcTypeHandle(new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), 0, 0, Optional.empty(), Optional.empty()))
.setJdbcTypeHandle(new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), 0, Optional.empty(), Optional.empty(), Optional.empty()))
.build();

private static final JdbcClient JDBC_CLIENT = new MySqlClient(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -271,7 +271,7 @@ public Optional<ColumnMapping> toPrestoType(ConnectorSession session, Connection
oracleDoubleWriteFunction(),
OracleClient::fullPushdownIfSupported));
case OracleTypes.NUMBER:
int decimalDigits = typeHandle.getDecimalDigits();
int decimalDigits = typeHandle.getDecimalDigits().orElseThrow(() -> new IllegalStateException("decimal digits not present"));
// Map negative scale to decimal(p+s, 0).
int precision = columnSize + max(-decimalDigits, 0);
int scale = max(decimalDigits, 0);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,6 @@ public Optional<JdbcExpression> rewrite(AggregateFunction aggregateFunction, Cap

return Optional.of(new JdbcExpression(
format("avg(CAST(%s AS double precision))", columnHandle.toSqlExpression(context.getIdentifierQuote())),
new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), 0, 0, Optional.empty(), Optional.empty())));
new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), 0, Optional.empty(), Optional.empty(), Optional.empty())));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ public PostgreSqlClient(
}
this.tableTypes = tableTypes.toArray(new String[0]);

JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), 0, 0, Optional.empty(), Optional.empty());
JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), 0, Optional.empty(), Optional.empty(), Optional.empty());
this.aggregateFunctionRewriter = new AggregateFunctionRewriter(
this::quoted,
ImmutableSet.<AggregateFunctionRule>builder()
Expand Down Expand Up @@ -263,7 +263,7 @@ public List<JdbcColumnHandle> getColumns(ConnectorSession session, JdbcTableHand
resultSet.getInt("DATA_TYPE"),
Optional.of(resultSet.getString("TYPE_NAME")),
resultSet.getInt("COLUMN_SIZE"),
resultSet.getInt("DECIMAL_DIGITS"),
getInteger(resultSet, "DECIMAL_DIGITS"),
Optional.ofNullable(arrayColumnDimensions.get(columnName)),
Optional.empty());
Optional<ColumnMapping> columnMapping = toPrestoType(session, connection, typeHandle);
Expand Down Expand Up @@ -347,7 +347,8 @@ public Optional<ColumnMapping> toPrestoType(ConnectorSession session, Connection
return Optional.of(jsonColumnMapping());
case "timestamptz":
// PostgreSQL's "timestamp with time zone" is reported as Types.TIMESTAMP rather than Types.TIMESTAMP_WITH_TIMEZONE
return Optional.of(timestampWithTimeZoneColumnMapping(typeHandle.getDecimalDigits()));
int decimalDigits = typeHandle.getDecimalDigits().orElseThrow(() -> new IllegalStateException("decimal digits not present"));
return Optional.of(timestampWithTimeZoneColumnMapping(decimalDigits));
case "hstore":
return Optional.of(hstoreColumnMapping(session));
}
Expand All @@ -361,7 +362,8 @@ public Optional<ColumnMapping> toPrestoType(ConnectorSession session, Connection
return Optional.of(timeColumnMappingWithTruncation());
}
if (typeHandle.getJdbcType() == Types.TIMESTAMP) {
TimestampType timestampType = createTimestampType(typeHandle.getDecimalDigits());
int decimalDigits = typeHandle.getDecimalDigits().orElseThrow(() -> new IllegalStateException("decimal digits not present"));
TimestampType timestampType = createTimestampType(decimalDigits);
return Optional.of(ColumnMapping.longMapping(
timestampType,
timestampReadFunction(timestampType),
Expand All @@ -374,8 +376,9 @@ public Optional<ColumnMapping> toPrestoType(ConnectorSession session, Connection
return Optional.of(decimalColumnMapping(createDecimalType(Decimals.MAX_PRECISION, getDecimalDefaultScale(session)), getDecimalRoundingMode(session)));
}
int precision = typeHandle.getColumnSize();
int decimalDigits = typeHandle.getDecimalDigits().orElseThrow(() -> new IllegalStateException("decimal digits not present"));
if (precision > Decimals.MAX_PRECISION) {
int scale = min(typeHandle.getDecimalDigits(), getDecimalDefaultScale(session));
int scale = min(decimalDigits, getDecimalDefaultScale(session));
return Optional.of(decimalColumnMapping(createDecimalType(Decimals.MAX_PRECISION, scale), getDecimalRoundingMode(session)));
}
}
Expand Down Expand Up @@ -481,7 +484,7 @@ public Optional<JdbcExpression> implementAggregation(ConnectorSession session, A

private static Optional<JdbcTypeHandle> toTypeHandle(DecimalType decimalType)
{
return Optional.of(new JdbcTypeHandle(Types.NUMERIC, Optional.of("decimal"), decimalType.getPrecision(), decimalType.getScale(), Optional.empty(), Optional.empty()));
return Optional.of(new JdbcTypeHandle(Types.NUMERIC, Optional.of("decimal"), decimalType.getPrecision(), Optional.of(decimalType.getScale()), Optional.empty(), Optional.empty()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,14 +49,14 @@ public class TestPostgreSqlClient
JdbcColumnHandle.builder()
.setColumnName("c_bigint")
.setColumnType(BIGINT)
.setJdbcTypeHandle(new JdbcTypeHandle(Types.BIGINT, Optional.of("int8"), 0, 0, Optional.empty(), Optional.empty()))
.setJdbcTypeHandle(new JdbcTypeHandle(Types.BIGINT, Optional.of("int8"), 0, Optional.empty(), Optional.empty(), Optional.empty()))
.build();

private static final JdbcColumnHandle DOUBLE_COLUMN =
JdbcColumnHandle.builder()
.setColumnName("c_double")
.setColumnType(DOUBLE)
.setJdbcTypeHandle(new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), 0, 0, Optional.empty(), Optional.empty()))
.setJdbcTypeHandle(new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), 0, Optional.empty(), Optional.empty(), Optional.empty()))
.build();

private static final JdbcClient JDBC_CLIENT = new PostgreSqlClient(
Expand Down

0 comments on commit f6bf01b

Please sign in to comment.