-
Notifications
You must be signed in to change notification settings - Fork 13.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-19449][table-planner] LEAD/LAG cannot work correctly in stream…
…ing mode This closes #15747
- Loading branch information
1 parent
6fa954b
commit 89aacaf
Showing
3 changed files
with
293 additions
and
0 deletions.
There are no files selected for viewing
163 changes: 163 additions & 0 deletions
163
...k/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LagAggFunction.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,163 @@ | ||
/* | ||
* 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 { | ||
if (offset < 0) { | ||
throw new TableException(String.format("Offset(%d) should be positive.", offset)); | ||
} | ||
|
||
acc.offset = offset; | ||
accumulate(acc, value); | ||
} | ||
|
||
public void accumulate(LagAcc<T> acc, T value, int offset, T defaultValue) throws Exception { | ||
acc.defaultValue = defaultValue; | ||
accumulate(acc, value, offset); | ||
} | ||
|
||
public void resetAccumulator(LagAcc<T> acc) throws Exception { | ||
acc.offset = 1; | ||
acc.defaultValue = null; | ||
acc.buffer.clear(); | ||
} | ||
|
||
@Override | ||
public T getValue(LagAcc<T> acc) { | ||
if (acc.buffer.size() < acc.offset + 1) { | ||
return acc.defaultValue; | ||
} else if (acc.buffer.size() == acc.offset + 1) { | ||
return acc.buffer.getFirst(); | ||
} else { | ||
throw new TableException("Too more elements: " + acc); | ||
} | ||
} | ||
|
||
@Override | ||
public LagAcc<T> createAccumulator() { | ||
return new LagAcc<>(); | ||
} | ||
|
||
/** Accumulator for LAG. */ | ||
public static class LagAcc<T> { | ||
public int offset = 1; | ||
public T defaultValue = null; | ||
public LinkedList<T> buffer = new LinkedList<>(); | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
LagAcc<?> lagAcc = (LagAcc<?>) o; | ||
return offset == lagAcc.offset | ||
&& Objects.equals(defaultValue, lagAcc.defaultValue) | ||
&& Objects.equals(buffer, lagAcc.buffer); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(offset, defaultValue, buffer); | ||
} | ||
} | ||
} |
62 changes: 62 additions & 0 deletions
62
...c/test/java/org/apache/flink/table/planner/functions/aggfunctions/LagAggFunctionTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,62 @@ | ||
/* | ||
* 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.table.data.StringData; | ||
import org.apache.flink.table.functions.AggregateFunction; | ||
import org.apache.flink.table.types.logical.CharType; | ||
import org.apache.flink.table.types.logical.IntType; | ||
import org.apache.flink.table.types.logical.LogicalType; | ||
import org.apache.flink.table.types.logical.VarCharType; | ||
|
||
import java.util.Arrays; | ||
import java.util.Collections; | ||
import java.util.List; | ||
|
||
import static org.apache.flink.table.data.StringData.fromString; | ||
|
||
/** Test for {@link LagAggFunction}. */ | ||
public class LagAggFunctionTest | ||
extends AggFunctionTestBase<StringData, LagAggFunction.LagAcc<StringData>> { | ||
|
||
@Override | ||
protected List<List<StringData>> getInputValueSets() { | ||
return Arrays.asList( | ||
Collections.singletonList(fromString("1")), | ||
Arrays.asList(fromString("1"), null), | ||
Arrays.asList(null, null), | ||
Arrays.asList(null, fromString("10"))); | ||
} | ||
|
||
@Override | ||
protected List<StringData> getExpectedResults() { | ||
return Arrays.asList(null, fromString("1"), null, null); | ||
} | ||
|
||
@Override | ||
protected AggregateFunction<StringData, LagAggFunction.LagAcc<StringData>> getAggregator() { | ||
return new LagAggFunction<>( | ||
new LogicalType[] {new VarCharType(), new IntType(), new CharType()}); | ||
} | ||
|
||
@Override | ||
protected Class<?> getAccClass() { | ||
return LagAggFunction.LagAcc.class; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters