Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Aggregation and window refactor #9837

Merged
merged 14 commits into from
Dec 17, 2021
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@
import io.trino.operator.aggregation.DoubleRegressionAggregation;
import io.trino.operator.aggregation.DoubleSumAggregation;
import io.trino.operator.aggregation.GeometricMeanAggregations;
import io.trino.operator.aggregation.InternalAggregationFunction;
import io.trino.operator.aggregation.IntervalDayToSecondAverageAggregation;
import io.trino.operator.aggregation.IntervalDayToSecondSumAggregation;
import io.trino.operator.aggregation.IntervalYearToMonthAverageAggregation;
Expand Down Expand Up @@ -343,7 +342,6 @@
import static io.trino.operator.scalar.TryCastFunction.TRY_CAST;
import static io.trino.operator.scalar.ZipFunction.ZIP_FUNCTIONS;
import static io.trino.operator.scalar.ZipWithFunction.ZIP_WITH_FUNCTION;
import static io.trino.operator.window.AggregateWindowFunction.supplier;
import static io.trino.type.DecimalCasts.BIGINT_TO_DECIMAL_CAST;
import static io.trino.type.DecimalCasts.BOOLEAN_TO_DECIMAL_CAST;
import static io.trino.type.DecimalCasts.DECIMAL_TO_BIGINT_CAST;
Expand Down Expand Up @@ -384,7 +382,7 @@
public class FunctionRegistry
{
private final Cache<FunctionKey, ScalarFunctionImplementation> specializedScalarCache;
private final Cache<FunctionKey, InternalAggregationFunction> specializedAggregationCache;
private final Cache<FunctionKey, AggregationMetadata> specializedAggregationCache;
private final Cache<FunctionKey, WindowFunctionSupplier> specializedWindowCache;
private volatile FunctionMap functions = new FunctionMap();

Expand Down Expand Up @@ -838,12 +836,7 @@ public AggregationFunctionMetadata getAggregationFunctionMetadata(FunctionId fun

public WindowFunctionSupplier getWindowFunctionImplementation(FunctionId functionId, BoundSignature boundSignature, FunctionDependencies functionDependencies)
{
SqlFunction function = functions.get(functionId);
try {
if (function instanceof SqlAggregationFunction) {
InternalAggregationFunction aggregationFunction = specializedAggregationCache.get(new FunctionKey(functionId, boundSignature), () -> specializedAggregation(functionId, boundSignature, functionDependencies));
return supplier(function.getFunctionMetadata().getSignature(), aggregationFunction);
}
return specializedWindowCache.get(new FunctionKey(functionId, boundSignature), () -> specializeWindow(functionId, boundSignature, functionDependencies));
}
catch (ExecutionException | UncheckedExecutionException e) {
Expand All @@ -858,7 +851,7 @@ private WindowFunctionSupplier specializeWindow(FunctionId functionId, BoundSign
return function.specialize(boundSignature, functionDependencies);
}

public InternalAggregationFunction getAggregateFunctionImplementation(FunctionId functionId, BoundSignature boundSignature, FunctionDependencies functionDependencies)
public AggregationMetadata getAggregateFunctionImplementation(FunctionId functionId, BoundSignature boundSignature, FunctionDependencies functionDependencies)
{
try {
return specializedAggregationCache.get(new FunctionKey(functionId, boundSignature), () -> specializedAggregation(functionId, boundSignature, functionDependencies));
Expand All @@ -869,11 +862,10 @@ public InternalAggregationFunction getAggregateFunctionImplementation(FunctionId
}
}

private InternalAggregationFunction specializedAggregation(FunctionId functionId, BoundSignature boundSignature, FunctionDependencies functionDependencies)
private AggregationMetadata specializedAggregation(FunctionId functionId, BoundSignature boundSignature, FunctionDependencies functionDependencies)
{
SqlAggregationFunction aggregationFunction = (SqlAggregationFunction) functions.get(functionId);
AggregationMetadata aggregationMetadata = aggregationFunction.specialize(boundSignature, functionDependencies);
return new InternalAggregationFunction(boundSignature, aggregationMetadata);
return aggregationFunction.specialize(boundSignature, functionDependencies);
}

public FunctionDependencyDeclaration getFunctionDependencies(FunctionBinding functionBinding)
Expand Down
4 changes: 2 additions & 2 deletions core/trino-main/src/main/java/io/trino/metadata/Metadata.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
import io.airlift.slice.Slice;
import io.trino.Session;
import io.trino.connector.CatalogName;
import io.trino.operator.aggregation.InternalAggregationFunction;
import io.trino.operator.aggregation.AggregationMetadata;
import io.trino.operator.window.WindowFunctionSupplier;
import io.trino.spi.TrinoException;
import io.trino.spi.block.BlockEncodingSerde;
Expand Down Expand Up @@ -671,7 +671,7 @@ default ResolvedFunction getCoercion(Session session, Type fromType, Type toType

WindowFunctionSupplier getWindowFunctionImplementation(ResolvedFunction resolvedFunction);

InternalAggregationFunction getAggregateFunctionImplementation(ResolvedFunction resolvedFunction);
AggregationMetadata getAggregateFunctionImplementation(ResolvedFunction resolvedFunction);

FunctionInvoker getScalarFunctionInvoker(ResolvedFunction resolvedFunction, InvocationConvention invocationConvention);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
import io.trino.connector.CatalogName;
import io.trino.metadata.Catalog.SecurityManagement;
import io.trino.metadata.ResolvedFunction.ResolvedFunctionDecoder;
import io.trino.operator.aggregation.InternalAggregationFunction;
import io.trino.operator.aggregation.AggregationMetadata;
import io.trino.operator.window.WindowFunctionSupplier;
import io.trino.spi.QueryId;
import io.trino.spi.TrinoException;
Expand Down Expand Up @@ -2683,7 +2683,7 @@ public WindowFunctionSupplier getWindowFunctionImplementation(ResolvedFunction r
}

@Override
public InternalAggregationFunction getAggregateFunctionImplementation(ResolvedFunction resolvedFunction)
public AggregationMetadata getAggregateFunctionImplementation(ResolvedFunction resolvedFunction)
{
FunctionDependencies functionDependencies = new FunctionDependencies(this, resolvedFunction.getTypeDependencies(), resolvedFunction.getFunctionDependencies());
return functions.getAggregateFunctionImplementation(resolvedFunction.getFunctionId(), resolvedFunction.getSignature(), functionDependencies);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,12 @@

import com.google.common.collect.ImmutableList;
import io.trino.memory.context.LocalMemoryContext;
import io.trino.operator.aggregation.AccumulatorFactory;
import io.trino.operator.aggregation.Aggregator;
import io.trino.operator.aggregation.AggregatorFactory;
import io.trino.spi.Page;
import io.trino.spi.PageBuilder;
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.type.Type;
import io.trino.sql.planner.plan.AggregationNode.Step;
import io.trino.sql.planner.plan.PlanNodeId;

import java.util.List;
Expand All @@ -40,17 +40,15 @@ public static class AggregationOperatorFactory
{
private final int operatorId;
private final PlanNodeId planNodeId;
private final Step step;
private final List<AccumulatorFactory> accumulatorFactories;
private final List<AggregatorFactory> aggregatorFactories;
private final boolean useSystemMemory;
private boolean closed;

public AggregationOperatorFactory(int operatorId, PlanNodeId planNodeId, Step step, List<AccumulatorFactory> accumulatorFactories, boolean useSystemMemory)
public AggregationOperatorFactory(int operatorId, PlanNodeId planNodeId, List<AggregatorFactory> aggregatorFactories, boolean useSystemMemory)
{
this.operatorId = operatorId;
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
this.step = step;
this.accumulatorFactories = ImmutableList.copyOf(accumulatorFactories);
this.aggregatorFactories = ImmutableList.copyOf(aggregatorFactories);
this.useSystemMemory = useSystemMemory;
}

Expand All @@ -59,7 +57,7 @@ public Operator createOperator(DriverContext driverContext)
{
checkState(!closed, "Factory is already closed");
OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, AggregationOperator.class.getSimpleName());
return new AggregationOperator(operatorContext, step, accumulatorFactories, useSystemMemory);
return new AggregationOperator(operatorContext, aggregatorFactories, useSystemMemory);
}

@Override
Expand All @@ -71,7 +69,7 @@ public void noMoreOperators()
@Override
public OperatorFactory duplicate()
{
return new AggregationOperatorFactory(operatorId, planNodeId, step, accumulatorFactories, useSystemMemory);
return new AggregationOperatorFactory(operatorId, planNodeId, aggregatorFactories, useSystemMemory);
}
}

Expand All @@ -90,22 +88,16 @@ private enum State

private State state = State.NEEDS_INPUT;

public AggregationOperator(OperatorContext operatorContext, Step step, List<AccumulatorFactory> accumulatorFactories, boolean useSystemMemory)
public AggregationOperator(OperatorContext operatorContext, List<AggregatorFactory> aggregatorFactories, boolean useSystemMemory)
{
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.systemMemoryContext = operatorContext.newLocalSystemMemoryContext(AggregationOperator.class.getSimpleName());
this.userMemoryContext = operatorContext.localUserMemoryContext();
this.useSystemMemory = useSystemMemory;

requireNonNull(step, "step is null");

// wrapper each function with an aggregator
requireNonNull(accumulatorFactories, "accumulatorFactories is null");
ImmutableList.Builder<Aggregator> builder = ImmutableList.builder();
for (AccumulatorFactory accumulatorFactory : accumulatorFactories) {
builder.add(new Aggregator(accumulatorFactory, step));
}
aggregates = builder.build();
aggregates = aggregatorFactories.stream()
.map(AggregatorFactory::createAggregator)
.collect(toImmutableList());
}

@Override
Expand Down
79 changes: 0 additions & 79 deletions core/trino-main/src/main/java/io/trino/operator/Aggregator.java

This file was deleted.

Loading