|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.table.planner.functions.aggfunctions; |
| 20 | + |
| 21 | +import org.apache.flink.api.common.typeutils.TypeSerializer; |
| 22 | +import org.apache.flink.table.api.DataTypes; |
| 23 | +import org.apache.flink.table.api.TableException; |
| 24 | +import org.apache.flink.table.functions.AggregateFunction; |
| 25 | +import org.apache.flink.table.runtime.functions.aggregate.BuiltInAggregateFunction; |
| 26 | +import org.apache.flink.table.runtime.typeutils.InternalSerializers; |
| 27 | +import org.apache.flink.table.runtime.typeutils.LinkedListSerializer; |
| 28 | +import org.apache.flink.table.types.DataType; |
| 29 | +import org.apache.flink.table.types.logical.LogicalType; |
| 30 | +import org.apache.flink.table.types.logical.LogicalTypeRoot; |
| 31 | +import org.apache.flink.table.types.utils.DataTypeUtils; |
| 32 | + |
| 33 | +import java.util.Arrays; |
| 34 | +import java.util.LinkedList; |
| 35 | +import java.util.List; |
| 36 | +import java.util.Objects; |
| 37 | + |
| 38 | +/** Lag {@link AggregateFunction}. */ |
| 39 | +public class LagAggFunction<T> extends BuiltInAggregateFunction<T, LagAggFunction.LagAcc<T>> { |
| 40 | + |
| 41 | + private final transient DataType[] valueDataTypes; |
| 42 | + |
| 43 | + @SuppressWarnings("unchecked") |
| 44 | + public LagAggFunction(LogicalType[] valueTypes) { |
| 45 | + this.valueDataTypes = |
| 46 | + Arrays.stream(valueTypes) |
| 47 | + .map(DataTypeUtils::toInternalDataType) |
| 48 | + .toArray(DataType[]::new); |
| 49 | + if (valueDataTypes.length == 3 |
| 50 | + && valueDataTypes[2].getLogicalType().getTypeRoot() != LogicalTypeRoot.NULL) { |
| 51 | + if (valueDataTypes[0].getConversionClass() != valueDataTypes[2].getConversionClass()) { |
| 52 | + throw new TableException( |
| 53 | + String.format( |
| 54 | + "Please explicitly cast default value %s to %s.", |
| 55 | + valueDataTypes[2], valueDataTypes[1])); |
| 56 | + } |
| 57 | + } |
| 58 | + } |
| 59 | + |
| 60 | + // -------------------------------------------------------------------------------------------- |
| 61 | + // Planning |
| 62 | + // -------------------------------------------------------------------------------------------- |
| 63 | + |
| 64 | + @Override |
| 65 | + public List<DataType> getArgumentDataTypes() { |
| 66 | + return Arrays.asList(valueDataTypes); |
| 67 | + } |
| 68 | + |
| 69 | + @Override |
| 70 | + public DataType getAccumulatorDataType() { |
| 71 | + return DataTypes.STRUCTURED( |
| 72 | + LagAcc.class, |
| 73 | + DataTypes.FIELD("offset", DataTypes.INT()), |
| 74 | + DataTypes.FIELD("defaultValue", valueDataTypes[0]), |
| 75 | + DataTypes.FIELD("buffer", getLinkedListType())); |
| 76 | + } |
| 77 | + |
| 78 | + @SuppressWarnings({"unchecked", "rawtypes"}) |
| 79 | + private DataType getLinkedListType() { |
| 80 | + TypeSerializer<T> serializer = |
| 81 | + InternalSerializers.create(getOutputDataType().getLogicalType()); |
| 82 | + return DataTypes.RAW( |
| 83 | + LinkedList.class, (TypeSerializer) new LinkedListSerializer<>(serializer)); |
| 84 | + } |
| 85 | + |
| 86 | + @Override |
| 87 | + public DataType getOutputDataType() { |
| 88 | + return valueDataTypes[0]; |
| 89 | + } |
| 90 | + |
| 91 | + // -------------------------------------------------------------------------------------------- |
| 92 | + // Runtime |
| 93 | + // -------------------------------------------------------------------------------------------- |
| 94 | + |
| 95 | + public void accumulate(LagAcc<T> acc, T value) throws Exception { |
| 96 | + acc.buffer.add(value); |
| 97 | + while (acc.buffer.size() > acc.offset + 1) { |
| 98 | + acc.buffer.removeFirst(); |
| 99 | + } |
| 100 | + } |
| 101 | + |
| 102 | + public void accumulate(LagAcc<T> acc, T value, int offset) throws Exception { |
| 103 | + if (offset < 0) { |
| 104 | + throw new TableException(String.format("Offset(%d) should be positive.", offset)); |
| 105 | + } |
| 106 | + |
| 107 | + acc.offset = offset; |
| 108 | + accumulate(acc, value); |
| 109 | + } |
| 110 | + |
| 111 | + public void accumulate(LagAcc<T> acc, T value, int offset, T defaultValue) throws Exception { |
| 112 | + acc.defaultValue = defaultValue; |
| 113 | + accumulate(acc, value, offset); |
| 114 | + } |
| 115 | + |
| 116 | + public void resetAccumulator(LagAcc<T> acc) throws Exception { |
| 117 | + acc.offset = 1; |
| 118 | + acc.defaultValue = null; |
| 119 | + acc.buffer.clear(); |
| 120 | + } |
| 121 | + |
| 122 | + @Override |
| 123 | + public T getValue(LagAcc<T> acc) { |
| 124 | + if (acc.buffer.size() < acc.offset + 1) { |
| 125 | + return acc.defaultValue; |
| 126 | + } else if (acc.buffer.size() == acc.offset + 1) { |
| 127 | + return acc.buffer.getFirst(); |
| 128 | + } else { |
| 129 | + throw new TableException("Too more elements: " + acc); |
| 130 | + } |
| 131 | + } |
| 132 | + |
| 133 | + @Override |
| 134 | + public LagAcc<T> createAccumulator() { |
| 135 | + return new LagAcc<>(); |
| 136 | + } |
| 137 | + |
| 138 | + /** Accumulator for LAG. */ |
| 139 | + public static class LagAcc<T> { |
| 140 | + public int offset = 1; |
| 141 | + public T defaultValue = null; |
| 142 | + public LinkedList<T> buffer = new LinkedList<>(); |
| 143 | + |
| 144 | + @Override |
| 145 | + public boolean equals(Object o) { |
| 146 | + if (this == o) { |
| 147 | + return true; |
| 148 | + } |
| 149 | + if (o == null || getClass() != o.getClass()) { |
| 150 | + return false; |
| 151 | + } |
| 152 | + LagAcc<?> lagAcc = (LagAcc<?>) o; |
| 153 | + return offset == lagAcc.offset |
| 154 | + && Objects.equals(defaultValue, lagAcc.defaultValue) |
| 155 | + && Objects.equals(buffer, lagAcc.buffer); |
| 156 | + } |
| 157 | + |
| 158 | + @Override |
| 159 | + public int hashCode() { |
| 160 | + return Objects.hash(offset, defaultValue, buffer); |
| 161 | + } |
| 162 | + } |
| 163 | +} |
0 commit comments