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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
31 changes: 9 additions & 22 deletions core/trino-main/src/main/java/io/trino/FeaturesConfig.java
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -82,6 +83,14 @@
})
public class FeaturesConfig
{
public enum DataIntegrityVerification
{
NONE,
ABORT,
RETRY,
/**/;
}

@VisibleForTesting
public static final String SPILLER_SPILL_PATH = "spiller-spill-path";

Expand All @@ -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;
Expand Down Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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";
Expand Down Expand Up @@ -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",
Expand Down Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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
Expand Down Expand Up @@ -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());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;

Expand Down Expand Up @@ -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);
Expand All @@ -266,7 +263,7 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect
if ((schemaDomain.isAll() && tableDomain.isAll()) || schemaFilter.isPresent()) {
QualifiedTablePrefix tablePrefix = tablePrefix(catalog, schemaFilter, tableFilter);
Map<SchemaTableName, List<ColumnMetadata>> tableColumns = listTableColumns(session, metadata, accessControl, tablePrefix);
addColumnsRow(table, catalog, tableColumns, omitDateTimeTypePrecision);
addColumnsRow(table, catalog, tableColumns);
}
else {
Collection<String> schemas = listSchemas(session, metadata, accessControl, catalog, schemaFilter);
Expand All @@ -286,22 +283,22 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect
}

Map<SchemaTableName, List<ColumnMetadata>> tableColumns = listTableColumns(session, metadata, accessControl, new QualifiedTablePrefix(catalog, schema, tableName));
addColumnsRow(table, catalog, tableColumns, omitDateTimeTypePrecision);
addColumnsRow(table, catalog, tableColumns);
}
}
}
}
return table.build().cursor();
}

private static void addColumnsRow(Builder builder, String catalog, Map<SchemaTableName, List<ColumnMetadata>> columns, boolean isOmitTimestampPrecision)
private static void addColumnsRow(Builder builder, String catalog, Map<SchemaTableName, List<ColumnMetadata>> columns)
{
for (Entry<SchemaTableName, List<ColumnMetadata>> 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<ColumnMetadata> columns, boolean isOmitTimestampPrecision)
private static void addColumnRows(Builder builder, String catalog, SchemaTableName tableName, List<ColumnMetadata> columns)
{
int ordinalPosition = 1;
for (ColumnMetadata column : columns) {
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;

Expand Down Expand Up @@ -134,7 +132,7 @@ protected Node visitDescribeInput(DescribeInput node, Void context)

ImmutableList.Builder<Row> 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
Expand Down Expand Up @@ -163,7 +161,7 @@ private static Query createDescribeInputQuery(Row[] rows, Optional<Node> 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) {
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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
Expand Down Expand Up @@ -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())));
}
Expand Down
63 changes: 0 additions & 63 deletions core/trino-main/src/main/java/io/trino/type/TypeUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -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 + "(", ")"));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,6 @@ public void testDefaults()
.setFilterAndProjectMinOutputPageRowCount(256)
.setMaxRecursionDepth(10)
.setMaxGroupingSets(2048)
.setOmitDateTimeTypePrecision(false)
.setLegacyCatalogRoles(false)
.setIncrementalHashArrayLoadFactorEnabled(true)
.setHideInaccessibleColumns(false)
Expand Down Expand Up @@ -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")
Expand Down Expand Up @@ -130,7 +128,6 @@ public void testExplicitPropertyMappings()
.setFilterAndProjectMinOutputPageRowCount(2048)
.setMaxRecursionDepth(8)
.setMaxGroupingSets(2047)
.setOmitDateTimeTypePrecision(true)
.setLegacyCatalogRoles(true)
.setIncrementalHashArrayLoadFactorEnabled(false)
.setHideInaccessibleColumns(true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,6 @@ public final class JdbcDriverCapabilities
{
private JdbcDriverCapabilities() {}

public static final int VERSION_HEAD = 0;

public static Optional<Integer> testedVersion()
{
return Optional.ofNullable(System.getenv("TRINO_JDBC_VERSION_UNDER_TEST")).map(Integer::valueOf);
Expand All @@ -50,37 +48,4 @@ private static Optional<Driver> 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;
}
}
Loading