Skip to content

Commit

Permalink
Add SQL routine compiler
Browse files Browse the repository at this point in the history
Co-authored-by: Dain Sundstrom <[email protected]>
  • Loading branch information
electrum and dain committed Oct 24, 2023
1 parent 4de41c1 commit 9d3fd1b
Show file tree
Hide file tree
Showing 31 changed files with 3,704 additions and 7 deletions.
12 changes: 12 additions & 0 deletions core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java
Original file line number Diff line number Diff line change
Expand Up @@ -648,6 +648,13 @@ public void registerTable(
columnMaskScopes.isEmpty()));
}

public Set<ResolvedFunction> getResolvedFunctions()
{
return resolvedFunctions.values().stream()
.map(RoutineEntry::getFunction)
.collect(toImmutableSet());
}

public ResolvedFunction getResolvedFunction(Expression node)
{
return resolvedFunctions.get(NodeRef.of(node)).getFunction();
Expand Down Expand Up @@ -680,6 +687,11 @@ public boolean isColumnReference(Expression expression)
return columnReferences.containsKey(NodeRef.of(expression));
}

public void addType(Expression expression, Type type)
{
this.types.put(NodeRef.of(expression), type);
}

public void addTypes(Map<NodeRef<Expression>, Type> types)
{
this.types.putAll(types);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -359,7 +359,7 @@ public BytecodeNode visitVariableReference(VariableReferenceExpression reference
};
}

static class CompiledLambda
public static class CompiledLambda
{
// lambda method information
private final Handle lambdaAsmHandle;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public class RowExpressionCompiler
private final FunctionManager functionManager;
private final Map<LambdaDefinitionExpression, CompiledLambda> compiledLambdaMap;

RowExpressionCompiler(
public RowExpressionCompiler(
CallSiteBinder callSiteBinder,
CachedInstanceBinder cachedInstanceBinder,
RowExpressionVisitor<BytecodeNode, Scope> fieldReferenceCompiler,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -909,7 +909,7 @@ private RelationPlanner getRelationPlanner(Analysis analysis)
return new RelationPlanner(analysis, symbolAllocator, idAllocator, buildLambdaDeclarationToSymbolMap(analysis, symbolAllocator), plannerContext, Optional.empty(), session, ImmutableMap.of());
}

private static Map<NodeRef<LambdaArgumentDeclaration>, Symbol> buildLambdaDeclarationToSymbolMap(Analysis analysis, SymbolAllocator symbolAllocator)
public static Map<NodeRef<LambdaArgumentDeclaration>, Symbol> buildLambdaDeclarationToSymbolMap(Analysis analysis, SymbolAllocator symbolAllocator)
{
Map<Key, Symbol> allocations = new HashMap<>();
Map<NodeRef<LambdaArgumentDeclaration>, Symbol> result = new LinkedHashMap<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@
* <li>AST expressions contain Identifiers, while IR expressions contain SymbolReferences</li>
* <li>FunctionCalls in AST expressions are SQL function names. In IR expressions, they contain an encoded name representing a resolved function</li>
*/
class TranslationMap
public class TranslationMap
{
// all expressions are rewritten in terms of fields declared by this relation plan
private final Scope scope;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ public void validate(PlanNode planNode,
ExpressionExtractor.forEachExpression(planNode, SugarFreeChecker::validate);
}

private static void validate(Expression expression)
public static void validate(Expression expression)
{
VISITOR.process(expression, null);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,15 +151,15 @@ public static RowExpression translate(
return result;
}

private static class Visitor
public static class Visitor
extends AstVisitor<RowExpression, Void>
{
private final Metadata metadata;
private final Map<NodeRef<Expression>, Type> types;
private final Map<Symbol, Integer> layout;
private final StandardFunctionResolution standardFunctionResolution;

private Visitor(
protected Visitor(
Metadata metadata,
Map<NodeRef<Expression>, Type> types,
Map<Symbol, Integer> layout)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.trino.sql.routine;

import com.google.common.collect.ImmutableMap;
import io.trino.spi.type.Type;
import io.trino.sql.analyzer.Analysis;

import java.util.Map;
import java.util.Optional;

import static java.util.Objects.requireNonNull;

public record SqlRoutineAnalysis(
String name,
Map<String, Type> arguments,
Type returnType,
boolean calledOnNull,
boolean deterministic,
Optional<String> comment,
Analysis analysis)
{
public SqlRoutineAnalysis
{
requireNonNull(name, "name is null");
arguments = ImmutableMap.copyOf(requireNonNull(arguments, "arguments is null"));
requireNonNull(returnType, "returnType is null");
requireNonNull(comment, "comment is null");
requireNonNull(analysis, "analysis is null");
}
}
Loading

0 comments on commit 9d3fd1b

Please sign in to comment.