diff --git a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java index 8cb1b539f0a5..92a79676730a 100644 --- a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java +++ b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java @@ -54,6 +54,7 @@ "deprecated.legacy-timestamp", "deprecated.legacy-unnest-array-rows", "deprecated.legacy-update-delete-implementation", + "deprecated.omit-datetime-type-precision", "experimental-syntax-enabled", "experimental.aggregation-operator-unspill-memory-limit", "experimental.filter-and-project-min-output-page-row-count", @@ -82,6 +83,14 @@ }) public class FeaturesConfig { + public enum DataIntegrityVerification + { + NONE, + ABORT, + RETRY, + /**/; + } + @VisibleForTesting public static final String SPILLER_SPILL_PATH = "spiller-spill-path"; @@ -96,7 +105,6 @@ public class FeaturesConfig private CompressionCodec exchangeCompressionCodec = NONE; private boolean exchangeVectorizedSerdeEnabled = true; private boolean pagesIndexEagerCompactionEnabled; - private boolean omitDateTimeTypePrecision; private int maxRecursionDepth = 10; private int re2JDfaStatesLimit = Integer.MAX_VALUE; @@ -126,27 +134,6 @@ public class FeaturesConfig private boolean legacyArithmeticDecimalOperators; - public enum DataIntegrityVerification - { - NONE, - ABORT, - RETRY, - /**/; - } - - public boolean isOmitDateTimeTypePrecision() - { - return omitDateTimeTypePrecision; - } - - @Config("deprecated.omit-datetime-type-precision") - @ConfigDescription("Enable compatibility mode for legacy clients when rendering datetime type names with default precision") - public FeaturesConfig setOmitDateTimeTypePrecision(boolean value) - { - this.omitDateTimeTypePrecision = value; - return this; - } - public boolean isRedistributeWrites() { return redistributeWrites; diff --git a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java index e2fdd1767b6b..cdfba7e0d0e2 100644 --- a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java +++ b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java @@ -151,7 +151,6 @@ public final class SystemSessionProperties public static final String REQUIRED_WORKERS_COUNT = "required_workers_count"; public static final String REQUIRED_WORKERS_MAX_WAIT_TIME = "required_workers_max_wait_time"; public static final String COST_ESTIMATION_WORKER_COUNT = "cost_estimation_worker_count"; - public static final String OMIT_DATETIME_TYPE_PRECISION = "omit_datetime_type_precision"; public static final String USE_LEGACY_WINDOW_FILTER_PUSHDOWN = "use_legacy_window_filter_pushdown"; public static final String MAX_UNACKNOWLEDGED_SPLITS_PER_TASK = "max_unacknowledged_splits_per_task"; public static final String MERGE_PROJECT_WITH_VALUES = "merge_project_with_values"; @@ -744,11 +743,6 @@ public SystemSessionProperties( null, value -> validateIntegerValue(value, COST_ESTIMATION_WORKER_COUNT, 1, true), true), - booleanProperty( - OMIT_DATETIME_TYPE_PRECISION, - "Omit precision when rendering datetime type names with default precision", - featuresConfig.isOmitDateTimeTypePrecision(), - false), booleanProperty( USE_LEGACY_WINDOW_FILTER_PUSHDOWN, "Use legacy window filter pushdown optimizer", @@ -1704,11 +1698,6 @@ public static Integer getCostEstimationWorkerCount(Session session) return session.getSystemProperty(COST_ESTIMATION_WORKER_COUNT, Integer.class); } - public static boolean isOmitDateTimeTypePrecision(Session session) - { - return session.getSystemProperty(OMIT_DATETIME_TYPE_PRECISION, Boolean.class); - } - public static boolean useLegacyWindowFilterPushdown(Session session) { return session.getSystemProperty(USE_LEGACY_WINDOW_FILTER_PUSHDOWN, Boolean.class); diff --git a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaPageSource.java b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaPageSource.java index ac950becba50..659d504f2a15 100644 --- a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaPageSource.java +++ b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaPageSource.java @@ -51,7 +51,6 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static io.trino.SystemSessionProperties.isOmitDateTimeTypePrecision; import static io.trino.connector.informationschema.InformationSchemaMetadata.defaultPrefixes; import static io.trino.connector.informationschema.InformationSchemaMetadata.isTablesEnumeratingTable; import static io.trino.metadata.MetadataListing.getRelationTypes; @@ -62,7 +61,6 @@ import static io.trino.metadata.MetadataListing.listTables; import static io.trino.spi.security.PrincipalType.USER; import static io.trino.spi.type.TypeUtils.writeNativeValue; -import static io.trino.type.TypeUtils.getDisplayLabel; import static java.util.Objects.requireNonNull; public class InformationSchemaPageSource @@ -257,7 +255,7 @@ private void addColumnsRecords(QualifiedTablePrefix prefix) ordinalPosition, column.getDefaultValue().orElse(null), column.isNullable() ? "YES" : "NO", - getDisplayLabel(column.getType(), isOmitDateTimeTypePrecision(session)), + column.getType().getDisplayName(), column.getComment(), column.getExtraInfo(), column.getComment()); diff --git a/core/trino-main/src/main/java/io/trino/connector/system/jdbc/ColumnJdbcTable.java b/core/trino-main/src/main/java/io/trino/connector/system/jdbc/ColumnJdbcTable.java index 66fa9d1d9749..2acc7650539e 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/jdbc/ColumnJdbcTable.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/jdbc/ColumnJdbcTable.java @@ -64,7 +64,6 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.airlift.slice.Slices.utf8Slice; -import static io.trino.SystemSessionProperties.isOmitDateTimeTypePrecision; import static io.trino.connector.system.jdbc.FilterUtil.isImpossibleObjectName; import static io.trino.connector.system.jdbc.FilterUtil.tablePrefix; import static io.trino.connector.system.jdbc.FilterUtil.tryGetSingleVarcharValue; @@ -83,7 +82,6 @@ import static io.trino.spi.type.TinyintType.TINYINT; import static io.trino.spi.type.VarbinaryType.VARBINARY; import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.type.TypeUtils.getDisplayLabel; import static java.lang.Math.min; import static java.util.Objects.requireNonNull; @@ -245,7 +243,6 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect } Session session = ((FullConnectorSession) connectorSession).getSession(); - boolean omitDateTimeTypePrecision = isOmitDateTimeTypePrecision(session); Domain catalogDomain = constraint.getDomain(0, VARCHAR); Domain schemaDomain = constraint.getDomain(1, VARCHAR); @@ -266,7 +263,7 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect if ((schemaDomain.isAll() && tableDomain.isAll()) || schemaFilter.isPresent()) { QualifiedTablePrefix tablePrefix = tablePrefix(catalog, schemaFilter, tableFilter); Map> tableColumns = listTableColumns(session, metadata, accessControl, tablePrefix); - addColumnsRow(table, catalog, tableColumns, omitDateTimeTypePrecision); + addColumnsRow(table, catalog, tableColumns); } else { Collection schemas = listSchemas(session, metadata, accessControl, catalog, schemaFilter); @@ -286,7 +283,7 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect } Map> tableColumns = listTableColumns(session, metadata, accessControl, new QualifiedTablePrefix(catalog, schema, tableName)); - addColumnsRow(table, catalog, tableColumns, omitDateTimeTypePrecision); + addColumnsRow(table, catalog, tableColumns); } } } @@ -294,14 +291,14 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect return table.build().cursor(); } - private static void addColumnsRow(Builder builder, String catalog, Map> columns, boolean isOmitTimestampPrecision) + private static void addColumnsRow(Builder builder, String catalog, Map> columns) { for (Entry> entry : columns.entrySet()) { - addColumnRows(builder, catalog, entry.getKey(), entry.getValue(), isOmitTimestampPrecision); + addColumnRows(builder, catalog, entry.getKey(), entry.getValue()); } } - private static void addColumnRows(Builder builder, String catalog, SchemaTableName tableName, List columns, boolean isOmitTimestampPrecision) + private static void addColumnRows(Builder builder, String catalog, SchemaTableName tableName, List columns) { int ordinalPosition = 1; for (ColumnMetadata column : columns) { @@ -320,7 +317,7 @@ private static void addColumnRows(Builder builder, String catalog, SchemaTableNa // data_type jdbcDataType(column.getType()), // type_name - getDisplayLabel(column.getType(), isOmitTimestampPrecision), + column.getType().getDisplayName(), // column_size columnSize(column.getType()), // buffer_length diff --git a/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeInputRewrite.java b/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeInputRewrite.java index 805dd5405586..2fffb3cb1872 100644 --- a/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeInputRewrite.java +++ b/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeInputRewrite.java @@ -42,7 +42,6 @@ import java.util.Map; import java.util.Optional; -import static io.trino.SystemSessionProperties.isOmitDateTimeTypePrecision; import static io.trino.execution.ParameterExtractor.extractParameters; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -56,7 +55,6 @@ import static io.trino.sql.QueryUtil.values; import static io.trino.sql.analyzer.QueryType.DESCRIBE; import static io.trino.sql.analyzer.TypeSignatureTranslator.toSqlType; -import static io.trino.type.TypeUtils.getDisplayLabel; import static io.trino.type.UnknownType.UNKNOWN; import static java.util.Objects.requireNonNull; @@ -134,7 +132,7 @@ protected Node visitDescribeInput(DescribeInput node, Void context) ImmutableList.Builder builder = ImmutableList.builder(); for (int i = 0; i < parameters.size(); i++) { - builder.add(createDescribeInputRow(session, i, parameters.get(i), analysis)); + builder.add(createDescribeInputRow(i, parameters.get(i), analysis)); } // return the positions and types of all parameters @@ -163,7 +161,7 @@ private static Query createDescribeInputQuery(Row[] rows, Optional limit) limit); } - private static Row createDescribeInputRow(Session session, int position, Parameter parameter, Analysis queryAnalysis) + private static Row createDescribeInputRow(int position, Parameter parameter, Analysis queryAnalysis) { Type type = queryAnalysis.getCoercion(parameter); if (type == null) { @@ -172,7 +170,7 @@ private static Row createDescribeInputRow(Session session, int position, Paramet return row( new LongLiteral(Integer.toString(position)), - new StringLiteral(getDisplayLabel(type, isOmitDateTimeTypePrecision(session)))); + new StringLiteral(type.getDisplayName())); } @Override diff --git a/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeOutputRewrite.java b/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeOutputRewrite.java index 4ac7daedf741..ee15d5f8e06f 100644 --- a/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeOutputRewrite.java +++ b/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeOutputRewrite.java @@ -45,7 +45,6 @@ import java.util.Map; import java.util.Optional; -import static io.trino.SystemSessionProperties.isOmitDateTimeTypePrecision; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -57,7 +56,6 @@ import static io.trino.sql.QueryUtil.values; import static io.trino.sql.analyzer.QueryType.DESCRIBE; import static io.trino.sql.analyzer.TypeSignatureTranslator.toSqlType; -import static io.trino.type.TypeUtils.getDisplayLabel; import static java.util.Objects.requireNonNull; public final class DescribeOutputRewrite @@ -187,7 +185,7 @@ private Row createDescribeOutputRow(Field field, Analysis analysis) new StringLiteral(originTable.map(QualifiedObjectName::catalogName).orElse("")), new StringLiteral(originTable.map(QualifiedObjectName::schemaName).orElse("")), new StringLiteral(originTable.map(QualifiedObjectName::objectName).orElse("")), - new StringLiteral(getDisplayLabel(field.getType(), isOmitDateTimeTypePrecision(session))), + new StringLiteral(field.getType().getDisplayName()), typeSize, new BooleanLiteral(String.valueOf(field.isAliased()))); } diff --git a/core/trino-main/src/main/java/io/trino/type/TypeUtils.java b/core/trino-main/src/main/java/io/trino/type/TypeUtils.java index fb9c2e61d5e7..18d05d3a584b 100644 --- a/core/trino-main/src/main/java/io/trino/type/TypeUtils.java +++ b/core/trino-main/src/main/java/io/trino/type/TypeUtils.java @@ -13,72 +13,9 @@ */ package io.trino.type; -import io.trino.spi.type.ArrayType; -import io.trino.spi.type.MapType; -import io.trino.spi.type.RowType; -import io.trino.spi.type.StandardTypes; -import io.trino.spi.type.TimeType; -import io.trino.spi.type.TimeWithTimeZoneType; -import io.trino.spi.type.TimestampType; -import io.trino.spi.type.TimestampWithTimeZoneType; -import io.trino.spi.type.Type; - -import static io.trino.spi.type.StandardTypes.ARRAY; -import static io.trino.spi.type.StandardTypes.MAP; -import static io.trino.spi.type.StandardTypes.ROW; -import static java.util.stream.Collectors.joining; - public final class TypeUtils { public static final int NULL_HASH_CODE = 0; private TypeUtils() {} - - public static String getDisplayLabel(Type type, boolean legacy) - { - if (legacy) { - return getDisplayLabelForLegacyClients(type); - } - return type.getDisplayName(); - } - - private static String getDisplayLabelForLegacyClients(Type type) - { - if (type instanceof TimestampType timestampType && timestampType.getPrecision() == TimestampType.DEFAULT_PRECISION) { - return StandardTypes.TIMESTAMP; - } - if (type instanceof TimestampWithTimeZoneType timestampWithTimeZoneType && timestampWithTimeZoneType.getPrecision() == TimestampWithTimeZoneType.DEFAULT_PRECISION) { - return StandardTypes.TIMESTAMP_WITH_TIME_ZONE; - } - if (type instanceof TimeType timeType && timeType.getPrecision() == TimeType.DEFAULT_PRECISION) { - return StandardTypes.TIME; - } - if (type instanceof TimeWithTimeZoneType timeWithTimeZoneType && timeWithTimeZoneType.getPrecision() == TimeWithTimeZoneType.DEFAULT_PRECISION) { - return StandardTypes.TIME_WITH_TIME_ZONE; - } - if (type instanceof ArrayType arrayType) { - return ARRAY + "(" + getDisplayLabelForLegacyClients(arrayType.getElementType()) + ")"; - } - if (type instanceof MapType mapType) { - return MAP + "(" + getDisplayLabelForLegacyClients(mapType.getKeyType()) + ", " + getDisplayLabelForLegacyClients(mapType.getValueType()) + ")"; - } - if (type instanceof RowType rowType) { - return getRowDisplayLabelForLegacyClients(rowType); - } - - return type.getDisplayName(); - } - - private static String getRowDisplayLabelForLegacyClients(RowType type) - { - return type.getFields().stream() - .map(field -> { - String typeDisplayName = getDisplayLabelForLegacyClients(field.getType()); - if (field.getName().isPresent()) { - return field.getName().get() + ' ' + typeDisplayName; - } - return typeDisplayName; - }) - .collect(joining(", ", ROW + "(", ")")); - } } diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java index 03f292ca7465..1387fa1559fb 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java @@ -61,7 +61,6 @@ public void testDefaults() .setFilterAndProjectMinOutputPageRowCount(256) .setMaxRecursionDepth(10) .setMaxGroupingSets(2048) - .setOmitDateTimeTypePrecision(false) .setLegacyCatalogRoles(false) .setIncrementalHashArrayLoadFactorEnabled(true) .setHideInaccessibleColumns(false) @@ -97,7 +96,6 @@ public void testExplicitPropertyMappings() .put("filter-and-project-min-output-page-row-count", "2048") .put("max-recursion-depth", "8") .put("analyzer.max-grouping-sets", "2047") - .put("deprecated.omit-datetime-type-precision", "true") .put("deprecated.legacy-catalog-roles", "true") .put("incremental-hash-array-load-factor.enabled", "false") .put("hide-inaccessible-columns", "true") @@ -130,7 +128,6 @@ public void testExplicitPropertyMappings() .setFilterAndProjectMinOutputPageRowCount(2048) .setMaxRecursionDepth(8) .setMaxGroupingSets(2047) - .setOmitDateTimeTypePrecision(true) .setLegacyCatalogRoles(true) .setIncrementalHashArrayLoadFactorEnabled(false) .setHideInaccessibleColumns(true) diff --git a/testing/trino-test-jdbc-compatibility-old-driver/src/test/java/io/trino/JdbcDriverCapabilities.java b/testing/trino-test-jdbc-compatibility-old-driver/src/test/java/io/trino/JdbcDriverCapabilities.java index 58585d15fde2..aba99c1badf7 100644 --- a/testing/trino-test-jdbc-compatibility-old-driver/src/test/java/io/trino/JdbcDriverCapabilities.java +++ b/testing/trino-test-jdbc-compatibility-old-driver/src/test/java/io/trino/JdbcDriverCapabilities.java @@ -22,8 +22,6 @@ public final class JdbcDriverCapabilities { private JdbcDriverCapabilities() {} - public static final int VERSION_HEAD = 0; - public static Optional testedVersion() { return Optional.ofNullable(System.getenv("TRINO_JDBC_VERSION_UNDER_TEST")).map(Integer::valueOf); @@ -50,37 +48,4 @@ private static Optional getDriver(String url) return Optional.empty(); } } - - public static boolean supportsSessionPropertiesViaConnectionUri() - { - return driverVersion() == VERSION_HEAD || driverVersion() >= 330; - } - - public static boolean supportsParametricTimestamp() - { - return driverVersion() == VERSION_HEAD || driverVersion() >= 335; - } - - public static boolean supportsParametricTimestampWithTimeZone() - { - return driverVersion() == VERSION_HEAD || driverVersion() >= 337; - } - - public static boolean correctlyReportsTimestampWithTimeZone() - { - return driverVersion() == VERSION_HEAD || driverVersion() >= 348; - } - - public static boolean supportsTimestampObjectRepresentationInCollections() - { - return driverVersion() == VERSION_HEAD || driverVersion() >= 348; - } - - public static boolean hasBrokenParametricTimestampWithTimeZoneSupport() - { - // Since 335 JDBC client reports ClientCapabilities.PARAMETRIC_DATETIME which is used server side to determine whether - // timestamp with time zone can be returned with given precision instead of default one (3). - // JDBC client 335 and 336 are broken in regard to handling timestamp with time zone correctly. - return driverVersion() == 335 || driverVersion() == 336; - } } diff --git a/testing/trino-test-jdbc-compatibility-old-driver/src/test/java/io/trino/TestJdbcCompatibility.java b/testing/trino-test-jdbc-compatibility-old-driver/src/test/java/io/trino/TestJdbcCompatibility.java index a10e22a225e4..bb9982c93f55 100644 --- a/testing/trino-test-jdbc-compatibility-old-driver/src/test/java/io/trino/TestJdbcCompatibility.java +++ b/testing/trino-test-jdbc-compatibility-old-driver/src/test/java/io/trino/TestJdbcCompatibility.java @@ -14,7 +14,6 @@ package io.trino; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import io.airlift.log.Logging; import io.trino.plugin.mongodb.MongoPlugin; import io.trino.server.testing.TestingTrinoServer; @@ -40,14 +39,8 @@ import java.util.Properties; import java.util.function.Consumer; -import static io.trino.JdbcDriverCapabilities.correctlyReportsTimestampWithTimeZone; import static io.trino.JdbcDriverCapabilities.driverVersion; -import static io.trino.JdbcDriverCapabilities.hasBrokenParametricTimestampWithTimeZoneSupport; import static io.trino.JdbcDriverCapabilities.jdbcDriver; -import static io.trino.JdbcDriverCapabilities.supportsParametricTimestamp; -import static io.trino.JdbcDriverCapabilities.supportsParametricTimestampWithTimeZone; -import static io.trino.JdbcDriverCapabilities.supportsSessionPropertiesViaConnectionUri; -import static io.trino.JdbcDriverCapabilities.supportsTimestampObjectRepresentationInCollections; import static io.trino.JdbcDriverCapabilities.testedVersion; import static java.lang.String.format; import static java.sql.Types.ARRAY; @@ -73,7 +66,6 @@ public class TestJdbcCompatibility { private static final Optional VERSION_UNDER_TEST = testedVersion(); - private static final int TIMESTAMP_DEFAULT_PRECISION = 3; private final TestingTrinoServer server; private final String serverUrl; @@ -136,34 +128,13 @@ public void testSelectTimestamp() @Test public void testSelectTimestampWithTimeZone() { - if (hasBrokenParametricTimestampWithTimeZoneSupport()) { - abort("This version reports PARAMETRIC_DATETIME client capability but TIMESTAMP WITH TIME ZONE is not supported"); - } - String query = "SELECT timestamp '2012-10-31 01:00 Australia/Eucla'"; - checkRepresentation(query, Timestamp.valueOf("2012-10-30 10:15:00.000"), correctlyReportsTimestampWithTimeZone() ? TIMESTAMP_WITH_TIMEZONE : TIMESTAMP, ResultSet::getTimestamp); + checkRepresentation(query, Timestamp.valueOf("2012-10-30 10:15:00.000"), TIMESTAMP_WITH_TIMEZONE, ResultSet::getTimestamp); } @Test public void testSelectParametricTimestamp() { - if (!supportsParametricTimestamp()) { - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23")), "2004-08-24 23:55:23.000"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.1'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.1")), "2004-08-24 23:55:23.100"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.12'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.12")), "2004-08-24 23:55:23.120"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.123'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.1234'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.12345'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.123456'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.1234567'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.12345678'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.123456789'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.1234567890'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.12345678901'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.123456789012'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.123")), "2004-08-24 23:55:23.123"); - return; - } - testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23")), "2004-08-24 23:55:23"); testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.1'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.1")), "2004-08-24 23:55:23.1"); testSelectParametricTimestamp("TIMESTAMP '2004-08-24 23:55:23.12'", Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.12")), "2004-08-24 23:55:23.12"); @@ -192,27 +163,6 @@ private void testSelectParametricTimestamp(String expression, Object expectedVal @Test public void testSelectParametricTimestampWithTimeZone() { - if (hasBrokenParametricTimestampWithTimeZoneSupport()) { - abort("This version reports PARAMETRIC_DATETIME client capability but TIMESTAMP WITH TIME ZONE is not supported"); - } - - if (!supportsParametricTimestampWithTimeZone()) { - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23")), "2004-08-24 23:55:23.000 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.1")), "2004-08-24 23:55:23.100 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.12")), "2004-08-24 23:55:23.120 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.123 Australia/Eucla'", 3, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.1234 Australia/Eucla'", 4, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.12345 Australia/Eucla'", 5, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.123456 Australia/Eucla'", 6, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.1234567 Australia/Eucla'", 7, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.12345678 Australia/Eucla'", 8, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.123456789 Australia/Eucla'", 9, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.1234567890 Australia/Eucla'", 10, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.12345678901 Australia/Eucla'", 11, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.123456789012 Australia/Eucla'", 12, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.123")), "2004-08-24 23:55:23.123 Australia/Eucla"); - return; - } - testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23")), "2004-08-24 23:55:23 Australia/Eucla"); testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.1")), "2004-08-24 23:55:23.1 Australia/Eucla"); testSelectParametricTimestampWithTimeZone("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T09:10:23.12")), "2004-08-24 23:55:23.12 Australia/Eucla"); @@ -231,7 +181,7 @@ public void testSelectParametricTimestampWithTimeZone() private void testSelectParametricTimestampWithTimeZone(String expression, int expectedPrecision, Object expectedValue, String expectedString) { String query = "SELECT " + expression; - checkRepresentation(query, correctlyReportsTimestampWithTimeZone() ? TIMESTAMP_WITH_TIMEZONE : TIMESTAMP, (resultSet, columnIndex) -> { + checkRepresentation(query, TIMESTAMP_WITH_TIMEZONE, (resultSet, columnIndex) -> { assertThat(resultSet.getTimestamp(columnIndex)).isEqualTo(expectedValue); assertThat(resultSet.getObject(columnIndex)).isEqualTo(expectedValue); assertThat(resultSet.getString(columnIndex)).isEqualTo(expectedString); @@ -242,40 +192,6 @@ private void testSelectParametricTimestampWithTimeZone(String expression, int ex @Test public void testSelectParametricTimestampInMap() { - if (!supportsParametricTimestamp()) { - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23'", 0, "2004-08-24 23:55:23.000"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.1'", 1, "2004-08-24 23:55:23.100"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.12'", 2, "2004-08-24 23:55:23.120"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.123'", 3, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.1234'", 4, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.12345'", 5, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.123456'", 6, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.1234567'", 7, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.12345678'", 8, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.123456789'", 9, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.1234567890'", 10, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.12345678901'", 11, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.123456789012'", 12, "2004-08-24 23:55:23.123"); - return; - } - - if (!supportsTimestampObjectRepresentationInCollections()) { - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23'", 0, "2004-08-24 23:55:23"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.1'", 1, "2004-08-24 23:55:23.1"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.12'", 2, "2004-08-24 23:55:23.12"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.123'", 3, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.1234'", 4, "2004-08-24 23:55:23.1234"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.12345'", 5, "2004-08-24 23:55:23.12345"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.123456'", 6, "2004-08-24 23:55:23.123456"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.1234567'", 7, "2004-08-24 23:55:23.1234567"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.12345678'", 8, "2004-08-24 23:55:23.12345678"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.123456789'", 9, "2004-08-24 23:55:23.123456789"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.1234567890'", 10, "2004-08-24 23:55:23.1234567890"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.12345678901'", 11, "2004-08-24 23:55:23.12345678901"); - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.123456789012'", 12, "2004-08-24 23:55:23.123456789012"); - return; - } - testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23'", 0, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23"))); testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.1'", 1, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.1"))); testSelectParametricTimestampInMap("TIMESTAMP '2004-08-24 23:55:23.12'", 2, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.12"))); @@ -301,44 +217,6 @@ private void testSelectParametricTimestampInMap(String elementExpression, int ex @Test public void testSelectParametricTimestampWithTimeZoneInMap() { - if (hasBrokenParametricTimestampWithTimeZoneSupport()) { - abort("This version reports PARAMETRIC_DATETIME client capability but TIMESTAMP WITH TIME ZONE is not supported"); - } - - if (!supportsParametricTimestampWithTimeZone()) { - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, "2004-08-24 23:55:23.000 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, "2004-08-24 23:55:23.100 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, "2004-08-24 23:55:23.120 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.123 Australia/Eucla'", 3, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.1234 Australia/Eucla'", 4, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.12345 Australia/Eucla'", 5, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.123456 Australia/Eucla'", 6, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.1234567 Australia/Eucla'", 7, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.12345678 Australia/Eucla'", 8, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.123456789 Australia/Eucla'", 9, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.1234567890 Australia/Eucla'", 10, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.12345678901 Australia/Eucla'", 11, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.123456789012 Australia/Eucla'", 12, "2004-08-24 23:55:23.123 Australia/Eucla"); - return; - } - - if (!supportsTimestampObjectRepresentationInCollections()) { - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, "2004-08-24 23:55:23 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, "2004-08-24 23:55:23.1 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, "2004-08-24 23:55:23.12 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.123 Australia/Eucla'", 3, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.1234 Australia/Eucla'", 4, "2004-08-24 23:55:23.1234 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.12345 Australia/Eucla'", 5, "2004-08-24 23:55:23.12345 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.123456 Australia/Eucla'", 6, "2004-08-24 23:55:23.123456 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.1234567 Australia/Eucla'", 7, "2004-08-24 23:55:23.1234567 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.12345678 Australia/Eucla'", 8, "2004-08-24 23:55:23.12345678 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.123456789 Australia/Eucla'", 9, "2004-08-24 23:55:23.123456789 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.1234567890 Australia/Eucla'", 10, "2004-08-24 23:55:23.1234567890 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.12345678901 Australia/Eucla'", 11, "2004-08-24 23:55:23.12345678901 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.123456789012 Australia/Eucla'", 12, "2004-08-24 23:55:23.123456789012 Australia/Eucla"); - return; - } - testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, Timestamp.from(ZonedDateTime.parse("2004-08-24T23:55:23+08:45[Australia/Eucla]").toInstant())); testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, Timestamp.from(ZonedDateTime.parse("2004-08-24T23:55:23.1+08:45[Australia/Eucla]").toInstant())); testSelectParametricTimestampWithTimeZoneInMap("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, Timestamp.from(ZonedDateTime.parse("2004-08-24T23:55:23.12+08:45[Australia/Eucla]").toInstant())); @@ -365,40 +243,6 @@ private void testSelectParametricTimestampWithTimeZoneInMap(String elementExpres @Test public void testSelectParametricTimestampInArray() { - if (!supportsParametricTimestamp()) { - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23'", 0, "2004-08-24 23:55:23.000"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.1'", 1, "2004-08-24 23:55:23.100"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.12'", 2, "2004-08-24 23:55:23.120"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.123'", 3, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.1234'", 4, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.12345'", 5, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.123456'", 6, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.1234567'", 7, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.12345678'", 8, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.123456789'", 9, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.1234567890'", 10, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.12345678901'", 11, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.123456789012'", 12, "2004-08-24 23:55:23.123"); - return; - } - - if (!supportsTimestampObjectRepresentationInCollections()) { - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23'", 0, "2004-08-24 23:55:23"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.1'", 1, "2004-08-24 23:55:23.1"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.12'", 2, "2004-08-24 23:55:23.12"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.123'", 3, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.1234'", 4, "2004-08-24 23:55:23.1234"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.12345'", 5, "2004-08-24 23:55:23.12345"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.123456'", 6, "2004-08-24 23:55:23.123456"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.1234567'", 7, "2004-08-24 23:55:23.1234567"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.12345678'", 8, "2004-08-24 23:55:23.12345678"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.123456789'", 9, "2004-08-24 23:55:23.123456789"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.1234567890'", 10, "2004-08-24 23:55:23.1234567890"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.12345678901'", 11, "2004-08-24 23:55:23.12345678901"); - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.123456789012'", 12, "2004-08-24 23:55:23.123456789012"); - return; - } - testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23'", 0, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23"))); testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.1'", 1, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.1"))); testSelectParametricTimestampInArray("TIMESTAMP '2004-08-24 23:55:23.12'", 2, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.12"))); @@ -424,44 +268,6 @@ private void testSelectParametricTimestampInArray(String elementExpression, int @Test public void testSelectParametricTimestampWithTimeZoneInArray() { - if (hasBrokenParametricTimestampWithTimeZoneSupport()) { - abort("This version reports PARAMETRIC_DATETIME client capability but TIMESTAMP WITH TIME ZONE is not supported"); - } - - if (!supportsParametricTimestampWithTimeZone()) { - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, "2004-08-24 23:55:23.000 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, "2004-08-24 23:55:23.100 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, "2004-08-24 23:55:23.120 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.123 Australia/Eucla'", 3, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.1234 Australia/Eucla'", 4, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.12345 Australia/Eucla'", 5, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.123456 Australia/Eucla'", 6, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.1234567 Australia/Eucla'", 7, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.12345678 Australia/Eucla'", 8, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.123456789 Australia/Eucla'", 9, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.1234567890 Australia/Eucla'", 10, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.12345678901 Australia/Eucla'", 11, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.123456789012 Australia/Eucla'", 12, "2004-08-24 23:55:23.123 Australia/Eucla"); - return; - } - - if (!supportsTimestampObjectRepresentationInCollections()) { - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, "2004-08-24 23:55:23 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, "2004-08-24 23:55:23.1 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, "2004-08-24 23:55:23.12 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.123 Australia/Eucla'", 3, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.1234 Australia/Eucla'", 4, "2004-08-24 23:55:23.1234 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.12345 Australia/Eucla'", 5, "2004-08-24 23:55:23.12345 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.123456 Australia/Eucla'", 6, "2004-08-24 23:55:23.123456 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.1234567 Australia/Eucla'", 7, "2004-08-24 23:55:23.1234567 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.12345678 Australia/Eucla'", 8, "2004-08-24 23:55:23.12345678 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.123456789 Australia/Eucla'", 9, "2004-08-24 23:55:23.123456789 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.1234567890 Australia/Eucla'", 10, "2004-08-24 23:55:23.1234567890 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.12345678901 Australia/Eucla'", 11, "2004-08-24 23:55:23.12345678901 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.123456789012 Australia/Eucla'", 12, "2004-08-24 23:55:23.123456789012 Australia/Eucla"); - return; - } - testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, Timestamp.from(ZonedDateTime.parse("2004-08-24T23:55:23+08:45[Australia/Eucla]").toInstant())); testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, Timestamp.from(ZonedDateTime.parse("2004-08-24T23:55:23.1+08:45[Australia/Eucla]").toInstant())); testSelectParametricTimestampWithTimeZoneInArray("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, Timestamp.from(ZonedDateTime.parse("2004-08-24T23:55:23.12+08:45[Australia/Eucla]").toInstant())); @@ -487,40 +293,6 @@ private void testSelectParametricTimestampWithTimeZoneInArray(String elementExpr @Test public void testSelectParametricTimestampInRow() { - if (!supportsParametricTimestamp()) { - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23'", 0, "2004-08-24 23:55:23.000"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.1'", 1, "2004-08-24 23:55:23.100"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.12'", 2, "2004-08-24 23:55:23.120"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.123'", 3, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.1234'", 4, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.12345'", 5, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.123456'", 6, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.1234567'", 7, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.12345678'", 8, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.123456789'", 9, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.1234567890'", 10, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.12345678901'", 11, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.123456789012'", 12, "2004-08-24 23:55:23.123"); - return; - } - - if (!supportsTimestampObjectRepresentationInCollections()) { - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23'", 0, "2004-08-24 23:55:23"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.1'", 1, "2004-08-24 23:55:23.1"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.12'", 2, "2004-08-24 23:55:23.12"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.123'", 3, "2004-08-24 23:55:23.123"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.1234'", 4, "2004-08-24 23:55:23.1234"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.12345'", 5, "2004-08-24 23:55:23.12345"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.123456'", 6, "2004-08-24 23:55:23.123456"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.1234567'", 7, "2004-08-24 23:55:23.1234567"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.12345678'", 8, "2004-08-24 23:55:23.12345678"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.123456789'", 9, "2004-08-24 23:55:23.123456789"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.1234567890'", 10, "2004-08-24 23:55:23.1234567890"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.12345678901'", 11, "2004-08-24 23:55:23.12345678901"); - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.123456789012'", 12, "2004-08-24 23:55:23.123456789012"); - return; - } - testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23'", 0, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23"))); testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.1'", 1, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.1"))); testSelectParametricTimestampInRow("TIMESTAMP '2004-08-24 23:55:23.12'", 2, Timestamp.valueOf(LocalDateTime.parse("2004-08-24T23:55:23.12"))); @@ -546,44 +318,6 @@ private void testSelectParametricTimestampInRow(String elementExpression, int pr @Test public void testSelectParametricTimestampWithTimeZoneInRow() { - if (hasBrokenParametricTimestampWithTimeZoneSupport()) { - abort("This version reports PARAMETRIC_DATETIME client capability but TIMESTAMP WITH TIME ZONE is not supported"); - } - - if (!supportsParametricTimestampWithTimeZone()) { - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, "2004-08-24 23:55:23.000 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, "2004-08-24 23:55:23.100 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, "2004-08-24 23:55:23.120 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.123 Australia/Eucla'", 3, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.1234 Australia/Eucla'", 4, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.12345 Australia/Eucla'", 5, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.123456 Australia/Eucla'", 6, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.1234567 Australia/Eucla'", 7, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.12345678 Australia/Eucla'", 8, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.123456789 Australia/Eucla'", 9, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.1234567890 Australia/Eucla'", 10, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.12345678901 Australia/Eucla'", 11, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.123456789012 Australia/Eucla'", 12, "2004-08-24 23:55:23.123 Australia/Eucla"); - return; - } - - if (!supportsTimestampObjectRepresentationInCollections()) { - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, "2004-08-24 23:55:23 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, "2004-08-24 23:55:23.1 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, "2004-08-24 23:55:23.12 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.123 Australia/Eucla'", 3, "2004-08-24 23:55:23.123 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.1234 Australia/Eucla'", 4, "2004-08-24 23:55:23.1234 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.12345 Australia/Eucla'", 5, "2004-08-24 23:55:23.12345 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.123456 Australia/Eucla'", 6, "2004-08-24 23:55:23.123456 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.1234567 Australia/Eucla'", 7, "2004-08-24 23:55:23.1234567 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.12345678 Australia/Eucla'", 8, "2004-08-24 23:55:23.12345678 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.123456789 Australia/Eucla'", 9, "2004-08-24 23:55:23.123456789 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.1234567890 Australia/Eucla'", 10, "2004-08-24 23:55:23.1234567890 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.12345678901 Australia/Eucla'", 11, "2004-08-24 23:55:23.12345678901 Australia/Eucla"); - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.123456789012 Australia/Eucla'", 12, "2004-08-24 23:55:23.123456789012 Australia/Eucla"); - return; - } - testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23 Australia/Eucla'", 0, Timestamp.from(ZonedDateTime.parse("2004-08-24T23:55:23+08:45[Australia/Eucla]").toInstant())); testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.1 Australia/Eucla'", 1, Timestamp.from(ZonedDateTime.parse("2004-08-24T23:55:23.1+08:45[Australia/Eucla]").toInstant())); testSelectParametricTimestampWithTimeZoneInRow("TIMESTAMP '2004-08-24 23:55:23.12 Australia/Eucla'", 2, Timestamp.from(ZonedDateTime.parse("2004-08-24T23:55:23.12+08:45[Australia/Eucla]").toInstant())); @@ -610,9 +344,8 @@ private void testSelectParametricTimestampWithTimeZoneInRow(String elementExpres public void testSelectMongoObjectId() { String query = "SELECT ObjectId('55b151633864d6438c61a9ce') AS objectId"; - checkRepresentation(query, JAVA_OBJECT, (resultSet, columnIndex) -> { - assertThat(resultSet.getObject(columnIndex)).isEqualTo(new byte[] {85, -79, 81, 99, 56, 100, -42, 67, -116, 97, -87, -50}); - }); + checkRepresentation(query, JAVA_OBJECT, (resultSet, columnIndex) -> + assertThat(resultSet.getObject(columnIndex)).isEqualTo(new byte[] {85, -79, 81, 99, 56, 100, -42, 67, -116, 97, -87, -50})); } @Test @@ -634,6 +367,7 @@ public void testSelectMultiMap() { String query = "SELECT multimap_from_entries(ARRAY[(1, 'x'), (2, 'y'), (1, 'z')])"; checkRepresentation(query, JAVA_OBJECT, (rs, column) -> { + @SuppressWarnings("unchecked") Map values = (Map) rs.getObject(column); assertThat(values).containsEntry(1, ImmutableList.of("x", "z")); assertThat(values).containsEntry(2, ImmutableList.of("y")); @@ -656,9 +390,8 @@ public void testSelectRow() public void testSelectJson() { String query = "SELECT json_parse('[{\"1\":\"value\"}, 2, 3]')"; - checkRepresentation(query, JAVA_OBJECT, (rs, column) -> { - assertThat(rs.getObject(column)).isEqualTo("[{\"1\":\"value\"},2,3]"); - }); + checkRepresentation(query, JAVA_OBJECT, (rs, column) -> + assertThat(rs.getObject(column)).isEqualTo("[{\"1\":\"value\"},2,3]")); } private void checkRepresentation(String query, T expectedValue, int expectedType, ResultSetMapper extractValue) @@ -676,7 +409,7 @@ private void checkRepresentation(String query, T expectedValue, int expected } } - public void checkRepresentation(String query, int expectedType, ResultSetAssertion extractValue) + private void checkRepresentation(String query, int expectedType, ResultSetAssertion extractValue) { try (Connection connection = getConnection(); PreparedStatement statement = connection.prepareStatement(query); @@ -693,20 +426,13 @@ public void checkRepresentation(String query, int expectedType, ResultSetAsserti private void checkDescribeTimestampType(String query, String expectedTypePattern, int precision, boolean withTimeZone) { - assertDescribeType(query, format(expectedTypePattern, describeTimestampType(precision, true, withTimeZone)), true); - assertDescribeType(query, format(expectedTypePattern, describeTimestampType(precision, false, withTimeZone)), false); - assertDescribeOutputType(query, format(expectedTypePattern, describeTimestampType(precision, true, withTimeZone)), true); - assertDescribeOutputType(query, format(expectedTypePattern, describeTimestampType(precision, false, withTimeZone)), false); + assertDescribeType(query, format(expectedTypePattern, describeTimestampType(precision, withTimeZone))); + assertDescribeOutputType(query, format(expectedTypePattern, describeTimestampType(precision, withTimeZone))); } - private void assertDescribeType(String query, String expectedType, boolean omitPrecission) + private void assertDescribeType(String query, String expectedType) { - if (omitPrecission && !supportsSessionPropertiesViaConnectionUri()) { - // Driver does not support setting session properties - return; - } - - useConnection(omitDateTimeTypePrecision(omitPrecission), connection -> { + useConnection(connection -> { try { connection.prepareStatement(format("SELECT 1 FROM (%s AS timestamp) WHERE timestamp = ?", query)); @@ -721,14 +447,9 @@ private void assertDescribeType(String query, String expectedType, boolean omitP }); } - private void assertDescribeOutputType(String query, String expectedType, boolean omitPrecision) + private void assertDescribeOutputType(String query, String expectedType) { - if (omitPrecision && !supportsSessionPropertiesViaConnectionUri()) { - // Driver does not support setting session properties - return; - } - - useConnection(omitDateTimeTypePrecision(omitPrecision), connection -> { + useConnection(connection -> { try { connection.prepareStatement(query); @@ -743,16 +464,8 @@ private void assertDescribeOutputType(String query, String expectedType, boolean }); } - private static String describeTimestampType(int precision, boolean ommitPrecision, boolean withTimezone) + private static String describeTimestampType(int precision, boolean withTimezone) { - if (ommitPrecision && precision == TIMESTAMP_DEFAULT_PRECISION) { - if (withTimezone) { - return "timestamp with time zone"; - } - - return "timestamp"; - } - if (withTimezone) { return format("timestamp(%d) with time zone", precision); } @@ -793,24 +506,17 @@ private static Object getSingleElementFromMap(ResultSet resultSet, int columnInd private Connection getConnection() throws SQLException - { - return getConnection(ImmutableMap.of()); - } - - private Connection getConnection(Map params) - throws SQLException { Properties properties = new Properties(); - properties.putAll(params); properties.put("user", "test"); properties.put("password", ""); return DriverManager.getConnection(serverUrl, properties); } - private void useConnection(Map connectionParameters, Consumer consumer) + private void useConnection(Consumer consumer) { - try (Connection conn = getConnection(connectionParameters)) { + try (Connection conn = getConnection()) { consumer.accept(conn); } catch (Exception e) { @@ -818,15 +524,6 @@ private void useConnection(Map connectionParameters, Consumer omitDateTimeTypePrecision(boolean omit) - { - if (!omit) { - return ImmutableMap.of(); - } - - return ImmutableMap.of("sessionProperties", "omit_datetime_type_precision:true"); - } - @FunctionalInterface interface ResultSetMapper { diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java index 4ae8fb510130..69e741dd173d 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java @@ -1302,7 +1302,6 @@ public void testDescribeInput() assertEqualsIgnoreOrder(actual, expected); session = Session.builder(getSession()) - .setSystemProperty("omit_datetime_type_precision", "false") .addPreparedStatement( "my_query", "SELECT 1 " + @@ -1322,20 +1321,6 @@ public void testDescribeInput() .row(4, "decimal(3,2)") .build(); assertEqualsIgnoreOrder(actual, expected); - - session = Session.builder(session) - .setSystemProperty("omit_datetime_type_precision", "true") - .build(); - - actual = computeActual(session, "DESCRIBE INPUT my_query"); - expected = resultBuilder(session, BIGINT, VARCHAR) - .row(0, "char(2)") - .row(1, "varchar") - .row(2, "timestamp") - .row(3, "timestamp(6)") - .row(4, "decimal(3,2)") - .build(); - assertEqualsIgnoreOrder(actual, expected); } @Test @@ -1416,27 +1401,14 @@ public void testDescribeOutput() public void testDescribeOutputDateTimeTypes() { Session session = Session.builder(getSession()) - .setSystemProperty("omit_datetime_type_precision", "true") .addPreparedStatement("my_query", "SELECT localtimestamp a, current_timestamp b, localtime c") .build(); MaterializedResult actual = computeActual(session, "DESCRIBE OUTPUT my_query"); MaterializedResult expected = resultBuilder(session, VARCHAR, VARCHAR, VARCHAR, VARCHAR, VARCHAR, BIGINT, BOOLEAN) - .row("a", "", "", "", "timestamp", 8, true) - .row("b", "", "", "", "timestamp with time zone", 8, true) - .row("c", "", "", "", "time", 8, true) - .build(); - assertEqualsIgnoreOrder(actual, expected); - - session = Session.builder(getSession()) - .setSystemProperty("omit_datetime_type_precision", "false") - .addPreparedStatement("my_query", "SELECT localtimestamp a, current_timestamp b") - .build(); - - actual = computeActual(session, "DESCRIBE OUTPUT my_query"); - expected = resultBuilder(session, VARCHAR, VARCHAR, VARCHAR, VARCHAR, VARCHAR, BIGINT, BOOLEAN) .row("a", "", "", "", "timestamp(3)", 8, true) .row("b", "", "", "", "timestamp(3) with time zone", 8, true) + .row("c", "", "", "", "time(3)", 8, true) .build(); assertEqualsIgnoreOrder(actual, expected); }