diff --git a/core/trino-main/src/main/java/io/trino/operator/window/AbstractWindowFunctionSupplier.java b/core/trino-main/src/main/java/io/trino/operator/window/AbstractWindowFunctionSupplier.java deleted file mode 100644 index ee7fafadcc26..000000000000 --- a/core/trino-main/src/main/java/io/trino/operator/window/AbstractWindowFunctionSupplier.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.operator.window; - -import io.trino.metadata.Signature; -import io.trino.operator.aggregation.LambdaProvider; -import io.trino.spi.function.WindowFunction; -import io.trino.type.FunctionType; - -import java.util.List; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Objects.requireNonNull; - -public abstract class AbstractWindowFunctionSupplier - implements WindowFunctionSupplier -{ - private final Signature signature; - private final String description; - private final List> lambdaInterfaces; - - protected AbstractWindowFunctionSupplier(Signature signature, String description, List> lambdaInterfaces) - { - this.signature = requireNonNull(signature, "signature is null"); - this.description = description; - this.lambdaInterfaces = lambdaInterfaces; - } - - @Override - public final Signature getSignature() - { - return signature; - } - - @Override - public final String getDescription() - { - return description; - } - - @Override - public List> getLambdaInterfaces() - { - return lambdaInterfaces; - } - - @Override - public final WindowFunction createWindowFunction(List argumentChannels, boolean ignoreNulls, List lambdaProviders) - { - requireNonNull(argumentChannels, "argumentChannels is null"); - - long argumentCount = signature.getArgumentTypes().stream() - .filter(type -> !type.getBase().equalsIgnoreCase(FunctionType.NAME)) - .count(); - - checkArgument(argumentChannels.size() == argumentCount, - "Expected %s arguments for function %s, but got %s", - argumentCount, - signature.getName(), - argumentChannels.size()); - - return newWindowFunction(argumentChannels, ignoreNulls, lambdaProviders); - } - - /** - * Create window function instance using the supplied arguments. The - * inputs have already validated. - */ - protected abstract WindowFunction newWindowFunction(List inputs, boolean ignoreNulls, List lambdaProviders); -} diff --git a/core/trino-main/src/main/java/io/trino/operator/window/AggregateWindowFunction.java b/core/trino-main/src/main/java/io/trino/operator/window/AggregateWindowFunction.java index 8ec645ed3553..c2ce4c7934fd 100644 --- a/core/trino-main/src/main/java/io/trino/operator/window/AggregateWindowFunction.java +++ b/core/trino-main/src/main/java/io/trino/operator/window/AggregateWindowFunction.java @@ -149,10 +149,16 @@ private void resetAccumulator() public static WindowFunctionSupplier supplier(Signature signature, InternalAggregationFunction function) { requireNonNull(function, "function is null"); - return new AbstractWindowFunctionSupplier(signature, null, function.getLambdaInterfaces()) + return new WindowFunctionSupplier() { @Override - protected WindowFunction newWindowFunction(List inputs, boolean ignoreNulls, List lambdaProviders) + public List> getLambdaInterfaces() + { + return function.getLambdaInterfaces(); + } + + @Override + public WindowFunction createWindowFunction(List inputs, boolean ignoreNulls, List lambdaProviders) { return new AggregateWindowFunction(function, inputs, lambdaProviders); } diff --git a/core/trino-main/src/main/java/io/trino/operator/window/ReflectionWindowFunctionSupplier.java b/core/trino-main/src/main/java/io/trino/operator/window/ReflectionWindowFunctionSupplier.java index 4e3afe448410..ea0e9147f0e8 100644 --- a/core/trino-main/src/main/java/io/trino/operator/window/ReflectionWindowFunctionSupplier.java +++ b/core/trino-main/src/main/java/io/trino/operator/window/ReflectionWindowFunctionSupplier.java @@ -15,22 +15,18 @@ import com.google.common.base.VerifyException; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import io.trino.metadata.Signature; import io.trino.operator.aggregation.LambdaProvider; -import io.trino.spi.function.Description; import io.trino.spi.function.ValueWindowFunction; import io.trino.spi.function.WindowFunction; -import io.trino.spi.type.Type; -import java.lang.reflect.AnnotatedElement; import java.lang.reflect.Constructor; import java.util.List; +import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; -public class ReflectionWindowFunctionSupplier - extends AbstractWindowFunctionSupplier +public class ReflectionWindowFunctionSupplier + implements WindowFunctionSupplier { private enum ConstructorType { @@ -39,22 +35,18 @@ private enum ConstructorType INPUTS_IGNORE_NULLS } - private final Constructor constructor; + private final int argumentCount; + private final Constructor constructor; private final ConstructorType constructorType; - public ReflectionWindowFunctionSupplier(String name, Type returnType, List argumentTypes, Class type) + public ReflectionWindowFunctionSupplier(int argumentCount, Class type) { - this(new Signature(name, returnType.getTypeSignature(), Lists.transform(argumentTypes, Type::getTypeSignature)), type); - } - - public ReflectionWindowFunctionSupplier(Signature signature, Class type) - { - super(signature, getDescription(requireNonNull(type, "type is null")), ImmutableList.of()); + this.argumentCount = argumentCount; try { - Constructor constructor; + Constructor constructor; ConstructorType constructorType; - if (signature.getArgumentTypes().isEmpty()) { + if (argumentCount == 0) { constructor = type.getConstructor(); constructorType = ConstructorType.NO_INPUTS; } @@ -83,16 +75,25 @@ else if (ValueWindowFunction.class.isAssignableFrom(type)) { } @Override - protected T newWindowFunction(List inputs, boolean ignoreNulls, List lambdaProviders) + public List> getLambdaInterfaces() { + return ImmutableList.of(); + } + + @Override + public WindowFunction createWindowFunction(List argumentChannels, boolean ignoreNulls, List lambdaProviders) + { + requireNonNull(argumentChannels, "inputs is null"); + checkArgument(argumentChannels.size() == argumentCount, "Expected %s arguments, but got %s", argumentCount, argumentChannels.size()); + try { switch (constructorType) { case NO_INPUTS: return constructor.newInstance(); case INPUTS: - return constructor.newInstance(inputs); + return constructor.newInstance(argumentChannels); case INPUTS_IGNORE_NULLS: - return constructor.newInstance(inputs, ignoreNulls); + return constructor.newInstance(argumentChannels, ignoreNulls); } throw new VerifyException("Unhandled constructor type: " + constructorType); } @@ -100,10 +101,4 @@ protected T newWindowFunction(List inputs, boolean ignoreNulls, List description, boolean deprecated, WindowFunctionSupplier supplier) { this.supplier = requireNonNull(supplier, "supplier is null"); - Signature signature = supplier.getSignature(); functionMetadata = new FunctionMetadata( signature, signature.getName(), new FunctionNullability(true, nCopies(signature.getArgumentTypes().size(), true)), false, true, - nullToEmpty(supplier.getDescription()), + description.orElse(""), WINDOW, deprecated); } diff --git a/core/trino-main/src/main/java/io/trino/operator/window/WindowAnnotationsParser.java b/core/trino-main/src/main/java/io/trino/operator/window/WindowAnnotationsParser.java index 838527caf5d3..a714cde9d374 100644 --- a/core/trino-main/src/main/java/io/trino/operator/window/WindowAnnotationsParser.java +++ b/core/trino-main/src/main/java/io/trino/operator/window/WindowAnnotationsParser.java @@ -17,11 +17,13 @@ import com.google.common.collect.ImmutableSet; import io.trino.metadata.Signature; import io.trino.metadata.TypeVariableConstraint; +import io.trino.spi.function.Description; import io.trino.spi.function.WindowFunction; import io.trino.spi.function.WindowFunctionSignature; import io.trino.spi.type.TypeSignature; import java.util.List; +import java.util.Optional; import java.util.stream.Stream; import static com.google.common.base.Preconditions.checkArgument; @@ -61,7 +63,10 @@ private static SqlWindowFunction parse(Class clazz, Wi argumentTypes, false); + Optional description = Optional.ofNullable(clazz.getAnnotation(Description.class)).map(Description::value); + boolean deprecated = clazz.getAnnotationsByType(Deprecated.class).length > 0; - return new SqlWindowFunction(new ReflectionWindowFunctionSupplier<>(signature, clazz), deprecated); + + return new SqlWindowFunction(signature, description, deprecated, new ReflectionWindowFunctionSupplier(window.argumentTypes().length, clazz)); } } diff --git a/core/trino-main/src/main/java/io/trino/operator/window/WindowFunctionSupplier.java b/core/trino-main/src/main/java/io/trino/operator/window/WindowFunctionSupplier.java index e1c10c0e1fda..2df6ef89a691 100644 --- a/core/trino-main/src/main/java/io/trino/operator/window/WindowFunctionSupplier.java +++ b/core/trino-main/src/main/java/io/trino/operator/window/WindowFunctionSupplier.java @@ -13,7 +13,6 @@ */ package io.trino.operator.window; -import io.trino.metadata.Signature; import io.trino.operator.aggregation.LambdaProvider; import io.trino.spi.function.WindowFunction; @@ -21,10 +20,6 @@ public interface WindowFunctionSupplier { - Signature getSignature(); - - String getDescription(); - WindowFunction createWindowFunction(List argumentChannels, boolean ignoreNulls, List lambdaProviders); List> getLambdaInterfaces(); diff --git a/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java index c2359f24d9e3..11982b8833d3 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java @@ -79,25 +79,25 @@ public class TestWindowOperator private static final FrameInfo UNBOUNDED_FRAME = new FrameInfo(RANGE, UNBOUNDED_PRECEDING, Optional.empty(), Optional.empty(), UNBOUNDED_FOLLOWING, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); public static final List ROW_NUMBER = ImmutableList.of( - window(new ReflectionWindowFunctionSupplier<>("row_number", BIGINT, ImmutableList.of(), RowNumberFunction.class), BIGINT, UNBOUNDED_FRAME, false, ImmutableList.of())); + window(new ReflectionWindowFunctionSupplier(0, RowNumberFunction.class), BIGINT, UNBOUNDED_FRAME, false, ImmutableList.of())); public static final List RANK = ImmutableList.of( - window(new ReflectionWindowFunctionSupplier<>("rank", BIGINT, ImmutableList.of(), RankFunction.class), BIGINT, UNBOUNDED_FRAME, false, ImmutableList.of())); + window(new ReflectionWindowFunctionSupplier(0, RankFunction.class), BIGINT, UNBOUNDED_FRAME, false, ImmutableList.of())); private static final List FIRST_VALUE = ImmutableList.of( - window(new ReflectionWindowFunctionSupplier<>("first_value", VARCHAR, ImmutableList.of(VARCHAR), FirstValueFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1)); + window(new ReflectionWindowFunctionSupplier(1, FirstValueFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1)); private static final List LAST_VALUE = ImmutableList.of( - window(new ReflectionWindowFunctionSupplier<>("last_value", VARCHAR, ImmutableList.of(VARCHAR), LastValueFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1)); + window(new ReflectionWindowFunctionSupplier(1, LastValueFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1)); private static final List NTH_VALUE = ImmutableList.of( - window(new ReflectionWindowFunctionSupplier<>("nth_value", VARCHAR, ImmutableList.of(VARCHAR, BIGINT), NthValueFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1, 3)); + window(new ReflectionWindowFunctionSupplier(2, NthValueFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1, 3)); private static final List LAG = ImmutableList.of( - window(new ReflectionWindowFunctionSupplier<>("lag", VARCHAR, ImmutableList.of(VARCHAR, BIGINT, VARCHAR), LagFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1, 3, 4)); + window(new ReflectionWindowFunctionSupplier(3, LagFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1, 3, 4)); private static final List LEAD = ImmutableList.of( - window(new ReflectionWindowFunctionSupplier<>("lead", VARCHAR, ImmutableList.of(VARCHAR, BIGINT, VARCHAR), LeadFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1, 3, 4)); + window(new ReflectionWindowFunctionSupplier(3, LeadFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1, 3, 4)); private ExecutorService executor; private ScheduledExecutorService scheduledExecutor;