Class SumWithRetractAggFunction
- java.lang.Object
-
- org.apache.flink.table.functions.UserDefinedFunction
-
- org.apache.flink.table.functions.DeclarativeAggregateFunction
-
- org.apache.flink.table.planner.functions.aggfunctions.SumWithRetractAggFunction
-
- All Implemented Interfaces:
Serializable
,FunctionDefinition
- Direct Known Subclasses:
SumWithRetractAggFunction.ByteSumWithRetractAggFunction
,SumWithRetractAggFunction.DecimalSumWithRetractAggFunction
,SumWithRetractAggFunction.DoubleSumWithRetractAggFunction
,SumWithRetractAggFunction.FloatSumWithRetractAggFunction
,SumWithRetractAggFunction.IntSumWithRetractAggFunction
,SumWithRetractAggFunction.LongSumWithRetractAggFunction
,SumWithRetractAggFunction.ShortSumWithRetractAggFunction
public abstract class SumWithRetractAggFunction extends DeclarativeAggregateFunction
built-in sum aggregate function with retraction.- See Also:
- Serialized Form
-
-
Nested Class Summary
Nested Classes Modifier and Type Class Description static class
SumWithRetractAggFunction.ByteSumWithRetractAggFunction
Built-in Byte Sum with retract aggregate function.static class
SumWithRetractAggFunction.DecimalSumWithRetractAggFunction
Built-in Decimal Sum with retract aggregate function.static class
SumWithRetractAggFunction.DoubleSumWithRetractAggFunction
Built-in Double Sum with retract aggregate function.static class
SumWithRetractAggFunction.FloatSumWithRetractAggFunction
Built-in Float Sum with retract aggregate function.static class
SumWithRetractAggFunction.IntSumWithRetractAggFunction
Built-in Int Sum with retract aggregate function.static class
SumWithRetractAggFunction.LongSumWithRetractAggFunction
Built-in Long Sum with retract aggregate function.static class
SumWithRetractAggFunction.ShortSumWithRetractAggFunction
Built-in Short Sum with retract aggregate function.
-
Constructor Summary
Constructors Constructor Description SumWithRetractAggFunction()
-
Method Summary
All Methods Instance Methods Abstract Methods Concrete Methods Modifier and Type Method Description Expression[]
accumulateExpressions()
Expressions for accumulating the mutable aggregation buffer based on an input row.protected UnresolvedCallExpression
adjustedMinus(Expression arg1, UnresolvedReferenceExpression arg2)
protected UnresolvedCallExpression
adjustedPlus(UnresolvedReferenceExpression arg1, UnresolvedReferenceExpression arg2)
UnresolvedReferenceExpression[]
aggBufferAttributes()
All fields of the aggregate buffer.DataType[]
getAggBufferTypes()
All types of the aggregate buffer.Expression
getValueExpression()
An expression which returns the final value for this aggregate function.Expression[]
initialValuesExpressions()
Expressions for initializing empty aggregation buffers.Expression[]
mergeExpressions()
A sequence of expressions for merging two aggregation buffers together.int
operandCount()
How many operands your function will deal with.Expression[]
retractExpressions()
Expressions for retracting the mutable aggregation buffer based on an input row.protected abstract Expression
zeroLiteral()
-
Methods inherited from class org.apache.flink.table.functions.DeclarativeAggregateFunction
getKind, getResultType, getTypeInference, mergeOperand, mergeOperands, operand, operands
-
Methods inherited from class org.apache.flink.table.functions.UserDefinedFunction
close, functionIdentifier, open, toString
-
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
-
Methods inherited from interface org.apache.flink.table.functions.FunctionDefinition
getRequirements, isDeterministic, supportsConstantFolding
-
-
-
-
Method Detail
-
operandCount
public int operandCount()
Description copied from class:DeclarativeAggregateFunction
How many operands your function will deal with.- Specified by:
operandCount
in classDeclarativeAggregateFunction
-
aggBufferAttributes
public UnresolvedReferenceExpression[] aggBufferAttributes()
Description copied from class:DeclarativeAggregateFunction
All fields of the aggregate buffer.- Specified by:
aggBufferAttributes
in classDeclarativeAggregateFunction
-
getAggBufferTypes
public DataType[] getAggBufferTypes()
Description copied from class:DeclarativeAggregateFunction
All types of the aggregate buffer.- Specified by:
getAggBufferTypes
in classDeclarativeAggregateFunction
-
initialValuesExpressions
public Expression[] initialValuesExpressions()
Description copied from class:DeclarativeAggregateFunction
Expressions for initializing empty aggregation buffers.- Specified by:
initialValuesExpressions
in classDeclarativeAggregateFunction
-
accumulateExpressions
public Expression[] accumulateExpressions()
Description copied from class:DeclarativeAggregateFunction
Expressions for accumulating the mutable aggregation buffer based on an input row.- Specified by:
accumulateExpressions
in classDeclarativeAggregateFunction
-
retractExpressions
public Expression[] retractExpressions()
Description copied from class:DeclarativeAggregateFunction
Expressions for retracting the mutable aggregation buffer based on an input row.- Specified by:
retractExpressions
in classDeclarativeAggregateFunction
-
mergeExpressions
public Expression[] mergeExpressions()
Description copied from class:DeclarativeAggregateFunction
A sequence of expressions for merging two aggregation buffers together. When defining these expressions, you can use the syntaxattributeName
andmergeOperand(attributeName)
to refer to the attributes corresponding to each of the buffers being merged.- Specified by:
mergeExpressions
in classDeclarativeAggregateFunction
-
getValueExpression
public Expression getValueExpression()
Description copied from class:DeclarativeAggregateFunction
An expression which returns the final value for this aggregate function.- Specified by:
getValueExpression
in classDeclarativeAggregateFunction
-
zeroLiteral
protected abstract Expression zeroLiteral()
-
adjustedPlus
protected UnresolvedCallExpression adjustedPlus(UnresolvedReferenceExpression arg1, UnresolvedReferenceExpression arg2)
-
adjustedMinus
protected UnresolvedCallExpression adjustedMinus(Expression arg1, UnresolvedReferenceExpression arg2)
-
-