public static ExpressionAnalyzer createConstantAnalyzer(Metadata metadata, Session session) { return createWithoutSubqueries( metadata.getFunctionRegistry(), metadata.getTypeManager(), session, EXPRESSION_NOT_CONSTANT, "Constant expression cannot contain a subquery"); }
private RowExpression toRowExpression( Expression projection, IdentityHashMap<Expression, Type> expressionTypes) { return SqlToRowExpressionTranslator.translate( projection, SCALAR, expressionTypes, metadata.getFunctionRegistry(), metadata.getTypeManager(), session, false); }
private InternalTable getInternalTable( ConnectorTransactionHandle transactionHandle, ConnectorSession connectorSession, ConnectorSplit connectorSplit, List<ColumnHandle> columns) { InformationSchemaTransactionHandle transaction = checkType(transactionHandle, InformationSchemaTransactionHandle.class, "transaction"); InformationSchemaSplit split = checkType(connectorSplit, InformationSchemaSplit.class, "split"); requireNonNull(columns, "columns is null"); InformationSchemaTableHandle handle = split.getTableHandle(); Map<String, NullableValue> filters = split.getFilters(); Session session = Session.builder(metadata.getSessionPropertyManager()) .setTransactionId(transaction.getTransactionId()) .setQueryId(new QueryId(connectorSession.getQueryId())) .setIdentity(connectorSession.getIdentity()) .setSource("information_schema") .setCatalog("") // default catalog is not be used .setSchema("") // default schema is not be used .setTimeZoneKey(connectorSession.getTimeZoneKey()) .setLocale(connectorSession.getLocale()) .setStartTime(connectorSession.getStartTime()) .build(); return getInformationSchemaTable( session, handle.getCatalogName(), handle.getSchemaTableName(), filters); }
@Override public void execute( DropTable statement, Session session, Metadata metadata, QueryStateMachine stateMachine) { QualifiedTableName tableName = createQualifiedTableName(session, statement.getTableName()); Optional<TableHandle> tableHandle = metadata.getTableHandle(session, tableName); if (!tableHandle.isPresent()) { if (!statement.isExists()) { throw new SemanticException( MISSING_TABLE, statement, "Table '%s' does not exist", tableName); } return; } metadata.dropTable(tableHandle.get()); }
private InternalTable buildSchemata(Session session, String catalogName) { InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_SCHEMATA)); for (String schema : metadata.listSchemaNames(session, catalogName)) { table.add(catalogName, schema); } return table.build(); }
private ByteCodeNode compileExpression( CallSiteBinder callSiteBinder, RowExpression expression, boolean sourceIsCursor, CompilerContext context, Block getSessionByteCode) { ByteCodeExpressionVisitor visitor = new ByteCodeExpressionVisitor( callSiteBinder, getSessionByteCode, metadata.getFunctionRegistry(), sourceIsCursor); return expression.accept(visitor, context); }
public static ExpressionAnalyzer create( Analysis analysis, Session session, Metadata metadata, SqlParser sqlParser, AccessControl accessControl, boolean experimentalSyntaxEnabled) { return new ExpressionAnalyzer( metadata.getFunctionRegistry(), metadata.getTypeManager(), node -> new StatementAnalyzer( analysis, metadata, sqlParser, accessControl, session, experimentalSyntaxEnabled, Optional.empty()), session); }
@Override public void execute( CreateTable statement, Session session, Metadata metadata, QueryStateMachine stateMachine) { checkArgument(!statement.getElements().isEmpty(), "no columns for table"); QualifiedTableName tableName = createQualifiedTableName(session, statement.getName()); Optional<TableHandle> tableHandle = metadata.getTableHandle(session, tableName); if (tableHandle.isPresent()) { if (!statement.isNotExists()) { throw new SemanticException( TABLE_ALREADY_EXISTS, statement, "Table '%s' already exists", tableName); } return; } List<ColumnMetadata> columns = new ArrayList<>(); for (TableElement element : statement.getElements()) { Type type = metadata.getType(parseTypeSignature(element.getType())); if ((type == null) || type.equals(UNKNOWN)) { throw new SemanticException( TYPE_MISMATCH, element, "Unknown type for column '%s' ", element.getName()); } columns.add(new ColumnMetadata(element.getName(), type, false)); } Map<String, Object> properties = metadata .getTablePropertyManager() .getTableProperties( tableName.getCatalogName(), statement.getProperties(), session, metadata); TableMetadata tableMetadata = new TableMetadata( tableName.getCatalogName(), new ConnectorTableMetadata( tableName.asSchemaTableName(), columns, properties, session.getUser(), false)); metadata.createTable(session, tableName.getCatalogName(), tableMetadata); }
@Override public ActualProperties visitTableScan( TableScanNode node, List<ActualProperties> inputProperties) { checkArgument(node.getLayout().isPresent(), "table layout has not yet been chosen"); TableLayout layout = metadata.getLayout(session, node.getLayout().get()); Map<ColumnHandle, Symbol> assignments = ImmutableBiMap.copyOf(node.getAssignments()).inverse(); ActualProperties.Builder properties = ActualProperties.builder(); // Globally constant assignments Map<ColumnHandle, NullableValue> globalConstants = new HashMap<>(); extractFixedValues(node.getCurrentConstraint()) .orElse(ImmutableMap.of()) .entrySet() .stream() .filter(entry -> !entry.getValue().isNull()) .forEach(entry -> globalConstants.put(entry.getKey(), entry.getValue())); Map<Symbol, NullableValue> symbolConstants = globalConstants .entrySet() .stream() .filter(entry -> assignments.containsKey(entry.getKey())) .collect(toMap(entry -> assignments.get(entry.getKey()), Map.Entry::getValue)); properties.constants(symbolConstants); // Partitioning properties properties.global(deriveGlobalProperties(layout, assignments, globalConstants)); // Append the global constants onto the local properties to maximize their translation // potential List<LocalProperty<ColumnHandle>> constantAppendedLocalProperties = ImmutableList.<LocalProperty<ColumnHandle>>builder() .addAll( globalConstants .keySet() .stream() .map(column -> new ConstantProperty<>(column)) .iterator()) .addAll(layout.getLocalProperties()) .build(); properties.local( LocalProperties.translate( constantAppendedLocalProperties, column -> Optional.ofNullable(assignments.get(column)))); return properties.build(); }
@Override public PlanNode visitWindow(WindowNode node, RewriteContext<Context> context) { if (!node.getWindowFunctions() .values() .stream() .map(function -> function.getFunctionCall().getName()) .allMatch(metadata.getFunctionRegistry()::isAggregationFunction)) { return node; } // Don't need this restriction if we can prove that all order by symbols are deterministically // produced if (!node.getOrderBy().isEmpty()) { return node; } // Only RANGE frame type currently supported for aggregation functions because it guarantees // the // same value for each peer group. // ROWS frame type requires the ordering to be fully deterministic (e.g. deterministically // sorted on all columns) if (node.getFrames() .stream() .map(WindowNode.Frame::getType) .anyMatch( type -> type != WindowFrame.Type .RANGE)) { // TODO: extract frames of type RANGE and allow optimization on // them return node; } // Lookup symbols can only be passed through if they are part of the partitioning Set<Symbol> partitionByLookupSymbols = context .get() .getLookupSymbols() .stream() .filter(node.getPartitionBy()::contains) .collect(toImmutableSet()); if (partitionByLookupSymbols.isEmpty()) { return node; } return context.defaultRewrite( node, new Context(partitionByLookupSymbols, context.get().getSuccess())); }
@Override protected Object visitGenericLiteral(GenericLiteral node, ConnectorSession session) { Type type = metadata.getType(parseTypeSignature(node.getType())); if (type == null) { throw new SemanticException(TYPE_MISMATCH, node, "Unknown type: " + node.getType()); } if (JSON.equals(type)) { ScalarFunctionImplementation operator = metadata .getFunctionRegistry() .getScalarFunctionImplementation( new Signature( "json_parse", SCALAR, JSON.getTypeSignature(), VARCHAR.getTypeSignature())); try { return ExpressionInterpreter.invoke( session, operator, ImmutableList.<Object>of(utf8Slice(node.getValue()))); } catch (Throwable throwable) { throw Throwables.propagate(throwable); } } ScalarFunctionImplementation operator; try { Signature signature = metadata.getFunctionRegistry().getCoercion(VARCHAR, type); operator = metadata.getFunctionRegistry().getScalarFunctionImplementation(signature); } catch (IllegalArgumentException e) { throw new SemanticException(TYPE_MISMATCH, node, "No literal form for type %s", type); } try { return ExpressionInterpreter.invoke( session, operator, ImmutableList.<Object>of(utf8Slice(node.getValue()))); } catch (Throwable throwable) { throw Throwables.propagate(throwable); } }
@Override protected RelationPlan visitTable(Table node, Void context) { Query namedQuery = analysis.getNamedQuery(node); if (namedQuery != null) { RelationPlan subPlan = process(namedQuery, null); return new RelationPlan( subPlan.getRoot(), analysis.getOutputDescriptor(node), subPlan.getOutputSymbols(), subPlan.getSampleWeight()); } TupleDescriptor descriptor = analysis.getOutputDescriptor(node); TableHandle handle = analysis.getTableHandle(node); ImmutableList.Builder<Symbol> outputSymbolsBuilder = ImmutableList.builder(); ImmutableMap.Builder<Symbol, ColumnHandle> columns = ImmutableMap.builder(); for (Field field : descriptor.getAllFields()) { Symbol symbol = symbolAllocator.newSymbol(field.getName().get(), field.getType()); outputSymbolsBuilder.add(symbol); columns.put(symbol, analysis.getColumn(field)); } List<Symbol> planOutputSymbols = outputSymbolsBuilder.build(); Optional<ColumnHandle> sampleWeightColumn = metadata.getSampleWeightColumnHandle(session, handle); Symbol sampleWeightSymbol = null; if (sampleWeightColumn.isPresent()) { sampleWeightSymbol = symbolAllocator.newSymbol("$sampleWeight", BIGINT); outputSymbolsBuilder.add(sampleWeightSymbol); columns.put(sampleWeightSymbol, sampleWeightColumn.get()); } List<Symbol> nodeOutputSymbols = outputSymbolsBuilder.build(); PlanNode root = new TableScanNode( idAllocator.getNextId(), handle, nodeOutputSymbols, columns.build(), Optional.empty(), TupleDomain.all(), null); return new RelationPlan( root, descriptor, planOutputSymbols, Optional.ofNullable(sampleWeightSymbol)); }
private PlanNode planTableScan(TableScanNode node, Expression predicate) { DomainTranslator.ExtractionResult decomposedPredicate = DomainTranslator.fromPredicate(metadata, session, predicate, symbolAllocator.getTypes()); TupleDomain<ColumnHandle> simplifiedConstraint = decomposedPredicate .getTupleDomain() .transform(node.getAssignments()::get) .intersect(node.getCurrentConstraint()); List<TableLayoutResult> layouts = metadata.getLayouts( session, node.getTable(), new Constraint<>(simplifiedConstraint, bindings -> true), Optional.of(ImmutableSet.copyOf(node.getAssignments().values()))); if (layouts.isEmpty()) { return new ValuesNode(idAllocator.getNextId(), node.getOutputSymbols(), ImmutableList.of()); } TableLayoutResult layout = layouts.get(0); TableScanNode result = new TableScanNode( node.getId(), node.getTable(), node.getOutputSymbols(), node.getAssignments(), Optional.of(layout.getLayout().getHandle()), simplifiedConstraint.intersect(layout.getLayout().getPredicate()), Optional.ofNullable(node.getOriginalConstraint()).orElse(predicate)); Map<ColumnHandle, Symbol> assignments = ImmutableBiMap.copyOf(node.getAssignments()).inverse(); Expression resultingPredicate = combineConjuncts( decomposedPredicate.getRemainingExpression(), DomainTranslator.toPredicate( layout.getUnenforcedConstraint().transform(assignments::get))); if (!BooleanLiteral.TRUE_LITERAL.equals(resultingPredicate)) { return new FilterNode(idAllocator.getNextId(), result, resultingPredicate); } return result; }
private InternalTable buildFunctions() { InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_INTERNAL_FUNCTIONS)); for (ParametricFunction function : metadata.listFunctions()) { if (function.isApproximate()) { continue; } table.add( function.getSignature().getName(), Joiner.on(", ").join(function.getSignature().getArgumentTypes()), function.getSignature().getReturnType().toString(), getFunctionType(function), function.isDeterministic(), nullToEmpty(function.getDescription())); } return table.build(); }
private MethodDefinition generateProjectMethod( ClassDefinition classDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, String methodName, RowExpression projection) { Parameter session = arg("session", ConnectorSession.class); List<Parameter> inputs = toBlockParameters(getInputChannels(projection)); Parameter position = arg("position", int.class); Parameter output = arg("output", BlockBuilder.class); MethodDefinition method = classDefinition.declareMethod( a(PUBLIC), methodName, type(void.class), ImmutableList.<Parameter>builder() .add(session) .addAll(inputs) .add(position) .add(output) .build()); method.comment("Projection: %s", projection.toString()); Scope scope = method.getScope(); BytecodeBlock body = method.getBody(); Variable wasNullVariable = scope.declareVariable("wasNull", body, constantFalse()); BytecodeExpressionVisitor visitor = new BytecodeExpressionVisitor( callSiteBinder, cachedInstanceBinder, fieldReferenceCompiler(callSiteBinder, position, wasNullVariable), metadata.getFunctionRegistry()); body.getVariable(output) .comment("evaluate projection: " + projection.toString()) .append(projection.accept(visitor, scope)) .append(generateWrite(callSiteBinder, scope, wasNullVariable, projection.getType())) .ret(); return method; }
@Override public PlanNode visitTableCommit(TableCommitNode node, RewriteContext<Void> context) { Optional<DeleteNode> delete = findNode(node.getSource(), DeleteNode.class); if (!delete.isPresent()) { return context.defaultRewrite(node); } Optional<TableScanNode> tableScan = findNode(delete.get().getSource(), TableScanNode.class); if (!tableScan.isPresent()) { return context.defaultRewrite(node); } TableScanNode tableScanNode = tableScan.get(); if (!metadata.supportsMetadataDelete( session, tableScanNode.getTable(), tableScanNode.getLayout().get())) { return context.defaultRewrite(node); } return new MetadataDeleteNode( idAllocator.getNextId(), delete.get().getTarget(), Iterables.getOnlyElement(node.getOutputSymbols()), tableScanNode.getLayout().get()); }
private void generateFilterMethod( ClassDefinition classDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, RowExpression filter) { Parameter session = arg("session", ConnectorSession.class); Parameter position = arg("position", int.class); List<Parameter> blocks = toBlockParameters(getInputChannels(filter)); MethodDefinition method = classDefinition.declareMethod( a(PUBLIC), "filter", type(boolean.class), ImmutableList.<Parameter>builder().add(session).addAll(blocks).add(position).build()); method.comment("Filter: %s", filter.toString()); BytecodeBlock body = method.getBody(); Scope scope = method.getScope(); Variable wasNullVariable = scope.declareVariable("wasNull", body, constantFalse()); BytecodeExpressionVisitor visitor = new BytecodeExpressionVisitor( callSiteBinder, cachedInstanceBinder, fieldReferenceCompiler(callSiteBinder, position, wasNullVariable), metadata.getFunctionRegistry()); BytecodeNode visitorBody = filter.accept(visitor, scope); Variable result = scope.declareVariable(boolean.class, "result"); body.append(visitorBody) .putVariable(result) .append( new IfStatement() .condition(wasNullVariable) .ifTrue(constantFalse().ret()) .ifFalse(result.ret())); }
private PlanNode planTableScan(TableScanNode node, Expression predicate, Context context) { DomainTranslator.ExtractionResult decomposedPredicate = DomainTranslator.fromPredicate(metadata, session, predicate, symbolAllocator.getTypes()); TupleDomain<ColumnHandle> simplifiedConstraint = decomposedPredicate .getTupleDomain() .transform(node.getAssignments()::get) .intersect(node.getCurrentConstraint()); checkState(node.getOutputSymbols().containsAll(context.getLookupSymbols())); Set<ColumnHandle> lookupColumns = context .getLookupSymbols() .stream() .map(node.getAssignments()::get) .collect(toImmutableSet()); Set<ColumnHandle> outputColumns = node.getOutputSymbols() .stream() .map(node.getAssignments()::get) .collect(toImmutableSet()); Optional<ResolvedIndex> optionalResolvedIndex = metadata.resolveIndex( session, node.getTable(), lookupColumns, outputColumns, simplifiedConstraint); if (!optionalResolvedIndex.isPresent()) { // No index available, so give up by returning something return node; } ResolvedIndex resolvedIndex = optionalResolvedIndex.get(); Map<ColumnHandle, Symbol> inverseAssignments = ImmutableBiMap.copyOf(node.getAssignments()).inverse(); PlanNode source = new IndexSourceNode( idAllocator.getNextId(), resolvedIndex.getIndexHandle(), node.getTable(), node.getLayout(), context.getLookupSymbols(), node.getOutputSymbols(), node.getAssignments(), simplifiedConstraint); Expression resultingPredicate = combineConjuncts( DomainTranslator.toPredicate( resolvedIndex.getUnresolvedTupleDomain().transform(inverseAssignments::get)), decomposedPredicate.getRemainingExpression()); if (!resultingPredicate.equals(TRUE_LITERAL)) { // todo it is likely we end up with redundant filters here because the predicate push down // has already been run... the fix is to run predicate push down again source = new FilterNode(idAllocator.getNextId(), source, resultingPredicate); } context.markSuccess(); return source; }
public FunctionAssertions addScalarFunctions(Class<?> clazz) { metadata.addFunctions( new FunctionListBuilder(metadata.getTypeManager()).scalar(clazz).getFunctions()); return this; }
private Map<QualifiedObjectName, ViewDefinition> getViews( Session session, String catalogName, Map<String, NullableValue> filters) { return metadata.getViews(session, extractQualifiedTablePrefix(catalogName, filters)); }
private List<QualifiedObjectName> getViewsList( Session session, String catalogName, Map<String, NullableValue> filters) { return metadata.listViews(session, extractQualifiedTablePrefix(catalogName, filters)); }
private List<QualifiedTableName> getTablesList( Session session, String catalogName, Map<String, SerializableNativeValue> filters) { return metadata.listTables(session, extractQualifiedTablePrefix(catalogName, filters)); }
private InternalTable buildPartitions( Session session, String catalogName, Map<String, SerializableNativeValue> filters) { QualifiedTableName tableName = extractQualifiedTableName(catalogName, filters); InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_INTERNAL_PARTITIONS)); Optional<TableHandle> tableHandle = metadata.getTableHandle(session, tableName); checkArgument(tableHandle.isPresent(), "Table %s does not exist", tableName); Map<ColumnHandle, String> columnHandles = ImmutableBiMap.copyOf(metadata.getColumnHandles(session, tableHandle.get())).inverse(); List<TableLayoutResult> layouts = metadata.getLayouts( session, tableHandle.get(), Constraint.<ColumnHandle>alwaysTrue(), Optional.empty()); if (layouts.size() == 1) { TableLayout layout = Iterables.getOnlyElement(layouts).getLayout(); layout .getDiscretePredicates() .ifPresent( domains -> { int partitionNumber = 1; for (TupleDomain<ColumnHandle> domain : domains) { for (Entry<ColumnHandle, SerializableNativeValue> entry : domain.extractNullableFixedValues().entrySet()) { ColumnHandle columnHandle = entry.getKey(); String columnName = columnHandles.get(columnHandle); String value = null; if (entry.getValue().getValue() != null) { ColumnMetadata columnMetadata = metadata.getColumnMetadata(session, tableHandle.get(), columnHandle); try { FunctionInfo operator = metadata .getFunctionRegistry() .getCoercion(columnMetadata.getType(), VARCHAR); value = ((Slice) operator .getMethodHandle() .invokeWithArguments(entry.getValue().getValue())) .toStringUtf8(); } catch (OperatorNotFoundException e) { value = "<UNREPRESENTABLE VALUE>"; } catch (Throwable throwable) { throw Throwables.propagate(throwable); } } table.add( catalogName, tableName.getSchemaName(), tableName.getTableName(), partitionNumber, columnName, value); } partitionNumber++; } }); } return table.build(); }
@Override public CompletableFuture<?> execute( Call call, TransactionManager transactionManager, Metadata metadata, AccessControl accessControl, QueryStateMachine stateMachine) { if (!stateMachine.isAutoCommit()) { throw new PrestoException( NOT_SUPPORTED, "Procedures cannot be called within a transaction (use autocommit mode)"); } Session session = stateMachine.getSession(); QualifiedObjectName procedureName = createQualifiedObjectName(session, call, call.getName()); Procedure procedure = metadata.getProcedureRegistry().resolve(procedureName); // map declared argument names to positions Map<String, Integer> positions = new HashMap<>(); for (int i = 0; i < procedure.getArguments().size(); i++) { positions.put(procedure.getArguments().get(i).getName(), i); } // per specification, do not allow mixing argument types Predicate<CallArgument> hasName = argument -> argument.getName().isPresent(); boolean anyNamed = call.getArguments().stream().anyMatch(hasName); boolean allNamed = call.getArguments().stream().allMatch(hasName); if (anyNamed && !allNamed) { throw new SemanticException( INVALID_PROCEDURE_ARGUMENTS, call, "Named and positional arguments cannot be mixed"); } // get the argument names in call order Map<String, CallArgument> names = new LinkedHashMap<>(); for (int i = 0; i < call.getArguments().size(); i++) { CallArgument argument = call.getArguments().get(i); if (argument.getName().isPresent()) { String name = argument.getName().get(); if (names.put(name, argument) != null) { throw new SemanticException( INVALID_PROCEDURE_ARGUMENTS, argument, "Duplicate procedure argument: %s", name); } if (!positions.containsKey(name)) { throw new SemanticException( INVALID_PROCEDURE_ARGUMENTS, argument, "Unknown argument name: %s", name); } } else if (i < procedure.getArguments().size()) { names.put(procedure.getArguments().get(i).getName(), argument); } else { throw new SemanticException( INVALID_PROCEDURE_ARGUMENTS, call, "Too many arguments for procedure"); } } // verify argument count if (names.size() < positions.size()) { throw new SemanticException( INVALID_PROCEDURE_ARGUMENTS, call, "Too few arguments for procedure"); } // get argument values Object[] values = new Object[procedure.getArguments().size()]; for (Entry<String, CallArgument> entry : names.entrySet()) { CallArgument callArgument = entry.getValue(); int index = positions.get(entry.getKey()); Argument argument = procedure.getArguments().get(index); Expression expression = callArgument.getValue(); Type type = argument.getType(); Object value = evaluateConstantExpression(expression, type, metadata, session); values[index] = toTypeObjectValue(session, type, value); } // validate arguments MethodType methodType = procedure.getMethodHandle().type(); for (int i = 0; i < procedure.getArguments().size(); i++) { if ((values[i] == null) && methodType.parameterType(i).isPrimitive()) { String name = procedure.getArguments().get(i).getName(); throw new PrestoException( INVALID_PROCEDURE_ARGUMENT, "Procedure argument cannot be null: " + name); } } // insert session argument List<Object> arguments = new ArrayList<>(); Iterator<Object> valuesIterator = asList(values).iterator(); for (Class<?> type : methodType.parameterList()) { if (ConnectorSession.class.isAssignableFrom(type)) { arguments.add(session.toConnectorSession(procedureName.getCatalogName())); } else { arguments.add(valuesIterator.next()); } } try { procedure.getMethodHandle().invokeWithArguments(arguments); } catch (Throwable t) { if (t instanceof InterruptedException) { Thread.currentThread().interrupt(); } propagateIfInstanceOf(t, PrestoException.class); throw new PrestoException(PROCEDURE_CALL_FAILED, t); } return completedFuture(null); }
public PlanOptimizers( Metadata metadata, SqlParser sqlParser, FeaturesConfig featuresConfig, boolean forceSingleNode) { ImmutableList.Builder<PlanOptimizer> builder = ImmutableList.builder(); builder.add( new DesugaringOptimizer( metadata, sqlParser), // Clean up all the sugar in expressions, e.g. AtTimeZone, must be run // before all the other optimizers new CanonicalizeExpressions(), new ImplementFilteredAggregations(), new ImplementSampleAsFilter(), new SimplifyExpressions(metadata, sqlParser), new UnaliasSymbolReferences(), new PruneIdentityProjections(), new SetFlatteningOptimizer(), new ImplementIntersectAndExceptAsUnion(), new LimitPushDown(), // Run the LimitPushDown after flattening set operators to make it // easier to do the set flattening new PruneUnreferencedOutputs(), new MergeProjections(), new TransformExistsApplyToScalarApply(metadata), new TransformQuantifiedComparisonApplyToScalarApply(metadata), new RemoveUnreferencedScalarInputApplyNodes(), new TransformUncorrelatedInPredicateSubqueryToSemiJoin(), new TransformUncorrelatedScalarToJoin(), new TransformCorrelatedScalarAggregationToJoin(metadata), new PredicatePushDown(metadata, sqlParser), new MergeProjections(), new SimplifyExpressions( metadata, sqlParser), // Re-run the SimplifyExpressions to simplify any recomposed expressions // from other optimizations new ProjectionPushDown(), new UnaliasSymbolReferences(), // Run again because predicate pushdown and projection // pushdown might add more projections new PruneUnreferencedOutputs(), // Make sure to run this before index join. Filtered // projections may not have all the columns. new IndexJoinOptimizer( metadata), // Run this after projections and filters have been fully simplified and // pushed down new CountConstantOptimizer(), new WindowFilterPushDown( metadata), // This must run after PredicatePushDown and LimitPushDown so that it // squashes any successive filter nodes and limits new MergeWindows(), new ReorderWindows(), // Should be after MergeWindows to avoid unnecessary reordering of // mergeable windows new MergeProjections(), new PruneUnreferencedOutputs(), // Make sure to run this at the end to help clean the plan // for logging/execution and not remove info that other // optimizers might need at an earlier point new PruneIdentityProjections(), // This MUST run after PruneUnreferencedOutputs as it may // introduce new redundant projections new MetadataQueryOptimizer(metadata), new EliminateCrossJoins(), // This can pull up Filter and Project nodes from between Joins, // so we need to push them down again new PredicatePushDown(metadata, sqlParser), new ProjectionPushDown()); if (featuresConfig.isOptimizeSingleDistinct()) { builder.add(new SingleDistinctOptimizer()); builder.add(new PruneUnreferencedOutputs()); } builder.add(new OptimizeMixedDistinctAggregations(metadata)); if (!forceSingleNode) { builder.add(new PushTableWriteThroughUnion()); // Must run before AddExchanges builder.add(new AddExchanges(metadata, sqlParser)); } builder.add(new PickLayout(metadata)); builder.add(new EmptyDeleteOptimizer()); // Run after table scan is removed by PickLayout builder.add( new PredicatePushDown( metadata, sqlParser)); // Run predicate push down one more time in case we can leverage new // information from layouts' effective predicate builder.add(new ProjectionPushDown()); builder.add(new MergeProjections()); builder.add( new UnaliasSymbolReferences()); // Run unalias after merging projections to simplify // projections more efficiently builder.add(new PruneUnreferencedOutputs()); builder.add(new PruneIdentityProjections()); // Optimizers above this don't understand local exchanges, so be careful moving this. builder.add(new AddLocalExchanges(metadata, sqlParser)); // Optimizers above this do not need to care about aggregations with the type other than SINGLE // This optimizer must be run after all exchange-related optimizers builder.add(new PartialAggregationPushDown(metadata.getFunctionRegistry())); builder.add(new PruneIdentityProjections()); // DO NOT add optimizers that change the plan shape (computations) after this point // Precomputed hashes - this assumes that partitioning will not change builder.add(new HashGenerationOptimizer()); builder.add(new MetadataDeleteOptimizer(metadata)); builder.add(new BeginTableWrite(metadata)); // HACK! see comments in BeginTableWrite // TODO: consider adding a formal final plan sanitization optimizer that prepares the plan for // transmission/execution/logging // TODO: figure out how to improve the set flattening optimizer so that it can run at any point this.optimizers = builder.build(); }
private InternalTable buildPartitions( Session session, String catalogName, Map<String, NullableValue> filters) { QualifiedObjectName tableName = extractQualifiedTableName(catalogName, filters); InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_INTERNAL_PARTITIONS)); Optional<TableHandle> tableHandle = metadata.getTableHandle(session, tableName); if (!tableHandle.isPresent()) { throw new TableNotFoundException(tableName.asSchemaTableName()); } List<TableLayoutResult> layouts = metadata.getLayouts( session, tableHandle.get(), Constraint.<ColumnHandle>alwaysTrue(), Optional.empty()); if (layouts.size() == 1) { Map<ColumnHandle, String> columnHandles = ImmutableBiMap.copyOf(metadata.getColumnHandles(session, tableHandle.get())).inverse(); Map<ColumnHandle, MethodHandle> methodHandles = new HashMap<>(); for (ColumnHandle columnHandle : columnHandles.keySet()) { try { ColumnMetadata columnMetadata = metadata.getColumnMetadata(session, tableHandle.get(), columnHandle); Signature operator = metadata.getFunctionRegistry().getCoercion(columnMetadata.getType(), VARCHAR); MethodHandle methodHandle = metadata .getFunctionRegistry() .getScalarFunctionImplementation(operator) .getMethodHandle(); methodHandles.put(columnHandle, methodHandle); } catch (OperatorNotFoundException exception) { // Do not put the columnHandle in the map. } } TableLayout layout = Iterables.getOnlyElement(layouts).getLayout(); layout .getDiscretePredicates() .ifPresent( domains -> { int partitionNumber = 1; for (TupleDomain<ColumnHandle> domain : domains) { for (Entry<ColumnHandle, NullableValue> entry : TupleDomain.extractFixedValues(domain).get().entrySet()) { ColumnHandle columnHandle = entry.getKey(); String columnName = columnHandles.get(columnHandle); String value = null; if (entry.getValue().getValue() != null) { if (methodHandles.containsKey(columnHandle)) { try { value = ((Slice) methodHandles .get(columnHandle) .invokeWithArguments(entry.getValue().getValue())) .toStringUtf8(); } catch (Throwable throwable) { throw Throwables.propagate(throwable); } } else { // OperatorNotFoundException was thrown for this columnHandle value = "<UNREPRESENTABLE VALUE>"; } } table.add( catalogName, tableName.getSchemaName(), tableName.getObjectName(), partitionNumber, columnName, value); } partitionNumber++; } }); } return table.build(); }
private Map<QualifiedObjectName, List<ColumnMetadata>> getColumnsList( Session session, String catalogName, Map<String, NullableValue> filters) { return metadata.listTableColumns(session, extractQualifiedTablePrefix(catalogName, filters)); }
public FunctionAssertions addFunctions(List<SqlFunction> functionInfos) { metadata.addFunctions(functionInfos); return this; }