-
Notifications
You must be signed in to change notification settings - Fork 13.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-19449][table-planner] LEAD/LAG cannot work correctly in streaming mode #15747
Changes from all commits
48526a1
72bd763
6fa954b
89aacaf
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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 { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ditto There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This method will invoke above method. |
||
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); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -45,14 +45,16 @@ import scala.collection.JavaConversions._ | |
* as subclasses of [[SqlAggFunction]] in Calcite but not as [[BridgingSqlAggFunction]]. The factory | ||
* returns [[DeclarativeAggregateFunction]] or [[BuiltInAggregateFunction]]. | ||
* | ||
* @param inputType the input rel data type | ||
* @param orderKeyIdx the indexes of order key (null when is not over agg) | ||
* @param needRetraction true if need retraction | ||
* @param inputRowType the input row type | ||
* @param orderKeyIndexes the indexes of order key (null when is not over agg) | ||
* @param aggCallNeedRetractions true if need retraction | ||
* @param isBounded true if the source is bounded source | ||
*/ | ||
class AggFunctionFactory( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. minor: |
||
inputRowType: RowType, | ||
orderKeyIndexes: Array[Int], | ||
aggCallNeedRetractions: Array[Boolean]) { | ||
aggCallNeedRetractions: Array[Boolean], | ||
isBounded: Boolean) { | ||
|
||
/** | ||
* The entry point to create an aggregate function from the given [[AggregateCall]]. | ||
|
@@ -94,8 +96,12 @@ class AggFunctionFactory( | |
case a: SqlRankFunction if a.getKind == SqlKind.DENSE_RANK => | ||
createDenseRankAggFunction(argTypes) | ||
|
||
case _: SqlLeadLagAggFunction => | ||
createLeadLagAggFunction(argTypes, index) | ||
case func: SqlLeadLagAggFunction => | ||
if (isBounded) { | ||
createBatchLeadLagAggFunction(argTypes, index) | ||
} else { | ||
createStreamLeadLagAggFunction(func, argTypes, index) | ||
} | ||
|
||
case _: SqlSingleValueAggFunction => | ||
createSingleValueAggFunction(argTypes) | ||
|
@@ -328,7 +334,22 @@ class AggFunctionFactory( | |
} | ||
} | ||
|
||
private def createLeadLagAggFunction( | ||
private def createStreamLeadLagAggFunction( | ||
func: SqlLeadLagAggFunction, | ||
argTypes: Array[LogicalType], | ||
index: Int): UserDefinedFunction = { | ||
if (func.getKind == SqlKind.LEAD) { | ||
throw new TableException("LEAD Function is not supported in stream mode.") | ||
} | ||
|
||
if (aggCallNeedRetractions(index)) { | ||
throw new TableException("LAG Function with retraction is not supported in stream mode.") | ||
} | ||
|
||
new LagAggFunction(argTypes) | ||
} | ||
|
||
private def createBatchLeadLagAggFunction( | ||
argTypes: Array[LogicalType], index: Int): UserDefinedFunction = { | ||
argTypes(0).getTypeRoot match { | ||
case TINYINT => | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
better validating offset > -1 and giving some meaningful exception message.