KurtYoung commented on a change in pull request #10316:
[FLINK-14624][table-blink] Support computed column as rowtime attribute
URL: https://github.com/apache/flink/pull/10316#discussion_r353631556
##########
File path:
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
##########
@@ -62,37 +63,36 @@ class StreamExecWatermarkAssigner(
override def requireWatermark: Boolean = false
- override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]):
RelNode = {
- new StreamExecWatermarkAssigner(
- cluster,
- traitSet,
- inputs.get(0),
- rowtimeFieldIndex,
- watermarkDelay)
+ override def copy(
+ traitSet: RelTraitSet,
+ input: RelNode,
+ rowtime: Int,
+ watermark: RexNode): RelNode = {
+ new StreamExecWatermarkAssigner(cluster, traitSet, input, rowtime,
watermark)
}
+ /**
+ * Fully override this method to have a better display name of this RelNode.
+ */
override def explainTerms(pw: RelWriter): RelWriter = {
- val miniBatchInterval =
traits.getTrait(MiniBatchIntervalTraitDef.INSTANCE).getMiniBatchInterval
-
- val value = if (miniBatchInterval.mode == MiniBatchMode.None ||
- miniBatchInterval.interval == 0) {
- // 1. redundant watermark definition in DDL
- // 2. existing window aggregate
- // 3. operator requiring watermark, but minibatch is not enabled
- "None"
- } else if (miniBatchInterval.mode == MiniBatchMode.ProcTime) {
- val tableConfig =
cluster.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
- val miniBatchLatency = getMillisecondFromConfigDuration(tableConfig,
- ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY)
- Preconditions.checkArgument(miniBatchLatency > 0,
- "MiniBatch latency must be greater that 0 ms.", null)
- s"Proctime, ${miniBatchLatency}ms"
- } else if (miniBatchInterval.mode == MiniBatchMode.RowTime) {
- s"Rowtime, ${miniBatchInterval.interval}ms"
- } else {
- throw new TableException(s"Unsupported mode: $miniBatchInterval")
- }
- super.explainTerms(pw).item("miniBatchInterval", value)
+ val inFieldNames = inputRel.getRowType.getFieldNames.toList
+ val rowtimeFieldName = inFieldNames(rowtimeFieldIndex)
+ val miniBatchInterval = traits
Review comment:
It seems watermark assigner is influenced heavily by mini batch assigner,
I'm not sure whether this is right way to creating physical operators. Normally
all physical operators should be independent from each other.
----------------------------------------------------------------
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