Skip to content

Commit

Permalink
Rename AggregationImplementation to ParametricAggregationImplementation
Browse files Browse the repository at this point in the history
  • Loading branch information
dain committed Sep 4, 2022
1 parent cbd7571 commit 728da10
Show file tree
Hide file tree
Showing 5 changed files with 57 additions and 57 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@
import static com.google.common.base.Strings.emptyToNull;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.Iterables.getOnlyElement;
import static io.trino.operator.aggregation.AggregationImplementation.Parser.parseImplementation;
import static io.trino.operator.aggregation.ParametricAggregationImplementation.Parser.parseImplementation;
import static io.trino.operator.aggregation.state.StateCompiler.generateInOutStateFactory;
import static io.trino.operator.aggregation.state.StateCompiler.generateStateFactory;
import static io.trino.operator.aggregation.state.StateCompiler.generateStateSerializer;
Expand Down Expand Up @@ -107,11 +107,11 @@ else if (combineFunction.isPresent()) {
}

// Input functions can have either an exact signature, or generic/calculate signature
List<AggregationImplementation> exactImplementations = new ArrayList<>();
List<AggregationImplementation> nonExactImplementations = new ArrayList<>();
List<ParametricAggregationImplementation> exactImplementations = new ArrayList<>();
List<ParametricAggregationImplementation> nonExactImplementations = new ArrayList<>();
for (Method inputFunction : getInputFunctions(aggregationDefinition, stateDetails)) {
Optional<Method> removeInputFunction = getRemoveInputFunction(aggregationDefinition, inputFunction);
AggregationImplementation implementation = parseImplementation(
ParametricAggregationImplementation implementation = parseImplementation(
aggregationDefinition,
header.getName(),
stateDetails,
Expand Down Expand Up @@ -141,13 +141,13 @@ private static List<ParametricAggregation> buildFunctions(
String name,
AggregationHeader header,
List<AccumulatorStateDetails<?>> stateDetails,
List<AggregationImplementation> exactImplementations,
List<AggregationImplementation> nonExactImplementations)
List<ParametricAggregationImplementation> exactImplementations,
List<ParametricAggregationImplementation> nonExactImplementations)
{
ImmutableList.Builder<ParametricAggregation> functions = ImmutableList.builder();

// create a separate function for each exact implementation
for (AggregationImplementation exactImplementation : exactImplementations) {
for (ParametricAggregationImplementation exactImplementation : exactImplementations) {
functions.add(new ParametricAggregation(
exactImplementation.getSignature().withName(name),
header,
Expand All @@ -157,9 +157,9 @@ private static List<ParametricAggregation> buildFunctions(

// if there are non-exact functions, create a single generic/calculated function using these implementations
if (!nonExactImplementations.isEmpty()) {
ParametricImplementationsGroup.Builder<AggregationImplementation> implementationsBuilder = ParametricImplementationsGroup.builder();
ParametricImplementationsGroup.Builder<ParametricAggregationImplementation> implementationsBuilder = ParametricImplementationsGroup.builder();
nonExactImplementations.forEach(implementationsBuilder::addImplementation);
ParametricImplementationsGroup<AggregationImplementation> implementations = implementationsBuilder.build();
ParametricImplementationsGroup<ParametricAggregationImplementation> implementations = implementationsBuilder.build();
functions.add(new ParametricAggregation(
implementations.getSignature().withName(name),
header,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,14 +52,14 @@
public class ParametricAggregation
extends SqlAggregationFunction
{
private final ParametricImplementationsGroup<AggregationImplementation> implementations;
private final ParametricImplementationsGroup<ParametricAggregationImplementation> implementations;
private final List<AccumulatorStateDetails<?>> stateDetails;

public ParametricAggregation(
Signature signature,
AggregationHeader details,
List<AccumulatorStateDetails<?>> stateDetails,
ParametricImplementationsGroup<AggregationImplementation> implementations)
ParametricImplementationsGroup<ParametricAggregationImplementation> implementations)
{
super(
createFunctionMetadata(signature, details, implementations.getFunctionNullability()),
Expand Down Expand Up @@ -126,9 +126,9 @@ public FunctionDependencyDeclaration getFunctionDependencies()
return builder.build();
}

private static void declareDependencies(FunctionDependencyDeclarationBuilder builder, Collection<AggregationImplementation> implementations)
private static void declareDependencies(FunctionDependencyDeclarationBuilder builder, Collection<ParametricAggregationImplementation> implementations)
{
for (AggregationImplementation implementation : implementations) {
for (ParametricAggregationImplementation implementation : implementations) {
for (ImplementationDependency dependency : implementation.getInputDependencies()) {
dependency.declareDependencies(builder);
}
Expand All @@ -145,7 +145,7 @@ private static void declareDependencies(FunctionDependencyDeclarationBuilder bui
public AggregationMetadata specialize(BoundSignature boundSignature, FunctionDependencies functionDependencies)
{
// Find implementation matching arguments
AggregationImplementation concreteImplementation = findMatchingImplementation(boundSignature);
ParametricAggregationImplementation concreteImplementation = findMatchingImplementation(boundSignature);

// Build state factory and serializer
FunctionMetadata metadata = getFunctionMetadata();
Expand Down Expand Up @@ -189,20 +189,20 @@ public List<AccumulatorStateDetails<?>> getStateDetails()
}

@VisibleForTesting
public ParametricImplementationsGroup<AggregationImplementation> getImplementations()
public ParametricImplementationsGroup<ParametricAggregationImplementation> getImplementations()
{
return implementations;
}

private AggregationImplementation findMatchingImplementation(BoundSignature boundSignature)
private ParametricAggregationImplementation findMatchingImplementation(BoundSignature boundSignature)
{
Signature signature = boundSignature.toSignature();
Optional<AggregationImplementation> foundImplementation = Optional.empty();
Optional<ParametricAggregationImplementation> foundImplementation = Optional.empty();
if (implementations.getExactImplementations().containsKey(signature)) {
foundImplementation = Optional.of(implementations.getExactImplementations().get(signature));
}
else {
for (AggregationImplementation candidate : implementations.getGenericImplementations()) {
for (ParametricAggregationImplementation candidate : implementations.getGenericImplementations()) {
if (candidate.areTypesAssignable(boundSignature)) {
if (foundImplementation.isPresent()) {
throw new TrinoException(AMBIGUOUS_FUNCTION_CALL, format("Ambiguous function call (%s) for %s", boundSignature, getFunctionMetadata().getSignature()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@
import static io.trino.util.Reflection.methodHandle;
import static java.util.Objects.requireNonNull;

public class AggregationImplementation
public class ParametricAggregationImplementation
implements ParametricImplementation
{
public static class AggregateNativeContainerType
Expand Down Expand Up @@ -105,7 +105,7 @@ public boolean isBlockPosition()
private final List<AggregationParameterKind> inputParameterKinds;
private final FunctionNullability functionNullability;

public AggregationImplementation(
public ParametricAggregationImplementation(
Signature signature,
Class<?> definitionClass,
MethodHandle inputFunction,
Expand Down Expand Up @@ -232,9 +232,9 @@ public boolean areTypesAssignable(BoundSignature boundSignature)
}

@Override
public AggregationImplementation withAlias(String alias)
public ParametricImplementation withAlias(String alias)
{
return new AggregationImplementation(
return new ParametricAggregationImplementation(
signature.withName(alias),
definitionClass,
inputFunction,
Expand Down Expand Up @@ -323,9 +323,9 @@ private Parser(
outputHandle = methodHandle(outputFunction);
}

private AggregationImplementation get()
private ParametricAggregationImplementation get()
{
return new AggregationImplementation(
return new ParametricAggregationImplementation(
signatureBuilder.build(),
aggregationDefinition,
inputHandle,
Expand All @@ -340,7 +340,7 @@ private AggregationImplementation get()
inputParameterKinds);
}

public static AggregationImplementation parseImplementation(
public static ParametricAggregationImplementation parseImplementation(
Class<?> aggregationDefinition,
String name,
List<AccumulatorStateDetails<?>> stateDetails,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@
*/
package io.trino.operator;

import io.trino.operator.aggregation.AggregationImplementation;
import io.trino.operator.aggregation.ParametricAggregationImplementation;
import io.trino.operator.scalar.ParametricScalar;

import static org.testng.Assert.assertEquals;
Expand All @@ -32,7 +32,7 @@ void assertImplementationCount(ParametricImplementationsGroup<?> implementations
assertEquals(implementations.getGenericImplementations().size(), generic);
}

void assertDependencyCount(AggregationImplementation implementation, int input, int combine, int output)
void assertDependencyCount(ParametricAggregationImplementation implementation, int input, int combine, int output)
{
assertEquals(implementation.getInputDependencies().size(), input);
assertEquals(implementation.getCombineDependencies().size(), combine);
Expand Down
Loading

0 comments on commit 728da10

Please sign in to comment.