You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/24 05:06:07 UTC

[GitHub] [flink] leonardBang commented on a change in pull request #13721: [FLINK-19694][table] Support Upsert ChangelogMode for ScanTableSource

leonardBang commented on a change in pull request #13721:
URL: https://github.com/apache/flink/pull/13721#discussion_r511321199



##########
File path: flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml
##########
@@ -307,7 +307,7 @@ Join(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], leftInputSpec=[NoUn
 :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[single])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
-      +- GroupAggregate(select=[MIN(i) AS m])
+      +- GroupAggregate(select=[MIN_RETRACT(i) AS m])

Review comment:
       Why the function change for existed test?

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecUpsertMaterialize.scala
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.plan.nodes.physical.stream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.flink.api.dag.Transformation
+import org.apache.flink.streaming.api.operators.KeyedProcessOperator
+import org.apache.flink.streaming.api.transformations.OneInputTransformation
+import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.data.RowData
+import org.apache.flink.table.planner.delegation.StreamPlanner
+import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode}
+import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils, KeySelectorUtil}
+import org.apache.flink.table.runtime.operators.bundle.KeyedMapBundleOperator
+import org.apache.flink.table.runtime.operators.deduplicate.{DeduplicateKeepLastRowFunction, MiniBatchDeduplicateKeepLastRowFunction}
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo
+
+import java.util
+
+import scala.collection.JavaConversions._
+
+/**
+ * Stream physical RelNode which materializes an upsert stream where where each data record

Review comment:
       ```suggestion
    * Stream physical RelNode which materializes an upsert stream where each record
   ```

##########
File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala
##########
@@ -130,6 +130,28 @@ class TableScanTest extends TableTestBase {
     util.verifyPlan("SELECT * FROM src WHERE a > 1")
   }
 
+  @Test
+  def testScanOnUpsertSource(): Unit = {
+    util.addTable(
+      """
+        |CREATE TABLE src (
+        |  id STRING,
+        |  a INT,
+        |  b DOUBLE,
+        |  PRIMARY KEY (id) NOT ENFORCED
+        |) WITH (
+        |  'connector' = 'values',
+        |  'bounded' = 'true',
+        |  'changelog-mode' = 'UA,D'
+        |)
+      """.stripMargin)
+    thrown.expect(classOf[UnsupportedOperationException])
+    thrown.expectMessage(

Review comment:
       The test failed, please update the exception message.

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecUpsertMaterialize.scala
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.plan.nodes.physical.stream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.flink.api.dag.Transformation
+import org.apache.flink.streaming.api.operators.KeyedProcessOperator
+import org.apache.flink.streaming.api.transformations.OneInputTransformation
+import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.data.RowData
+import org.apache.flink.table.planner.delegation.StreamPlanner
+import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode}
+import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils, KeySelectorUtil}
+import org.apache.flink.table.runtime.operators.bundle.KeyedMapBundleOperator
+import org.apache.flink.table.runtime.operators.deduplicate.{DeduplicateKeepLastRowFunction, MiniBatchDeduplicateKeepLastRowFunction}
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo
+
+import java.util
+
+import scala.collection.JavaConversions._
+
+/**
+ * Stream physical RelNode which materializes an upsert stream where where each data record
+ * represents an update or delete event. The materialize node will merge upsert/delete records
+ * and output normalized changelog stream that contains INSERT/UPDATE_BEFORE/UPDATE_AFTER/DELETE
+ * records.
+ */
+class StreamExecUpsertMaterialize(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    input: RelNode,
+    val uniqueKeys: Array[Int])
+  extends SingleRel(cluster, traitSet, input)
+  with StreamPhysicalRel
+  with StreamExecNode[RowData] {
+
+  override def requireWatermark: Boolean = false
+
+  override def deriveRowType(): RelDataType = getInput.getRowType
+
+  override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
+    new StreamExecUpsertMaterialize(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      uniqueKeys)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    val fieldNames = getRowType.getFieldNames
+    super.explainTerms(pw)
+      .item("key", uniqueKeys.map(fieldNames.get).mkString(", "))
+  }
+
+  //~ ExecNode methods -----------------------------------------------------------
+
+  override def getInputNodes: util.List[ExecNode[StreamPlanner, _]] = {
+    List(getInput.asInstanceOf[ExecNode[StreamPlanner, _]])
+  }
+
+  override def replaceInputNode(
+    ordinalInParent: Int,
+    newInputNode: ExecNode[StreamPlanner, _]): Unit = {
+    replaceInput(ordinalInParent, newInputNode.asInstanceOf[RelNode])
+  }
+
+  override protected def translateToPlanInternal(
+      planner: StreamPlanner): Transformation[RowData] = {
+
+    val inputTransform = getInputNodes.get(0).translateToPlan(planner)
+      .asInstanceOf[Transformation[RowData]]
+
+    val rowTypeInfo = inputTransform.getOutputType.asInstanceOf[InternalTypeInfo[RowData]]
+    val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this)
+    val tableConfig = planner.getTableConfig
+    val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
+      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
+    val operator = if (isMiniBatchEnabled) {
+      val exeConfig = planner.getExecEnv.getConfig
+      val rowSerializer = rowTypeInfo.createSerializer(exeConfig)
+      val processFunction = new MiniBatchDeduplicateKeepLastRowFunction(
+        rowTypeInfo,
+        generateUpdateBefore,
+        true,   // generateInsert
+        false,  // inputInsertOnly
+        rowSerializer,
+        // disable state ttl, the upsert materialize should keep all state to have data integrity
+        // we can enable state ttl if this is really needed in some cases

Review comment:
       Looks like this conflicts with global TTL setting? I understand the motivation, but it may confuse user.




----------------------------------------------------------------
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:
users@infra.apache.org