Skip to content

Commit

Permalink
Extract toSqlProperties method to PropertyUtil
Browse files Browse the repository at this point in the history
  • Loading branch information
electrum committed May 9, 2024
1 parent d74ec69 commit ec64be7
Show file tree
Hide file tree
Showing 2 changed files with 77 additions and 98 deletions.
67 changes: 67 additions & 0 deletions core/trino-main/src/main/java/io/trino/metadata/PropertyUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,9 @@
package io.trino.metadata;

import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSortedMap;
import com.google.common.collect.Maps;
import com.google.common.primitives.Primitives;
import io.trino.Session;
import io.trino.security.AccessControl;
import io.trino.spi.ErrorCodeSupplier;
Expand All @@ -23,18 +26,27 @@
import io.trino.spi.type.Type;
import io.trino.sql.PlannerContext;
import io.trino.sql.planner.ParameterRewriter;
import io.trino.sql.tree.Array;
import io.trino.sql.tree.BooleanLiteral;
import io.trino.sql.tree.DoubleLiteral;
import io.trino.sql.tree.Expression;
import io.trino.sql.tree.ExpressionTreeRewriter;
import io.trino.sql.tree.Identifier;
import io.trino.sql.tree.LongLiteral;
import io.trino.sql.tree.NodeRef;
import io.trino.sql.tree.Parameter;
import io.trino.sql.tree.Property;
import io.trino.sql.tree.StringLiteral;

import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.spi.type.TypeUtils.writeNativeValue;
import static io.trino.sql.analyzer.ConstantEvaluator.evaluateConstant;
import static io.trino.util.MoreLists.mappedCopy;
import static java.lang.String.format;
import static java.util.Locale.ENGLISH;

Expand Down Expand Up @@ -174,6 +186,61 @@ public static Object evaluateProperty(
return sqlObjectValue;
}

public static List<Property> toSqlProperties(
Object description,
ErrorCodeSupplier errorCode,
Map<String, Object> properties,
Iterable<PropertyMetadata<?>> metadata)
{
if (properties.isEmpty()) {
return List.of();
}

Map<String, PropertyMetadata<?>> indexedMetadata = Maps.uniqueIndex(metadata, PropertyMetadata::getName);
ImmutableSortedMap.Builder<String, Expression> sqlProperties = ImmutableSortedMap.naturalOrder();

properties.forEach((name, value) -> {
if (value == null) {
throw new TrinoException(errorCode, "Property %s for %s cannot have a null value".formatted(name, description));
}

PropertyMetadata<?> property = indexedMetadata.get(name);
if (property == null) {
throw new TrinoException(errorCode, "No PropertyMetadata for property: " + name);
}
if (!Primitives.wrap(property.getJavaType()).isInstance(value)) {
throw new TrinoException(errorCode, "Property %s for %s should have value of type %s, not %s"
.formatted(name, description, property.getJavaType().getName(), value.getClass().getName()));
}

sqlProperties.put(name, toExpression(errorCode, property, value));
});

return sqlProperties.build().entrySet().stream()
.map(entry -> new Property(new Identifier(entry.getKey()), entry.getValue()))
.collect(toImmutableList());
}

private static <T> Expression toExpression(ErrorCodeSupplier errorcode, PropertyMetadata<T> property, Object value)
throws TrinoException
{
return toExpression(errorcode, property.encode(property.getJavaType().cast(value)));
}

private static Expression toExpression(ErrorCodeSupplier errorCode, Object value)
throws TrinoException
{
return switch (value) {
case String _ -> new StringLiteral(value.toString());
case Boolean _ -> new BooleanLiteral(value.toString());
case Long _, Integer _ -> new LongLiteral(value.toString());
case Double _ -> new DoubleLiteral(value.toString());
case List<?> list -> new Array(mappedCopy(list, item -> toExpression(errorCode, item)));
case null -> throw new TrinoException(errorCode, "Property value is null");
default -> throw new TrinoException(errorCode, "Failed to convert object of type %s to expression".formatted(value.getClass().getName()));
};
}

