twalthr commented on code in PR #27426:
URL: https://github.com/apache/flink/pull/27426#discussion_r2735154164


##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/InsertConflictStrategy.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.api;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * Defines the conflict resolution strategies for INSERT INTO statements when 
the query's upsert key
+ * differs from the target table's primary key.
+ *
+ * <p>These strategies are used with the ON CONFLICT clause:

Review Comment:
   nit: an example INSERT INTO with an upsert key and primary key could be 
useful here. not sure if everybody knows what an upsert key is.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java:
##########
@@ -336,13 +337,35 @@ private Operation convertSqlInsert(RichSqlInsert insert) {
         int[][] columnIndices =
                 getTargetColumnIndices(contextResolvedTable, 
insert.getTargetColumnList());
 
+        // Convert parser conflict strategy to API conflict strategy
+        org.apache.flink.table.api.InsertConflictStrategy conflictStrategy =

Review Comment:
   nit: import



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala:
##########
@@ -120,12 +124,17 @@ class StreamPhysicalSink(
         .filter(strategy => 
!strategy.equals(SinkUpsertMaterializeStrategy.LEGACY))
         .orElse(null),
       UpsertKeyUtil.getSmallestKey(inputUpsertKeys),
+      conflictStrategy,
       getRelDetailedDescription)
   }
 
   override def explainTerms(pw: RelWriter): RelWriter = {
     super
       .explainTerms(pw)
       .itemIf("upsertMaterialize", "true", upsertMaterialize)
+      .itemIf(
+        "conflictStrategy",
+        conflictStrategy,
+        conflictStrategy != null && 
!InsertConflictStrategy.deduplicate().equals(conflictStrategy))

Review Comment:
   why not?
   ```suggestion
           conflictStrategy != null)
   ```



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala:
##########
@@ -1069,7 +1076,26 @@ class FlinkChangelogModeInferenceProgram extends 
FlinkOptimizeProgram[StreamOpti
           // if input has updates and primary key != upsert key (upsert key 
can be null) we should
           // enable upsertMaterialize. An optimize is: do not enable 
upsertMaterialize when sink
           // pk(s) contains input changeLogUpsertKeys
-          changeLogUpsertKeys == null || 
!changeLogUpsertKeys.exists(pks.contains)
+          val upsertKeyDiffersFromPk =
+            changeLogUpsertKeys == null || 
!changeLogUpsertKeys.exists(pks.contains)
+
+          // Validate that ON CONFLICT is specified when upsert key differs 
from primary key
+          val requireOnConflict =
+            
tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SINK_REQUIRE_ON_CONFLICT)
+          if (requireOnConflict && upsertKeyDiffersFromPk && 
sink.conflictStrategy == null) {
+            throw new ValidationException(
+              "The query has an upsert key that differs from the primary key 
of the sink table " +

Review Comment:
   give more insights by printing the upsert and primary key



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to