cshuo commented on a change in pull request #15747:
URL: https://github.com/apache/flink/pull/15747#discussion_r619785598



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LagAggFunction.java
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.flink.table.planner.functions.aggfunctions;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.functions.AggregateFunction;
+import 
org.apache.flink.table.runtime.functions.aggregate.BuiltInAggregateFunction;
+import org.apache.flink.table.runtime.typeutils.InternalSerializers;
+import org.apache.flink.table.runtime.typeutils.LinkedListSerializer;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.utils.DataTypeUtils;
+
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+
+/** Lag {@link AggregateFunction}. */
+public class LagAggFunction<T> extends BuiltInAggregateFunction<T, 
LagAggFunction.LagAcc<T>> {
+
+    private final transient DataType[] valueDataTypes;
+
+    @SuppressWarnings("unchecked")
+    public LagAggFunction(LogicalType[] valueTypes) {
+        this.valueDataTypes =
+                Arrays.stream(valueTypes)
+                        .map(DataTypeUtils::toInternalDataType)
+                        .toArray(DataType[]::new);
+        if (valueDataTypes.length == 3
+                && valueDataTypes[2].getLogicalType().getTypeRoot() != 
LogicalTypeRoot.NULL) {
+            if (valueDataTypes[0].getConversionClass() != 
valueDataTypes[2].getConversionClass()) {
+                throw new TableException(
+                        String.format(
+                                "Please explicitly cast default value %s to 
%s.",
+                                valueDataTypes[2], valueDataTypes[1]));
+            }
+        }
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Planning
+    // 
--------------------------------------------------------------------------------------------
+
+    @Override
+    public List<DataType> getArgumentDataTypes() {
+        return Arrays.asList(valueDataTypes);
+    }
+
+    @Override
+    public DataType getAccumulatorDataType() {
+        return DataTypes.STRUCTURED(
+                LagAcc.class,
+                DataTypes.FIELD("offset", DataTypes.INT()),
+                DataTypes.FIELD("defaultValue", valueDataTypes[0]),
+                DataTypes.FIELD("buffer", getLinkedListType()));
+    }
+
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    private DataType getLinkedListType() {
+        TypeSerializer<T> serializer =
+                
InternalSerializers.create(getOutputDataType().getLogicalType());
+        return DataTypes.RAW(
+                LinkedList.class, (TypeSerializer) new 
LinkedListSerializer<>(serializer));
+    }
+
+    @Override
+    public DataType getOutputDataType() {
+        return valueDataTypes[0];
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Runtime
+    // 
--------------------------------------------------------------------------------------------
+
+    public void accumulate(LagAcc<T> acc, T value) throws Exception {
+        acc.buffer.add(value);
+        while (acc.buffer.size() > acc.offset + 1) {
+            acc.buffer.removeFirst();
+        }
+    }
+
+    public void accumulate(LagAcc<T> acc, T value, int offset) throws 
Exception {
+        acc.offset = offset;

Review comment:
       better validating offset > -1 and giving some meaningful exception 
message.

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LagAggFunction.java
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.flink.table.planner.functions.aggfunctions;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.functions.AggregateFunction;
+import 
org.apache.flink.table.runtime.functions.aggregate.BuiltInAggregateFunction;
+import org.apache.flink.table.runtime.typeutils.InternalSerializers;
+import org.apache.flink.table.runtime.typeutils.LinkedListSerializer;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.utils.DataTypeUtils;
+
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+
+/** Lag {@link AggregateFunction}. */
+public class LagAggFunction<T> extends BuiltInAggregateFunction<T, 
LagAggFunction.LagAcc<T>> {
+
+    private final transient DataType[] valueDataTypes;
+
+    @SuppressWarnings("unchecked")
+    public LagAggFunction(LogicalType[] valueTypes) {
+        this.valueDataTypes =
+                Arrays.stream(valueTypes)
+                        .map(DataTypeUtils::toInternalDataType)
+                        .toArray(DataType[]::new);
+        if (valueDataTypes.length == 3
+                && valueDataTypes[2].getLogicalType().getTypeRoot() != 
LogicalTypeRoot.NULL) {
+            if (valueDataTypes[0].getConversionClass() != 
valueDataTypes[2].getConversionClass()) {
+                throw new TableException(
+                        String.format(
+                                "Please explicitly cast default value %s to 
%s.",
+                                valueDataTypes[2], valueDataTypes[1]));
+            }
+        }
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Planning
+    // 
--------------------------------------------------------------------------------------------
+
+    @Override
+    public List<DataType> getArgumentDataTypes() {
+        return Arrays.asList(valueDataTypes);
+    }
+
+    @Override
+    public DataType getAccumulatorDataType() {
+        return DataTypes.STRUCTURED(
+                LagAcc.class,
+                DataTypes.FIELD("offset", DataTypes.INT()),
+                DataTypes.FIELD("defaultValue", valueDataTypes[0]),
+                DataTypes.FIELD("buffer", getLinkedListType()));
+    }
+
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    private DataType getLinkedListType() {
+        TypeSerializer<T> serializer =
+                
InternalSerializers.create(getOutputDataType().getLogicalType());
+        return DataTypes.RAW(
+                LinkedList.class, (TypeSerializer) new 
LinkedListSerializer<>(serializer));
+    }
+
+    @Override
+    public DataType getOutputDataType() {
+        return valueDataTypes[0];
+    }
+
+    // 
--------------------------------------------------------------------------------------------
+    // Runtime
+    // 
--------------------------------------------------------------------------------------------
+
+    public void accumulate(LagAcc<T> acc, T value) throws Exception {
+        acc.buffer.add(value);
+        while (acc.buffer.size() > acc.offset + 1) {
+            acc.buffer.removeFirst();
+        }
+    }
+
+    public void accumulate(LagAcc<T> acc, T value, int offset) throws 
Exception {
+        acc.offset = offset;
+        accumulate(acc, value);
+    }
+
+    public void accumulate(LagAcc<T> acc, T value, int offset, T defaultValue) 
throws Exception {

Review comment:
       ditto

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggFunctionFactory.scala
##########
@@ -52,7 +52,8 @@ import scala.collection.JavaConversions._
 class AggFunctionFactory(

Review comment:
       minor: 
   param in doc is miss matching.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to