private static String capitalize(String value)
{
return Character.toUpperCase(value.charAt(0)) + value.substring(1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,8 @@
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSortedMap;
import com.google.common.collect.Maps;
import com.google.common.collect.Multimap;
import com.google.common.collect.Multimaps;
import com.google.common.primitives.Primitives;
import com.google.inject.Inject;
import io.trino.Session;
import io.trino.execution.querystats.PlanOptimizersStatsCollector;
Expand All @@ -41,8 +38,6 @@
import io.trino.metadata.ViewDefinition;
import io.trino.metadata.ViewPropertyManager;
import io.trino.security.AccessControl;
import io.trino.spi.StandardErrorCode;
import io.trino.spi.TrinoException;
import io.trino.spi.connector.CatalogHandle;
import io.trino.spi.connector.CatalogSchemaName;
import io.trino.spi.connector.ConnectorTableMetadata;
Expand All @@ -61,7 +56,6 @@
import io.trino.sql.parser.ParsingException;
import io.trino.sql.parser.SqlParser;
import io.trino.sql.tree.AllColumns;
import io.trino.sql.tree.Array;
import io.trino.sql.tree.AstVisitor;
import io.trino.sql.tree.BooleanLiteral;
import io.trino.sql.tree.Cast;
Expand All @@ -70,14 +64,12 @@
import io.trino.sql.tree.CreateSchema;
import io.trino.sql.tree.CreateTable;
import io.trino.sql.tree.CreateView;
import io.trino.sql.tree.DoubleLiteral;
import io.trino.sql.tree.Explain;
import io.trino.sql.tree.ExplainAnalyze;
import io.trino.sql.tree.Expression;
import io.trino.sql.tree.GrantObject;
import io.trino.sql.tree.Identifier;
import io.trino.sql.tree.LikePredicate;
import io.trino.sql.tree.LongLiteral;
import io.trino.sql.tree.Node;
import io.trino.sql.tree.NodeRef;
import io.trino.sql.tree.NullLiteral;
Expand Down Expand Up @@ -108,7 +100,6 @@
import io.trino.sql.tree.Values;

import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
Expand All @@ -129,6 +120,7 @@
import static io.trino.metadata.MetadataUtil.createQualifiedObjectName;
import static io.trino.metadata.MetadataUtil.getRequiredCatalogHandle;
import static io.trino.metadata.MetadataUtil.processRoleCommandCatalog;
import static io.trino.metadata.PropertyUtil.toSqlProperties;
import static io.trino.spi.StandardErrorCode.CATALOG_NOT_FOUND;
import static io.trino.spi.StandardErrorCode.INVALID_COLUMN_PROPERTY;
import static io.trino.spi.StandardErrorCode.INVALID_MATERIALIZED_VIEW_PROPERTY;
Expand Down Expand Up @@ -168,7 +160,6 @@
import static io.trino.sql.tree.CreateView.Security.INVOKER;
import static io.trino.sql.tree.LogicalExpression.and;
import static io.trino.sql.tree.SaveMode.FAIL;
import static java.lang.String.format;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.toList;
Expand Down Expand Up @@ -519,40 +510,6 @@ protected Node visitShowColumns(ShowColumns showColumns, Void context)
ordering(ascending("ordinal_position")));
}

private static <T> Expression getExpression(PropertyMetadata<T> property, Object value)
throws TrinoException
{
return toExpression(property.encode(property.getJavaType().cast(value)));
}

private static Expression toExpression(Object value)
throws TrinoException
{
if (value instanceof String) {
return new StringLiteral(value.toString());
}

if (value instanceof Boolean) {
return new BooleanLiteral(value.toString());
}

if (value instanceof Long || value instanceof Integer) {
return new LongLiteral(value.toString());
}

if (value instanceof Double) {
return new DoubleLiteral(value.toString());
}

if (value instanceof List<?> list) {
return new Array(list.stream()
.map(Visitor::toExpression)
.collect(toList()));
}

throw new TrinoException(INVALID_TABLE_PROPERTY, format("Failed to convert object of type %s to expression: %s", value.getClass().getName(), value));
}

