beyond1920 commented on a change in pull request #8203:
[Flink-12208][table-planner-blink] Support translation from StreamExecSort /
TemporalSort / SortLimit/ Limit to StreamTransformation.
URL: https://github.com/apache/flink/pull/8203#discussion_r277203098
##########
File path:
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala
##########
@@ -72,4 +90,78 @@ class StreamExecLimit(
.item("fetch", RelExplainUtil.fetchToString(fetch))
}
+ //~ ExecNode methods
-----------------------------------------------------------
+
+ override def getInputNodes: util.List[ExecNode[StreamTableEnvironment, _]] =
{
+ List(getInput.asInstanceOf[ExecNode[StreamTableEnvironment, _]])
+ }
+
+ override protected def translateToPlanInternal(
+ tableEnv: StreamTableEnvironment): StreamTransformation[BaseRow] = {
+ if (fetch == null) {
+ throw new TableException(
+ "FETCH is missed, which on streaming table is not supported currently")
+ }
+ val inputRowTypeInfo =
FlinkTypeFactory.toInternalRowType(getInput.getRowType).toTypeInfo
+ val generateRetraction = StreamExecRetractionRules.isAccRetract(this)
+ val tableConfig = tableEnv.getConfig
+ val minIdleStateRetentionTime = tableConfig.getMinIdleStateRetentionTime
+ val maxIdleStateRetentionTime = tableConfig.getMaxIdleStateRetentionTime
+
+ // rankStart begin with 1
+ val rankRange = new ConstantRankRange(limitStart + 1, limitEnd)
+ val rankType = RankType.ROW_NUMBER
+ val outputRankNumber = false
+ // Use TopNFunction underlying StreamExecLimit currently
+ val sortKeySelector = NullBinaryRowKeySelector.INSTANCE
+ val sortKeyComparator = ComparatorCodeGenerator.gen(
+ tableConfig, "AlwaysEqualsComparator", Array(), Array(), Array(),
Array())
+
+ val processFunction = if (generateRetraction) {
+ val cacheSize =
tableConfig.getConf.getLong(TableConfigOptions.SQL_EXEC_TOPN_CACHE_SIZE)
+ new AppendTopNFunction(
+ minIdleStateRetentionTime,
+ maxIdleStateRetentionTime,
+ inputRowTypeInfo,
+ sortKeyComparator,
+ sortKeySelector,
+ rankType,
+ rankRange,
+ generateRetraction,
+ outputRankNumber,
+ cacheSize)
+ } else {
+ val equaliserCodeGen = new
EqualiserCodeGenerator(inputRowTypeInfo.getInternalTypes)
+ val generatedEqualiser =
equaliserCodeGen.generateRecordEqualiser("LimitValueEqualiser")
+ new RetractTopNFunction(
+ minIdleStateRetentionTime,
+ maxIdleStateRetentionTime,
+ inputRowTypeInfo,
+ sortKeyComparator,
+ sortKeySelector,
+ rankType,
+ rankRange,
+ generatedEqualiser,
+ generateRetraction,
+ outputRankNumber)
+ }
+ val operator = new KeyedProcessOperator(processFunction)
+ processFunction.setKeyContext(operator)
+
+ val inputTransform = getInputNodes.get(0).translateToPlan(tableEnv)
+ .asInstanceOf[StreamTransformation[BaseRow]]
+
+ val outputRowTypeInfo =
FlinkTypeFactory.toInternalRowType(getRowType).toTypeInfo
+ val ret = new OneInputTransformation(
+ inputTransform,
+ s"Limit(offset: $limitStart, fetch: ${fetchToString(fetch)})",
+ operator,
+ outputRowTypeInfo,
+ inputTransform.getParallelism)
Review comment:
Since we limit the exchange must be singleton before StreamExecLimit,
SortLimit in the rules, is it still necessary to force parallelism to 1 here.
----------------------------------------------------------------
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:
[email protected]
With regards,
Apache Git Services