rodmeneses commented on code in PR #12071: URL: https://github.com/apache/iceberg/pull/12071#discussion_r2098754472
########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java: ########## @@ -645,70 +676,121 @@ 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; + return distributeDataStreamByRangeDistributionMode(input, schema, spec, sortOrder); + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + private DataStream<RowData> distributeDataStreamByNoneDistributionMode( Review Comment: Breaking `distributeDataStream` into smaller functions, each one for a given distributionMode. due to: ``` Cyclomatic Complexity is 13 (max allowed is 12). [CyclomaticComplexity] ``` This way, it is also clear what each distributionMode needs as function parameters, i.e: 1. `distributeDataStreamByNoneDistributionMode -> (DataStream<RowData> input, Schema schema) ` 2. `distributeDataStreamByHashDistributionMode -> (DataStream<RowData> input, Schema schema, PartitionSpec spec)` 3. `distributeDataStreamByRangeDistributionMode` -> DataStream<RowData> input, Schema schema, PartitionSpec spec, SortOrder sortOrderParam` Which brings nice information about what each distributionMode needs for its internal calculation -- 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