@Override
protected Node visitShowCreate(ShowCreate node, Void context)
{
Expand Down Expand Up @@ -593,7 +550,7 @@ private Query showCreateMaterializedView(ShowCreate node)
Map<String, Object> properties = metadata.getMaterializedViewProperties(session, objectName, viewDefinition.get());
CatalogHandle catalogHandle = getRequiredCatalogHandle(metadata, session, node, catalogName.getValue());
Collection<PropertyMetadata<?>> allMaterializedViewProperties = materializedViewPropertyManager.getAllProperties(catalogHandle);
List<Property> propertyNodes = buildProperties(objectName, Optional.empty(), INVALID_MATERIALIZED_VIEW_PROPERTY, properties, allMaterializedViewProperties);
List<Property> propertyNodes = toSqlProperties("materialized view " + objectName, INVALID_MATERIALIZED_VIEW_PROPERTY, properties, allMaterializedViewProperties);

String sql = formatSql(new CreateMaterializedView(
Optional.empty(),
Expand Down Expand Up @@ -635,7 +592,7 @@ private Query showCreateView(ShowCreate node)
Map<String, Object> properties = metadata.getViewProperties(session, objectName);
CatalogHandle catalogHandle = getRequiredCatalogHandle(metadata, session, node, catalogName.getValue());
Collection<PropertyMetadata<?>> allViewProperties = viewPropertyManager.getAllProperties(catalogHandle);
List<Property> propertyNodes = buildProperties(objectName, Optional.empty(), INVALID_VIEW_PROPERTY, properties, allViewProperties);
List<Property> propertyNodes = toSqlProperties("view " + objectName, INVALID_VIEW_PROPERTY, properties, allViewProperties);
CreateView.Security security = viewDefinition.get().isRunAsInvoker() ? INVOKER : DEFINER;
String sql = formatSql(new CreateView(
QualifiedName.of(ImmutableList.of(catalogName, schemaName, tableName)),
Expand Down Expand Up @@ -673,7 +630,11 @@ private Query showCreateTable(ShowCreate node)
List<TableElement> columns = connectorTableMetadata.getColumns().stream()
.filter(column -> !column.isHidden())
.map(column -> {
List<Property> propertyNodes = buildProperties(targetTableName, Optional.of(column.getName()), INVALID_COLUMN_PROPERTY, column.getProperties(), allColumnProperties);
List<Property> propertyNodes = toSqlProperties(
"column %s of table %s".formatted(column.getName(), objectName),
INVALID_COLUMN_PROPERTY,
column.getProperties(),
allColumnProperties);
return new ColumnDefinition(
QualifiedName.of(column.getName()),
toSqlType(column.getType()),
Expand All @@ -685,7 +646,7 @@ private Query showCreateTable(ShowCreate node)

Map<String, Object> properties = connectorTableMetadata.getProperties();
Collection<PropertyMetadata<?>> allTableProperties = tablePropertyManager.getAllProperties(tableHandle.catalogHandle());
List<Property> propertyNodes = buildProperties(targetTableName, Optional.empty(), INVALID_TABLE_PROPERTY, properties, allTableProperties);
List<Property> propertyNodes = toSqlProperties("table " + targetTableName, INVALID_TABLE_PROPERTY, properties, allTableProperties);

CreateTable createTable = new CreateTable(
QualifiedName.of(targetTableName.catalogName(), targetTableName.schemaName(), targetTableName.objectName()),
Expand All @@ -710,7 +671,7 @@ private Query showCreateSchema(ShowCreate node)
CatalogHandle catalogHandle = getRequiredCatalogHandle(metadata, session, node, schemaName.getCatalogName());
Collection<PropertyMetadata<?>> allTableProperties = schemaPropertyManager.getAllProperties(catalogHandle);
QualifiedName qualifiedSchemaName = QualifiedName.of(schemaName.getCatalogName(), schemaName.getSchemaName());
List<Property> propertyNodes = buildProperties(qualifiedSchemaName, Optional.empty(), INVALID_SCHEMA_PROPERTY, properties, allTableProperties);
List<Property> propertyNodes = toSqlProperties("schema " + qualifiedSchemaName, INVALID_SCHEMA_PROPERTY, properties, allTableProperties);

Optional<PrincipalSpecification> owner = metadata.getSchemaOwner(session, schemaName).map(MetadataUtil::createPrincipal);

Expand Down Expand Up @@ -742,55 +703,6 @@ private Node showCreateFunction(ShowCreate node)
aliased(new Values(rows), "t", ImmutableList.of("Create Function")));
}

private static List<Property> buildProperties(
Object objectName,
Optional<String> columnName,
StandardErrorCode errorCode,
Map<String, Object> properties,
Collection<PropertyMetadata<?>> allProperties)
{
if (properties.isEmpty()) {
return Collections.emptyList();
}

Map<String, PropertyMetadata<?>> indexedPropertyMetadata = Maps.uniqueIndex(allProperties, PropertyMetadata::getName);
ImmutableSortedMap.Builder<String, Expression> sqlProperties = ImmutableSortedMap.naturalOrder();

for (Map.Entry<String, Object> propertyEntry : properties.entrySet()) {
String propertyName = propertyEntry.getKey();
Object value = propertyEntry.getValue();
if (value == null) {
throw new TrinoException(errorCode, format("Property %s for %s cannot have a null value", propertyName, toQualifiedName(objectName, columnName)));
}

PropertyMetadata<?> property = indexedPropertyMetadata.get(propertyName);
if (property == null) {
throw new TrinoException(errorCode, "No PropertyMetadata for property: " + propertyName);
}
if (!Primitives.wrap(property.getJavaType()).isInstance(value)) {
throw new TrinoException(errorCode, format(
"Property %s for %s should have value of type %s, not %s",
propertyName,
toQualifiedName(objectName, columnName),
property.getJavaType().getName(),
value.getClass().getName()));
}

Expression sqlExpression = getExpression(property, value);
sqlProperties.put(propertyName, sqlExpression);
}

return sqlProperties.build().entrySet().stream()
.map(entry -> new Property(new Identifier(entry.getKey()), entry.getValue()))
.collect(toImmutableList());
}

private static String toQualifiedName(Object objectName, Optional<String> columnName)
{
return columnName.map(s -> format("column %s of table %s", s, objectName))
.orElseGet(() -> "table " + objectName);
}

@Override
protected Node visitShowFunctions(ShowFunctions node, Void context)
{
Expand Down

0 comments on commit ec64be7

Please sign in to comment.