openinx commented on a change in pull request #2886:
URL: https://github.com/apache/iceberg/pull/2886#discussion_r679753863



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -246,52 +248,71 @@ public Builder uidPrefix(String newPrefix) {
         }
       }
 
-      // Find out the equality field id list based on the user-provided 
equality field column names.
-      List<Integer> equalityFieldIds = Lists.newArrayList();
-      if (equalityFieldColumns != null && equalityFieldColumns.size() > 0) {
-        for (String column : equalityFieldColumns) {
-          org.apache.iceberg.types.Types.NestedField field = 
table.schema().findField(column);
-          Preconditions.checkNotNull(field, "Missing required equality field 
column '%s' in table schema %s",
-              column, table.schema());
-          equalityFieldIds.add(field.fieldId());
-        }
-      }
-
       // Convert the requested flink table schema to flink row type.
       RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema);
 
       // Distribute the records from input data stream based on the 
write.distribution-mode.
       rowDataInput = distributeDataStream(rowDataInput, table.properties(), 
table.spec(), table.schema(), flinkRowType);
 
-      // Chain the iceberg stream writer and committer operator.
-      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, 
flinkRowType, equalityFieldIds);
-      IcebergFilesCommitter filesCommitter = new 
IcebergFilesCommitter(tableLoader, overwrite);
+      // Add parallel writers that append rows to files
+      SingleOutputStreamOperator<WriteResult> writerStream = 
appendWriter(rowDataInput, flinkRowType);
 
-      this.writeParallelism = writeParallelism == null ? 
rowDataInput.getParallelism() : writeParallelism;
+      // Add single-parallelism committer that commits files
+      // after successful checkpoint or end of input
+      SingleOutputStreamOperator<Void> committerStream = 
appendCommitter(writerStream);
 
-      SingleOutputStreamOperator<WriteResult> writerStream = rowDataInput
-          .transform(ICEBERG_STREAM_WRITER_NAME, 
TypeInformation.of(WriteResult.class), streamWriter)
-          .setParallelism(writeParallelism);
+      // Add dummy discard sink
+      return appendDummySink(committerStream);
+    }
+
+    private String getOperatorName(String suffix) {
+      return uidPrefix != null ? uidPrefix + "-" + suffix : suffix;
+    }
+
+    private DataStreamSink<RowData> 
appendDummySink(SingleOutputStreamOperator<Void> committerStream) {
+      DataStreamSink<RowData> resultStream = committerStream
+          .addSink(new DiscardingSink())
+          .name(getOperatorName(String.format("IcebergSink %s", 
this.table.name())))
+          .setParallelism(1);
       if (uidPrefix != null) {
-        writerStream = writerStream.uid(uidPrefix + "-writer");
+        resultStream = resultStream.uid(uidPrefix + "-dummysink");
       }
+      return resultStream;
+    }

Review comment:
       I think the `build()` will need to be changed to return a 
`DataStreamSink<Void>` because the stream won't actually emit any real 
`RowData`.  But as it's a public API,  will this break the upstream flink 
datastream job?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to