You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "hililiwei (via GitHub)" <gi...@apache.org> on 2023/04/06 12:34:57 UTC

[GitHub] [iceberg] hililiwei commented on a diff in pull request #7077: Flink 1.16: Change distribution modes

hililiwei commented on code in PR #7077:
URL: https://github.com/apache/iceberg/pull/7077#discussion_r1159740064


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java:
##########
@@ -484,63 +484,23 @@ private DataStream<RowData> distributeDataStream(
         PartitionSpec partitionSpec,
         Schema iSchema,
         RowType flinkRowType) {
-      DistributionMode writeMode = flinkWriteConf.distributionMode();
+      DistributionMode writeMode =
+          flinkWriteConf.distributionMode(equalityFieldIds, equalityFieldColumns);
 
       LOG.info("Write distribution mode is '{}'", writeMode.modeName());
       switch (writeMode) {
         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(iSchema, flinkRowType, equalityFieldIds));
-          }
+          return input;
 
         case HASH:
-          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));
-            }
-          }
-
-        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 {
+          if (partitionSpec.isUnpartitioned()) {

Review Comment:
   I did make a change here, sorry I forgot to add a comment.
   
   Because for now, Range is not supported, if it's a non-partitioned table, it's still going to be EqualityFieldKeySelector, and for partitioned tables, there's no obvious benefit to using EqualityFieldKeySelector.
   
   But I've removed that part of the logic. So let's skip it.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java:
##########
@@ -153,16 +167,89 @@ public String orcCompressionStrategy() {
         .parse();
   }
 
-  public DistributionMode distributionMode() {

Review Comment:
   On reflection, I removed this logic and kept it in FlinkSink to keep the `FlinkWriteConf` pure.



-- 
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-unsubscribe@iceberg.apache.org

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


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