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/25 15:15:19 UTC

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

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



##########
File path: flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml
##########
@@ -87,6 +129,39 @@ LogicalProject(a=[$0], b=[$1], c=[+($0, 1)], d=[TO_TIMESTAMP($1)], e=[my_udf($0)
       <![CDATA[
 Calc(select=[a, b, +(a, 1) AS c, TO_TIMESTAMP(b) AS d, my_udf(a) AS e])
 +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testTemporalJoinOnUpsertSource">
+    <Resource name="sql">
+      <![CDATA[
+SELECT o.currency, o.amount, r.rate, o.amount * r.rate
+FROM orders AS o LEFT JOIN rates_history FOR SYSTEM_TIME AS OF o.proctime AS r
+ON o.currency = r.currency
+]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(currency=[$1], amount=[$0], rate=[$4], EXPR$3=[*($0, $4)])
++- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{1, 2}])
+   :- LogicalProject(amount=[$0], currency=[$1], proctime=[PROCTIME()])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, orders]])
+   +- LogicalFilter(condition=[=($cor0.currency, $0)])
+      +- LogicalSnapshot(period=[$cor0.proctime])
+         +- LogicalTableScan(table=[[default_catalog, default_database, rates_history]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[currency, amount, rate, *(amount, rate) AS EXPR$3], changelogMode=[I])
++- TemporalJoin(joinType=[LeftOuterJoin], where=[AND(=(currency, currency0), __TEMPORAL_JOIN_CONDITION(proctime, __TEMPORAL_JOIN_LEFT_KEY(currency), __TEMPORAL_JOIN_RIGHT_KEY(currency0)))], select=[amount, currency, proctime, currency0, rate], changelogMode=[I])
+   :- Exchange(distribution=[hash[currency]], changelogMode=[I])
+   :  +- Calc(select=[amount, currency, PROCTIME() AS proctime], changelogMode=[I])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[amount, currency], changelogMode=[I])
+   +- Exchange(distribution=[hash[currency]], changelogMode=[I,UB,UA,D])
+      +- UpsertMaterialize(key=[currency], changelogMode=[I,UB,UA,D])

Review comment:
       does `UpsertMaterialize` produce  `UB` message ?

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sources/DynamicSourceUtils.java
##########
@@ -241,30 +241,31 @@ private static void validateScanSource(
 		validateWatermarks(sourceIdentifier, schema);
 
 		if (isStreamingMode) {
-			validateScanSourceForStreaming(sourceIdentifier, scanSource, changelogMode);
+			validateScanSourceForStreaming(sourceIdentifier, schema, scanSource, changelogMode);
 		} else {
 			validateScanSourceForBatch(sourceIdentifier, changelogMode, provider);
 		}
 	}
 
 	private static void validateScanSourceForStreaming(
 			ObjectIdentifier sourceIdentifier,
+			TableSchema schema,
 			ScanTableSource scanSource,
 			ChangelogMode changelogMode) {
 		// sanity check for produced ChangelogMode
 		final boolean hasUpdateBefore = changelogMode.contains(RowKind.UPDATE_BEFORE);
 		final boolean hasUpdateAfter = changelogMode.contains(RowKind.UPDATE_AFTER);
 		if (!hasUpdateBefore && hasUpdateAfter) {
 			// only UPDATE_AFTER
-			throw new TableException(
-				String.format(
-					"Unsupported source for table '%s'. Currently, a %s doesn't support a changelog which contains " +
-						"UPDATE_AFTER but no UPDATE_BEFORE. Please adapt the implementation of class '%s'.",
-					sourceIdentifier.asSummaryString(),
-					ScanTableSource.class.getSimpleName(),
-					scanSource.getClass().getName()
-				)
-			);
+			if (!schema.getPrimaryKey().isPresent()) {
+				throw new TableException(
+					String.format(
+						"Table '%s' produces a changelog stream contains UPDATE_AFTER no UPDATE_BEFORE, " +

Review comment:
       `which contains only UPDATE_AFTER, no UPDATE_BEFORE.` ?

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecTableSourceScanRule.scala
##########
@@ -56,12 +66,57 @@ class StreamExecTableSourceScanRule
   def convert(rel: RelNode): RelNode = {
     val scan = rel.asInstanceOf[FlinkLogicalTableSourceScan]
     val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.STREAM_PHYSICAL)
-
-    new StreamExecTableSourceScan(
+    val newScan = new StreamExecTableSourceScan(
       rel.getCluster,
       traitSet,
-      scan.getTable.asInstanceOf[TableSourceTable]
-    )
+      scan.getTable.asInstanceOf[TableSourceTable])
+
+    val table = scan.getTable.asInstanceOf[TableSourceTable]
+    val tableSource = table.tableSource.asInstanceOf[ScanTableSource]
+    val changelogMode = tableSource.getChangelogMode
+    if (changelogMode.contains(RowKind.UPDATE_AFTER) &&
+        !changelogMode.contains(RowKind.UPDATE_BEFORE)) {
+      // generate upsert materialize node for upsert source
+      val primaryKey = table.catalogTable.getSchema.getPrimaryKey
+      if (!primaryKey.isPresent) {
+        throw new TableException(s"Table '${table.tableIdentifier.asSummaryString()}' produces" +
+          " a changelog stream contains UPDATE_AFTER but no UPDATE_BEFORE," +
+          " this requires to define primary key on the table.")
+      }
+      val keyFields = primaryKey.get().getColumns
+      val inputFieldNames = newScan.getRowType.getFieldNames
+      val primaryKeyIndices = getPrimaryKeyIndices(inputFieldNames, keyFields)
+      val requiredDistribution = FlinkRelDistribution.hash(primaryKeyIndices, requireStrict = true)
+      val requiredTraitSet = rel.getCluster.getPlanner.emptyTraitSet()
+        .replace(requiredDistribution)
+        .replace(FlinkConventions.STREAM_PHYSICAL)
+      val newInput: RelNode = RelOptRule.convert(newScan, requiredTraitSet)
+
+      new StreamExecUpsertMaterialize(

Review comment:
       If the primary key fields are not required, we should add a Calc here to reduce output fields.

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniqueness.scala
##########
@@ -304,6 +304,14 @@ class FlinkRelMdColumnUniqueness private extends MetadataHandler[BuiltInMetadata
     columns != null && util.Arrays.equals(columns.toArray, rel.getUniqueKeys)
   }
 
+  def areColumnsUnique(
+      rel: StreamExecUpsertMaterialize,
+      mq: RelMetadataQuery,
+      columns: ImmutableBitSet,
+      ignoreNulls: Boolean): JBoolean = {
+    columns != null && util.Arrays.equals(columns.toArray, rel.uniqueKeys)

Review comment:
       is the values of `rel.uniqueKeys` in order ?




----------------------------------------------------------------
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