stevenzwu commented on code in PR #12071:
URL: https://github.com/apache/iceberg/pull/12071#discussion_r2101587781


##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java:
##########
@@ -645,72 +711,135 @@ private DataStream<RowData> 
distributeDataStream(DataStream<RowData> input) {
     DistributionMode mode = flinkWriteConf.distributionMode();
     Schema schema = table.schema();
     PartitionSpec spec = table.spec();
+    SortOrder sortOrder = table.sortOrder();
+
     LOG.info("Write distribution mode is '{}'", mode.modeName());
     switch (mode) {
       case NONE:
-        if (equalityFieldIds.isEmpty()) {
-          return input;
-        } else {
-          LOG.info("Distribute rows by equality fields, because there are 
equality fields set");
-          return input.keyBy(new EqualityFieldKeySelector(schema, 
flinkRowType, equalityFieldIds));
-        }
-
+        return distributeDataStreamByNoneDistributionMode(input, schema);
       case HASH:
-        if (equalityFieldIds.isEmpty()) {
-          if (table.spec().isUnpartitioned()) {
-            LOG.warn(
-                "Fallback to use 'none' distribution mode, because there are 
no equality fields set "
-                    + "and table is unpartitioned");
-            return input;
-          } else {
-            if (BucketPartitionerUtil.hasOneBucketField(spec)) {
-              return input.partitionCustom(
-                  new BucketPartitioner(spec),
-                  new BucketPartitionKeySelector(spec, schema, flinkRowType));
-            } else {
-              return input.keyBy(new PartitionKeySelector(spec, schema, 
flinkRowType));
-            }
-          }
-        } else {
-          if (spec.isUnpartitioned()) {
-            LOG.info(
-                "Distribute rows by equality fields, because there are 
equality fields set "
-                    + "and table is unpartitioned");
-            return input.keyBy(
-                new EqualityFieldKeySelector(schema, flinkRowType, 
equalityFieldIds));
-          } else {
-            for (PartitionField partitionField : spec.fields()) {
-              Preconditions.checkState(
-                  equalityFieldIds.contains(partitionField.sourceId()),
-                  "In 'hash' distribution mode with equality fields set, 
partition field '%s' "
-                      + "should be included in equality fields: '%s'",
-                  partitionField,
-                  equalityFieldColumns);
-            }
-            return input.keyBy(new PartitionKeySelector(spec, schema, 
flinkRowType));
-          }
-        }
-
+        return distributeDataStreamByHashDistributionMode(input, schema, spec);
       case RANGE:
-        if (equalityFieldIds.isEmpty()) {
-          LOG.warn(
-              "Fallback to use 'none' distribution mode, because there are no 
equality fields set "
-                  + "and {}=range is not supported yet in flink",
-              WRITE_DISTRIBUTION_MODE);
-          return input;
-        } else {
-          LOG.info(
-              "Distribute rows by equality fields, because there are equality 
fields set "
-                  + "and{}=range is not supported yet in flink",
-              WRITE_DISTRIBUTION_MODE);
-          return input.keyBy(new EqualityFieldKeySelector(schema, 
flinkRowType, equalityFieldIds));
-        }
-
+        return distributeDataStreamByRangeDistributionMode(input, schema, 
spec, sortOrder);
       default:
         throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + 
": " + mode);
     }
   }
 
+  private DataStream<RowData> distributeDataStreamByNoneDistributionMode(
+      DataStream<RowData> input, Schema iSchema) {
+    if (equalityFieldIds.isEmpty()) {
+      return input;
+    } else {
+      LOG.info("Distribute rows by equality fields, because there are equality 
fields set");
+      return input.keyBy(new EqualityFieldKeySelector(iSchema, flinkRowType, 
equalityFieldIds));
+    }
+  }
+
+  private DataStream<RowData> distributeDataStreamByHashDistributionMode(
+      DataStream<RowData> input, Schema iSchema, PartitionSpec partitionSpec) {
+    if (equalityFieldIds.isEmpty()) {
+      if (partitionSpec.isUnpartitioned()) {
+        LOG.warn(
+            "Fallback to use 'none' distribution mode, because there are no 
equality fields set "
+                + "and table is unpartitioned");
+        return input;
+      } else {
+        return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, 
flinkRowType));
+      }
+    } else {
+      if (partitionSpec.isUnpartitioned()) {
+        LOG.info(
+            "Distribute rows by equality fields, because there are equality 
fields set "
+                + "and table is unpartitioned");
+        return input.keyBy(new EqualityFieldKeySelector(iSchema, flinkRowType, 
equalityFieldIds));
+      } else {
+        for (PartitionField partitionField : partitionSpec.fields()) {
+          Preconditions.checkState(
+              equalityFieldIds.contains(partitionField.sourceId()),
+              "In 'hash' distribution mode with equality fields set, partition 
field '%s' "
+                  + "should be included in equality fields: '%s'",
+              partitionField,
+              equalityFieldColumns);
+        }
+        return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, 
flinkRowType));
+      }
+    }
+  }
+
+  private DataStream<RowData> distributeDataStreamByRangeDistributionMode(
+      DataStream<RowData> input,
+      Schema iSchema,
+      PartitionSpec partitionSpec,
+      SortOrder sortOrderParam) {
+
+    int writerParallelism =

Review Comment:
   this logic should be applied to the writer parallelism for v2 sink. 
   
   if write parallelism is not configured, the v1 sink default the writer 
parallelism to the input parallelism to promote chaining. Want to confirm if 
that is the case for v2 sink? from reading the code, I thought v2 sink will 
default the writer parallelism to default job parallelism?
   ```
         // Note that IcebergSink internally consists o multiple operators 
(like writer, committer,
         // aggregator).
         // The following parallelism will be propagated to all of the above 
operators.
         if (sink.flinkWriteConf.writeParallelism() != null) {
           
rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism());
         }
   ```
   
   technically, if this is a behavior change problem for v2 sink, it is not 
caused by this PR. but it is critical that the same writer parallelism is used 
by the shuffle operator to properly range partition the data to downstream 
writer tasks. That is why in the v1 `FlinkSink`, you can see 
`writerParallelism` is computed once and pass to two methods.
   
   ```
         int writerParallelism =
             flinkWriteConf.writeParallelism() == null
                 ? rowDataInput.getParallelism()
                 : flinkWriteConf.writeParallelism();
   
         // Distribute the records from input data stream based on the 
write.distribution-mode and
         // equality fields.
         DataStream<RowData> distributeStream =
             distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, 
writerParallelism);
   
         // Add parallel writers that append rows to files
         SingleOutputStreamOperator<FlinkWriteResult> writerStream =
             appendWriter(distributeStream, flinkRowType, equalityFieldIds, 
writerParallelism);
   ```



-- 
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: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to