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

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

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


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java:
##########
@@ -153,16 +167,89 @@ public String orcCompressionStrategy() {
         .parse();
   }
 
-  public DistributionMode distributionMode() {
+  public DistributionMode distributionMode(
+      List<Integer> equalityFieldIds, List<String> equalityFieldColumns) {
     String modeName =
         confParser
             .stringConf()
             .option(FlinkWriteOptions.DISTRIBUTION_MODE.key())
             .flinkConfig(FlinkWriteOptions.DISTRIBUTION_MODE)
             .tableProperty(TableProperties.WRITE_DISTRIBUTION_MODE)
-            .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_NONE)
-            .parse();
-    return DistributionMode.fromName(modeName);
+            .parseOptional();
+
+    DistributionMode writeMode =
+        modeName == null
+            ? defaultWriteDistributionMode(equalityFieldIds)
+            : DistributionMode.fromName(modeName);
+
+    switch (writeMode) {
+      case NONE:
+        if (equalityFieldIds.isEmpty()) {
+          return NONE;
+        } else {
+          LOG.warn("Switch to use 'hash' distribution mode, because there are equality fields set");
+          return HASH;
+        }
+
+      case HASH:
+        PartitionSpec partitionSpec = table.spec();
+        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 NONE;
+          }
+        } else {
+          if (partitionSpec.isPartitioned()) {
+            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 HASH;
+
+      case RANGE:

Review Comment:
   If the sort order is defined, it would be better to throw an exception to tell the user that flink writer can't guarantee the order rather than fallback to these writers. 



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