Skip to content

Commit

Permalink
Use TypeId as an opaque identifier for types
Browse files Browse the repository at this point in the history
TypeSignature is a bad proxy for and identifier that's used in many
inconsistent ways. TypeId is meant to capture the use case where
we need an opaque identifier we can store permanently (e.g., in View
definitions) or for serialization/deserialization of Type.
  • Loading branch information
martint committed Oct 11, 2019
1 parent 45184ba commit a157430
Show file tree
Hide file tree
Showing 34 changed files with 193 additions and 103 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import io.prestosql.plugin.accumulo.metadata.ZooKeeperMetadataManager;
import io.prestosql.spi.PrestoException;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;
import io.prestosql.spi.type.TypeManager;
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
Expand All @@ -49,7 +50,6 @@
import static io.airlift.json.JsonBinder.jsonBinder;
import static io.airlift.json.JsonCodecBinder.jsonCodecBinder;
import static io.prestosql.plugin.accumulo.AccumuloErrorCode.UNEXPECTED_ACCUMULO_ERROR;
import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;

Expand Down Expand Up @@ -117,7 +117,7 @@ public TypeDeserializer(TypeManager typeManager)
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
return typeManager.getType(parseTypeSignature(value));
return typeManager.getType(TypeId.of(value));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import io.prestosql.decoder.DecoderModule;
import io.prestosql.elasticsearch.client.ElasticsearchClient;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;
import io.prestosql.spi.type.TypeManager;

import javax.inject.Inject;
Expand All @@ -30,7 +31,6 @@
import static io.airlift.configuration.ConfigBinder.configBinder;
import static io.airlift.json.JsonBinder.jsonBinder;
import static io.prestosql.elasticsearch.ElasticsearchConfig.Security.AWS;
import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static java.util.Objects.requireNonNull;
import static java.util.function.Predicate.isEqual;

Expand Down Expand Up @@ -80,7 +80,7 @@ public TypeDeserializer(TypeManager typeManager)
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
return typeManager.getType(parseTypeSignature(value));
return typeManager.getType(TypeId.of(value));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import com.google.inject.Module;
import com.google.inject.Scopes;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;
import io.prestosql.spi.type.TypeManager;

import javax.inject.Inject;
Expand All @@ -27,7 +28,6 @@
import static io.airlift.json.JsonBinder.jsonBinder;
import static io.airlift.json.JsonCodec.listJsonCodec;
import static io.airlift.json.JsonCodecBinder.jsonCodecBinder;
import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static java.util.Objects.requireNonNull;

public class ExampleModule
Expand Down Expand Up @@ -71,7 +71,7 @@ public TypeDeserializer(TypeManager typeManager)
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
return typeManager.getType(parseTypeSignature(value));
return typeManager.getType(TypeId.of(value));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import com.google.inject.Module;
import com.google.inject.Scopes;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;
import io.prestosql.spi.type.TypeManager;

import javax.inject.Inject;
Expand All @@ -27,7 +28,6 @@
import static io.airlift.json.JsonBinder.jsonBinder;
import static io.airlift.json.JsonCodec.listJsonCodec;
import static io.airlift.json.JsonCodecBinder.jsonCodecBinder;
import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static java.util.Objects.requireNonNull;

public class SheetsModule
Expand Down Expand Up @@ -71,7 +71,7 @@ public TypeDeserializer(TypeManager typeManager)
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
return typeManager.getType(parseTypeSignature(value));
return typeManager.getType(TypeId.of(value));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2968,7 +2968,7 @@ private void doCreateView(SchemaTableName viewName, boolean replace)
viewData,
Optional.empty(),
Optional.empty(),
ImmutableList.of(new ViewColumn("test", BIGINT.getTypeSignature())),
ImmutableList.of(new ViewColumn("test", BIGINT.getTypeId())),
Optional.empty(),
true);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,14 @@
import com.google.inject.Scopes;
import io.prestosql.decoder.DecoderModule;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;
import io.prestosql.spi.type.TypeManager;

import javax.inject.Inject;

import static io.airlift.configuration.ConfigBinder.configBinder;
import static io.airlift.json.JsonBinder.jsonBinder;
import static io.airlift.json.JsonCodecBinder.jsonCodecBinder;
import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static java.util.Objects.requireNonNull;

/**
Expand Down Expand Up @@ -72,7 +72,7 @@ public TypeDeserializer(TypeManager typeManager)
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
return typeManager.getType(parseTypeSignature(value));
return typeManager.getType(TypeId.of(value));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,10 @@
import io.airlift.json.ObjectMapperProvider;
import io.prestosql.metadata.Metadata;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;

import java.util.function.Supplier;

import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;

public final class CodecSupplier<T>
implements Supplier<JsonCodec<T>>
{
Expand Down Expand Up @@ -61,7 +60,7 @@ public TypeDeserializer()
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
return metadata.getType(parseTypeSignature(value));
return metadata.getType(TypeId.of(value));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import io.prestosql.decoder.DecoderModule;
import io.prestosql.plugin.kinesis.s3config.S3TableConfigClient;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;
import io.prestosql.spi.type.TypeManager;

import javax.inject.Inject;
Expand All @@ -30,7 +31,6 @@
import static io.airlift.configuration.ConfigBinder.configBinder;
import static io.airlift.json.JsonBinder.jsonBinder;
import static io.airlift.json.JsonCodecBinder.jsonCodecBinder;
import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static java.util.Objects.requireNonNull;

public class KinesisModule
Expand Down Expand Up @@ -81,7 +81,7 @@ public TypeDeserializer(TypeManager typeManager)
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
Type type = typeManager.getType(parseTypeSignature(value));
Type type = typeManager.getType(TypeId.of(value));
checkArgument(type != null, "Unknown type %s", value);
return type;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@
import io.airlift.json.JsonCodec;
import io.airlift.json.JsonCodecFactory;
import io.airlift.json.ObjectMapperProvider;
import io.prestosql.spi.type.StandardTypes;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;

import java.util.Map;

Expand All @@ -29,8 +29,8 @@
import static io.prestosql.spi.type.DateType.DATE;
import static io.prestosql.spi.type.DoubleType.DOUBLE;
import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
import static io.prestosql.spi.type.VarcharType.VARCHAR;
import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType;
import static java.util.Locale.ENGLISH;

final class MetadataUtil
{
Expand All @@ -52,13 +52,13 @@ private MetadataUtil()
public static final class TestingTypeDeserializer
extends FromStringDeserializer<Type>
{
private final Map<String, Type> types = new ImmutableMap.Builder<String, Type>()
.put(StandardTypes.BOOLEAN, BOOLEAN)
.put(StandardTypes.BIGINT, BIGINT)
.put(StandardTypes.DOUBLE, DOUBLE)
.put(StandardTypes.TIMESTAMP, TIMESTAMP)
.put(StandardTypes.DATE, DATE)
.put(StandardTypes.VARCHAR, createUnboundedVarcharType())
private final Map<TypeId, Type> types = new ImmutableMap.Builder<TypeId, Type>()
.put(BOOLEAN.getTypeId(), BOOLEAN)
.put(BIGINT.getTypeId(), BIGINT)
.put(DOUBLE.getTypeId(), DOUBLE)
.put(TIMESTAMP.getTypeId(), TIMESTAMP)
.put(DATE.getTypeId(), DATE)
.put(VARCHAR.getTypeId(), createUnboundedVarcharType())
.build();

public TestingTypeDeserializer()
Expand All @@ -69,9 +69,9 @@ public TestingTypeDeserializer()
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
Type type = types.get(value.toLowerCase(ENGLISH));
Type type = types.get(TypeId.of(value));
if (type == null) {
throw new IllegalArgumentException(String.valueOf("Unknown type " + value));
throw new IllegalArgumentException("Unknown type " + value);
}
return type;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public ListenableFuture<?> execute(CreateView statement, TransactionManager tran

List<ViewColumn> columns = analysis.getOutputDescriptor(statement.getQuery())
.getVisibleFields().stream()
.map(field -> new ViewColumn(field.getName().get(), field.getType().getTypeSignature()))
.map(field -> new ViewColumn(field.getName().get(), field.getType().getTypeId()))
.collect(toImmutableList());

// use DEFINER security by default
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,7 @@
import io.prestosql.spi.PrestoException;
import io.prestosql.spi.function.OperatorType;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;
import io.prestosql.spi.type.TypeSignature;
import io.prestosql.sql.DynamicFilters;
import io.prestosql.sql.analyzer.FeaturesConfig;
Expand Down Expand Up @@ -297,7 +298,6 @@
import static io.prestosql.spi.StandardErrorCode.AMBIGUOUS_FUNCTION_CALL;
import static io.prestosql.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_MISSING;
import static io.prestosql.spi.StandardErrorCode.FUNCTION_NOT_FOUND;
import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static io.prestosql.sql.analyzer.TypeSignatureProvider.fromTypeSignatures;
import static io.prestosql.sql.analyzer.TypeSignatureProvider.fromTypes;
import static io.prestosql.type.DecimalCasts.BIGINT_TO_DECIMAL_CAST;
Expand Down Expand Up @@ -720,7 +720,7 @@ Signature resolveFunction(QualifiedName name, List<TypeSignatureProvider> parame
String typeName = name.getSuffix().substring(LITERAL_FUNCTION_NAME.length());

// lookup the type
Type type = metadata.getType(parseTypeSignature(typeName));
Type type = metadata.getType(TypeId.of(typeName));

// verify we have one parameter of the proper type
checkArgument(parameterTypes.size() == 1, "Expected one argument to literal function, but got %s", parameterTypes);
Expand Down Expand Up @@ -1018,7 +1018,7 @@ private SpecializedFunctionKey doGetSpecializedFunctionKey(Signature signature)
String typeName = signature.getName().substring(LITERAL_FUNCTION_NAME.length());

// lookup the type
Type type = metadata.getType(parseTypeSignature(typeName));
Type type = metadata.getType(TypeId.of(typeName));

// verify we have one parameter of the proper type
checkArgument(parameterTypes.size() == 1, "Expected one argument to literal function, but got %s", parameterTypes);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ public static Signature getLiteralFunctionSignature(Type type)
TypeSignature argumentType = typeForLiteralFunctionArgument(type).getTypeSignature();

return new Signature(
LITERAL_FUNCTION_NAME + type.getTypeSignature(),
LITERAL_FUNCTION_NAME + type.getTypeId().getId(),
SCALAR,
type.getTypeSignature(),
argumentType);
Expand Down
6 changes: 6 additions & 0 deletions presto-main/src/main/java/io/prestosql/metadata/Metadata.java
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import io.prestosql.spi.statistics.TableStatisticsMetadata;
import io.prestosql.spi.type.ParametricType;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;
import io.prestosql.spi.type.TypeSignature;
import io.prestosql.spi.type.TypeSignatureParameter;
import io.prestosql.sql.analyzer.TypeSignatureProvider;
Expand Down Expand Up @@ -401,6 +402,11 @@ public interface Metadata

Type getType(TypeSignature signature);

default Type getType(TypeId id)
{
return getType(TypeSignature.parseTypeSignature(id.getId()));
}

default Type getParameterizedType(String baseTypeName, List<TypeSignatureParameter> typeParameters)
{
return getType(new TypeSignature(baseTypeName, typeParameters));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@
import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer;
import io.prestosql.metadata.Metadata;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;

import javax.inject.Inject;

import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static java.util.Objects.requireNonNull;

public final class TypeDeserializer
Expand All @@ -38,6 +38,6 @@ public TypeDeserializer(Metadata metadata)
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
return metadata.getType(parseTypeSignature(value));
return metadata.getType(TypeId.of(value));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ public TestInformationSchemaMetadata()
"select 1",
Optional.of("test_catalog"),
Optional.of("test_schema"),
ImmutableList.of(new ViewColumn("test", BIGINT.getTypeSignature())),
ImmutableList.of(new ViewColumn("test", BIGINT.getTypeId())),
Optional.empty(),
true);
SchemaTableName viewName = new SchemaTableName("test_schema", "test_view");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,6 @@
import static io.prestosql.spi.session.PropertyMetadata.stringProperty;
import static io.prestosql.spi.type.BigintType.BIGINT;
import static io.prestosql.spi.type.DoubleType.DOUBLE;
import static io.prestosql.spi.type.TypeSignature.parseTypeSignature;
import static io.prestosql.spi.type.VarcharType.VARCHAR;
import static io.prestosql.testing.TestingSession.testSessionBuilder;
import static io.prestosql.testing.assertions.PrestoExceptionAssert.assertPrestoExceptionThrownBy;
Expand Down Expand Up @@ -1951,7 +1950,7 @@ public void setup()
"select a from t1",
Optional.of(TPCH_CATALOG),
Optional.of("s1"),
ImmutableList.of(new ViewColumn("a", BIGINT.getTypeSignature())),
ImmutableList.of(new ViewColumn("a", BIGINT.getTypeId())),
Optional.of("user"),
false);
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName(TPCH_CATALOG, "s1", "v1"), viewData1, false));
Expand All @@ -1961,7 +1960,7 @@ public void setup()
"select a from t1",
Optional.of(TPCH_CATALOG),
Optional.of("s1"),
ImmutableList.of(new ViewColumn("a", parseTypeSignature("varchar"))),
ImmutableList.of(new ViewColumn("a", VARCHAR.getTypeId())),
Optional.of("user"),
false);
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName(TPCH_CATALOG, "s1", "v2"), viewData2, false));
Expand All @@ -1971,7 +1970,7 @@ public void setup()
"select a from t4",
Optional.of(SECOND_CATALOG),
Optional.of("s2"),
ImmutableList.of(new ViewColumn("a", BIGINT.getTypeSignature())),
ImmutableList.of(new ViewColumn("a", BIGINT.getTypeId())),
Optional.of("owner"),
false);
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName(THIRD_CATALOG, "s3", "v3"), viewData3, false));
Expand All @@ -1981,7 +1980,7 @@ public void setup()
"select A from t1",
Optional.of("tpch"),
Optional.of("s1"),
ImmutableList.of(new ViewColumn("a", BIGINT.getTypeSignature())),
ImmutableList.of(new ViewColumn("a", BIGINT.getTypeId())),
Optional.of("user"),
false);
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName("tpch", "s1", "v4"), viewData4, false));
Expand All @@ -1991,7 +1990,7 @@ public void setup()
"select * from v5",
Optional.of(TPCH_CATALOG),
Optional.of("s1"),
ImmutableList.of(new ViewColumn("a", BIGINT.getTypeSignature())),
ImmutableList.of(new ViewColumn("a", BIGINT.getTypeId())),
Optional.of("user"),
false);
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName(TPCH_CATALOG, "s1", "v5"), viewData5, false));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,7 @@ private static ConnectorViewDefinition testingViewDefinition(String sql)
sql,
Optional.empty(),
Optional.empty(),
ImmutableList.of(new ViewColumn("test", BIGINT.getTypeSignature())),
ImmutableList.of(new ViewColumn("test", BIGINT.getTypeId())),
Optional.empty(),
true);
}
Expand Down
Loading

0 comments on commit a157430

Please sign in to comment.