From 95a28a2b97fbe4744817df619b9f247fd142468f Mon Sep 17 00:00:00 2001 From: Han You Date: Tue, 24 Feb 2026 09:54:48 -0600 Subject: [PATCH] Flink: Fix distribution mode none in DynamicIcebergSink Currently, DistributionMode.NONE actually performs a round robin. This commit changes the behavior so that records tagged as NONE will go to a passthrough side output to enable chaining. A new distribution mode ROUND_ROBIN is added which behaves like NONE before this change. --- .../org/apache/iceberg/DistributionMode.java | 5 +- docs/docs/flink-writes.md | 14 +++- .../sink/dynamic/DynamicIcebergSink.java | 45 +++++++--- .../sink/dynamic/DynamicRecordProcessor.java | 83 +++++++++++++------ .../flink/sink/dynamic/HashKeyGenerator.java | 8 +- .../iceberg/flink/TestIcebergConnector.java | 2 +- .../sink/dynamic/TestDynamicIcebergSink.java | 13 +-- .../dynamic/TestDynamicIcebergSinkPerf.java | 2 +- .../sink/dynamic/TestHashKeyGenerator.java | 73 ++++++++++++---- .../apache/iceberg/spark/SparkWriteUtil.java | 6 ++ .../apache/iceberg/spark/SparkWriteUtil.java | 6 ++ .../apache/iceberg/spark/SparkWriteUtil.java | 6 ++ .../apache/iceberg/spark/SparkWriteUtil.java | 6 ++ 13 files changed, 206 insertions(+), 63 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/DistributionMode.java b/api/src/main/java/org/apache/iceberg/DistributionMode.java index a7625675e1d7..56c281c4e010 100644 --- a/api/src/main/java/org/apache/iceberg/DistributionMode.java +++ b/api/src/main/java/org/apache/iceberg/DistributionMode.java @@ -39,7 +39,8 @@ public enum DistributionMode { NONE("none"), HASH("hash"), - RANGE("range"); + RANGE("range"), + ROUND_ROBIN("round-robin"); private final String modeName; @@ -54,7 +55,7 @@ public String modeName() { public static DistributionMode fromName(String modeName) { Preconditions.checkArgument(null != modeName, "Invalid distribution mode: null"); try { - return DistributionMode.valueOf(modeName.toUpperCase(Locale.ENGLISH)); + return DistributionMode.valueOf(modeName.toUpperCase(Locale.ENGLISH).replace('-', '_')); } catch (IllegalArgumentException e) { throw new IllegalArgumentException(String.format("Invalid distribution mode: %s", modeName)); } diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index 3fef3a1bf3bf..8d9f80006c95 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -483,7 +483,7 @@ We need the following information (DynamicRecord) for every record: | `Schema` | The schema of the record. | | `Spec` | The expected partitioning specification for the record. | | `RowData` | The actual row data to be written. | -| `DistributionMode` | The distribution mode for writing the record (currently supports NONE or HASH). | +| `DistributionMode` | The distribution mode for writing the record (supports NONE, ROUND_ROBIN, or HASH). See [Distribution Modes](#distribution-modes-1) below. | | `Parallelism` | The maximum number of parallel writers for a given table/branch/schema/spec (WriteTarget). | | `UpsertMode` | Overrides this table's write.upsert.enabled (optional). | | `EqualityFields` | The equality fields for the table(optional). | @@ -547,6 +547,18 @@ The Dynamic Iceberg Flink Sink is configured using the Builder pattern. Here are | `tableCreator(TableCreator creator)` | When DynamicIcebergSink creates new Iceberg tables, allows overriding how tables are created - setting custom table properties and location based on the table name. | | `dropUnusedColumns(boolean enabled)` | When enabled, drops all columns from the current table schema which are not contained in the input schema (see the caveats above on dropping columns). | +### Distribution Modes + +The `DistributionMode` set on each `DynamicRecord` controls how that record is routed from the processor to the writer: + +| Mode | Behavior | +|------|----------| +| `NONE` | True forward/no-redistribution. Records are sent directly from the processor to the writer using a forward edge, which enables Flink operator chaining. This avoids serialization and network shuffle overhead. Table metadata updates are always performed immediately inside the processor (regardless of `immediateTableUpdate` setting). Best for high-throughput pipelines where each subtask writes independently. | +| `ROUND_ROBIN` | Records are distributed across writer subtasks in a round-robin fashion (or by equality fields if set). This is the default when no distribution mode is set on the `DynamicRecord`. | +| `HASH` | Records are distributed by partition key (partitioned tables) or equality fields (unpartitioned tables). Ensures that records for the same partition are handled by the same writer subtask. | + +Records with different distribution modes can be mixed in the same pipeline. The sink internally maintains two write paths: a forward path for `NONE` records (chainable, no shuffle) and a shuffle path for `ROUND_ROBIN`/`HASH` records (keyed by writer key). + ### Notes - **Range distribution mode**: Currently, the dynamic sink does not support the `RANGE` distribution mode, if set, it will fall back to `HASH`. diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java index afafbe5b5974..fa23ed963f45 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicIcebergSink.java @@ -78,6 +78,7 @@ public class DynamicIcebergSink private final Map writeProperties; private final Configuration flinkConfig; private final int cacheMaximumSize; + private final boolean forwardOnly; DynamicIcebergSink( CatalogLoader catalogLoader, @@ -85,13 +86,15 @@ public class DynamicIcebergSink String uidPrefix, Map writeProperties, Configuration flinkConfig, - int cacheMaximumSize) { + int cacheMaximumSize, + boolean forwardOnly) { this.catalogLoader = catalogLoader; this.snapshotProperties = snapshotProperties; this.uidPrefix = uidPrefix; this.writeProperties = writeProperties; this.flinkConfig = flinkConfig; this.cacheMaximumSize = cacheMaximumSize; + this.forwardOnly = forwardOnly; // We generate a random UUID every time when a sink is created. // This is used to separate files generated by different sinks writing the same table. // Also used to generate the aggregator operator name @@ -135,6 +138,9 @@ public void addPostCommitTopology( @Override public DataStream addPreWriteTopology( DataStream inputDataStream) { + if (forwardOnly) { + return inputDataStream; + } return distributeDataStream(inputDataStream); } @@ -357,7 +363,7 @@ private String operatorName(String suffix) { return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; } - private DynamicIcebergSink build() { + private DynamicIcebergSink build(boolean forwardOnly) { Preconditions.checkArgument( generator != null, "Please use withGenerator() to convert the input DataStream."); @@ -370,19 +376,20 @@ private DynamicIcebergSink build() { ? (Configuration) readableConfig : Configuration.fromMap(readableConfig.toMap()); - return instantiateSink(writeOptions, flinkConfig); + return instantiateSink(writeOptions, flinkConfig, forwardOnly); } @VisibleForTesting DynamicIcebergSink instantiateSink( - Map writeProperties, Configuration flinkWriteConf) { + Map writeProperties, Configuration flinkWriteConf, boolean forwardOnly) { return new DynamicIcebergSink( catalogLoader, snapshotSummary, uidPrefix, writeProperties, flinkWriteConf, - cacheMaximumSize); + cacheMaximumSize, + forwardOnly); } /** @@ -393,7 +400,6 @@ DynamicIcebergSink instantiateSink( public DataStreamSink append() { DynamicRecordInternalType type = new DynamicRecordInternalType(catalogLoader, false, cacheMaximumSize); - DynamicIcebergSink sink = build(); SingleOutputStreamOperator converted = input .process( @@ -411,7 +417,9 @@ public DataStreamSink append() { .name(operatorName("generator")) .returns(type); - DataStreamSink rowDataDataStreamSink = + // Shuffle sink: handles non-NONE distribution records (HASH, ROUND_ROBIN, RANGE) + DynamicIcebergSink shuffleSink = build(false); + DataStream shuffleInput = converted .getSideOutput( new OutputTag<>( @@ -430,16 +438,29 @@ public DataStreamSink append() { .uid(prefixIfNotNull(uidPrefix, "-updater")) .name(operatorName("Updater")) .returns(type) - .union(converted) - .sinkTo(sink) - .uid(prefixIfNotNull(uidPrefix, "-sink")); + .union(converted); + + DataStreamSink shuffleSinkResult = + shuffleInput.sinkTo(shuffleSink).uid(prefixIfNotNull(uidPrefix, "-shuffle-sink")); FlinkWriteConf flinkWriteConf = new FlinkWriteConf(writeOptions, readableConfig); if (flinkWriteConf.writeParallelism() != null) { - rowDataDataStreamSink.setParallelism(flinkWriteConf.writeParallelism()); + shuffleSinkResult.setParallelism(flinkWriteConf.writeParallelism()); } - return rowDataDataStreamSink; + // Forward sink: handles NONE distribution records (forward edge, chainable) + DynamicIcebergSink forwardSink = build(true); + OutputTag forwardTag = + new OutputTag<>( + DynamicRecordProcessor.DYNAMIC_FORWARD_STREAM, + new DynamicRecordInternalType(catalogLoader, true, cacheMaximumSize)); + + converted + .getSideOutput(forwardTag) + .sinkTo(forwardSink) + .uid(prefixIfNotNull(uidPrefix, "-forward-sink")); + + return shuffleSinkResult; } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java index 07dfad2780f7..ab09b804806f 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -26,10 +26,12 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.util.Collector; import org.apache.flink.util.OutputTag; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @Internal class DynamicRecordProcessor extends ProcessFunction @@ -37,6 +39,8 @@ class DynamicRecordProcessor extends ProcessFunction generator; private final CatalogLoader catalogLoader; private final boolean immediateUpdate; @@ -51,6 +55,7 @@ class DynamicRecordProcessor extends ProcessFunction updateStream; + private transient OutputTag forwardStream; private transient Collector collector; private transient Context context; @@ -90,9 +95,14 @@ public void open(OpenContext openContext) throws Exception { this.hashKeyGenerator = new HashKeyGenerator( cacheMaximumSize, getRuntimeContext().getTaskInfo().getMaxNumberOfParallelSubtasks()); - if (immediateUpdate) { - updater = new TableUpdater(tableCache, catalog, caseSensitive, dropUnusedColumns); - } else { + // Always create updater — needed for forced immediate updates on NONE records + this.updater = new TableUpdater(tableCache, catalog, caseSensitive, dropUnusedColumns); + // Always create forward stream tag for NONE distribution records + this.forwardStream = + new OutputTag<>( + DYNAMIC_FORWARD_STREAM, + new DynamicRecordInternalType(catalogLoader, true, cacheMaximumSize)) {}; + if (!immediateUpdate) { updateStream = new OutputTag<>( DYNAMIC_TABLE_UPDATE_STREAM, @@ -112,6 +122,10 @@ public void processElement(T element, Context ctx, Collector newData = updater.update( data.tableIdentifier(), data.branch(), data.schema(), data.spec(), tableCreator); emit( - collector, data, newData.f0.resolvedTableSchema(), newData.f0.recordConverter(), - newData.f1); + newData.f1, + isForward); } else { + // Non-NONE records with immediateUpdate=false go to update side output int writerKey = hashKeyGenerator.generateKey( data, @@ -159,33 +178,47 @@ public void collect(DynamicRecord data) { } } else { emit( - collector, data, foundSchema.resolvedTableSchema(), foundSchema.recordConverter(), - foundSpec); + foundSpec, + isForward); } } private void emit( - Collector out, DynamicRecord data, Schema schema, DataConverter recordConverter, - PartitionSpec spec) { + PartitionSpec spec, + boolean forward) { RowData rowData = (RowData) recordConverter.convert(data.rowData()); - int writerKey = hashKeyGenerator.generateKey(data, schema, spec, rowData); String tableName = data.tableIdentifier().toString(); - out.collect( - new DynamicRecordInternal( - tableName, - data.branch(), - schema, - rowData, - spec, - writerKey, - data.upsertMode(), - DynamicSinkUtil.getEqualityFieldIds(data.equalityFields(), schema))); + if (forward) { + DynamicRecordInternal record = + new DynamicRecordInternal( + tableName, + data.branch(), + schema, + rowData, + spec, + 0, + data.upsertMode(), + DynamicSinkUtil.getEqualityFieldIds(data.equalityFields(), schema)); + context.output(forwardStream, record); + } else { + int writerKey = hashKeyGenerator.generateKey(data, schema, spec, rowData); + collector.collect( + new DynamicRecordInternal( + tableName, + data.branch(), + schema, + rowData, + spec, + writerKey, + data.upsertMode(), + DynamicSinkUtil.getEqualityFieldIds(data.equalityFields(), schema))); + } } @Override diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 5eb31a9f705f..4ff0b6e2e073 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -97,7 +97,7 @@ int generateKey( MoreObjects.firstNonNull(tableSchema, dynamicRecord.schema()), MoreObjects.firstNonNull(tableSpec, dynamicRecord.spec()), MoreObjects.firstNonNull( - dynamicRecord.distributionMode(), DistributionMode.NONE), + dynamicRecord.distributionMode(), DistributionMode.ROUND_ROBIN), MoreObjects.firstNonNull( dynamicRecord.equalityFields(), Collections.emptySet()), Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism))); @@ -120,6 +120,12 @@ private KeySelector getKeySelector( "Creating new KeySelector for table '{}' with distribution mode '{}'", tableName, mode); switch (mode) { case NONE: + return row -> { + throw new IllegalStateException( + "Records with DistributionMode.NONE are routed via the forward path. They should not reach the key generator."); + }; + + case ROUND_ROBIN: if (equalityFields.isEmpty()) { return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); } else { diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index 309b55c115c5..189edc201605 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -468,7 +468,7 @@ public void generate(RowData inputRecord, Collector out) throws E schema, inputRecord, PartitionSpec.unpartitioned(), - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, 1); out.collect(dynamicRecord); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index ad4a13d5619b..b5093684c3eb 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -203,7 +203,7 @@ public void generate(DynamicIcebergDataImpl row, Collector out) { schema, converter(schema).toInternal(row.rowProvided), spec, - spec.isPartitioned() ? DistributionMode.HASH : DistributionMode.NONE, + spec.isPartitioned() ? DistributionMode.HASH : DistributionMode.ROUND_ROBIN, 10); dynamicRecord.setUpsertMode(row.upsertMode); dynamicRecord.setEqualityFields(row.equalityFields); @@ -1296,7 +1296,7 @@ static class CommitHookEnabledDynamicIcebergSink extends DynamicIcebergSink.B @Override DynamicIcebergSink instantiateSink( - Map writeProperties, Configuration flinkConfig) { + Map writeProperties, Configuration flinkConfig, boolean forwardOnly) { return new CommitHookDynamicIcebergSink( commitHook, CATALOG_EXTENSION.catalogLoader(), @@ -1304,7 +1304,8 @@ DynamicIcebergSink instantiateSink( "uidPrefix", writeProperties, flinkConfig, - 100); + 100, + forwardOnly); } } @@ -1320,14 +1321,16 @@ static class CommitHookDynamicIcebergSink extends DynamicIcebergSink { String uidPrefix, Map writeProperties, Configuration flinkConfig, - int cacheMaximumSize) { + int cacheMaximumSize, + boolean forwardOnly) { super( catalogLoader, snapshotProperties, uidPrefix, writeProperties, flinkConfig, - cacheMaximumSize); + cacheMaximumSize, + forwardOnly); this.commitHook = commitHook; this.overwriteMode = new FlinkWriteConf(writeProperties, flinkConfig).overwriteMode(); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java index 41c1f67d2926..ed4687c9f766 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java @@ -137,7 +137,7 @@ void before() { SCHEMA, RowDataConverter.convert(SCHEMA, records.get(i)), PartitionSpec.unpartitioned(), - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, WRITE_PARALLELISM)); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 7c1d3c3d0aeb..d2da5d371e73 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -49,7 +49,7 @@ class TestHashKeyGenerator { private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("default", "table"); @Test - void testRoundRobinWithDistributionModeNone() throws Exception { + void testRoundRobinWithDistributionModeRoundRobin() throws Exception { int writeParallelism = 10; int maxWriteParallelism = 2; HashKeyGenerator generator = new HashKeyGenerator(1, maxWriteParallelism); @@ -58,16 +58,36 @@ void testRoundRobinWithDistributionModeNone() throws Exception { GenericRowData row = GenericRowData.of(1, StringData.fromString("z")); int writeKey1 = getWriteKey( - generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptySet(), row); + generator, + spec, + DistributionMode.ROUND_ROBIN, + writeParallelism, + Collections.emptySet(), + row); int writeKey2 = getWriteKey( - generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptySet(), row); + generator, + spec, + DistributionMode.ROUND_ROBIN, + writeParallelism, + Collections.emptySet(), + row); int writeKey3 = getWriteKey( - generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptySet(), row); + generator, + spec, + DistributionMode.ROUND_ROBIN, + writeParallelism, + Collections.emptySet(), + row); int writeKey4 = getWriteKey( - generator, spec, DistributionMode.NONE, writeParallelism, Collections.emptySet(), row); + generator, + spec, + DistributionMode.ROUND_ROBIN, + writeParallelism, + Collections.emptySet(), + row); assertThat(writeKey1).isNotEqualTo(writeKey2); assertThat(writeKey3).isEqualTo(writeKey1); @@ -79,6 +99,29 @@ void testRoundRobinWithDistributionModeNone() throws Exception { assertThat(getSubTaskId(writeKey4, writeParallelism, maxWriteParallelism)).isEqualTo(5); } + @Test + void testDistributionModeNoneThrows() { + int writeParallelism = 10; + int maxWriteParallelism = 2; + HashKeyGenerator generator = new HashKeyGenerator(1, maxWriteParallelism); + PartitionSpec spec = PartitionSpec.unpartitioned(); + + GenericRowData row = GenericRowData.of(1, StringData.fromString("z")); + // NONE records are routed via the forward path and should never reach the key generator + assertThatThrownBy( + () -> + getWriteKey( + generator, + spec, + DistributionMode.NONE, + writeParallelism, + Collections.emptySet(), + row)) + .isInstanceOf(RuntimeException.class) + .hasCauseInstanceOf(IllegalStateException.class) + .hasMessageContaining("forward path"); + } + @Test void testBucketingWithDistributionModeHash() throws Exception { int writeParallelism = 3; @@ -130,7 +173,7 @@ void testEqualityKeys() throws Exception { getWriteKey( generator, unpartitioned, - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, writeParallelism, equalityColumns, row1); @@ -138,7 +181,7 @@ void testEqualityKeys() throws Exception { getWriteKey( generator, unpartitioned, - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, writeParallelism, equalityColumns, row2); @@ -146,7 +189,7 @@ void testEqualityKeys() throws Exception { getWriteKey( generator, unpartitioned, - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, writeParallelism, equalityColumns, row3); @@ -211,7 +254,7 @@ void testFailOnNonPositiveWriteParallelism() { getWriteKey( generator, PartitionSpec.unpartitioned(), - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, -1, // writeParallelism Collections.emptySet(), GenericRowData.of()); @@ -222,7 +265,7 @@ void testFailOnNonPositiveWriteParallelism() { getWriteKey( generator, PartitionSpec.unpartitioned(), - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, 0, // writeParallelism Collections.emptySet(), GenericRowData.of()); @@ -243,7 +286,7 @@ void testCapAtMaxWriteParallelism() throws Exception { getWriteKey( generator, unpartitioned, - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, writeParallelism, Collections.emptySet(), row)); @@ -259,7 +302,7 @@ void testCapAtMaxWriteParallelism() throws Exception { } @Test - void testHashModeWithoutEqualityFieldsFallsBackToNone() throws Exception { + void testHashModeWithoutEqualityFieldsFallsBackToRoundRobin() throws Exception { int writeParallelism = 2; int maxWriteParallelism = 8; HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); @@ -301,12 +344,12 @@ void testSchemaSpecOverrides() throws Exception { SCHEMA, GenericRowData.of(1, StringData.fromString("foo")), PartitionSpec.unpartitioned(), - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, writeParallelism); int writeKey1 = generator.generateKey(record); int writeKey2 = generator.generateKey(record); - // Assert that we are bucketing via NONE (round-robin) + // Assert that we are bucketing via ROUND_ROBIN assertThat(writeKey1).isNotEqualTo(writeKey2); // Schema has different id @@ -385,7 +428,7 @@ void testCaching() throws Exception { SCHEMA, GenericRowData.of(1, StringData.fromString("foo")), unpartitioned, - DistributionMode.NONE, + DistributionMode.ROUND_ROBIN, writeParallelism); int writeKey1 = generator.generateKey(record); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java index 8f9c39de99be..8d5edc15f874 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java @@ -76,6 +76,9 @@ public static SparkWriteRequirements writeRequirements( private static Distribution writeDistribution(Table table, DistributionMode mode) { switch (mode) { case NONE: + // Spark's connector API has no round-robin distribution concept, so treat it the same + // as unspecified and let Spark distribute data across tasks as it sees fit + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -107,6 +110,7 @@ private static Distribution copyOnWriteDeleteUpdateDistribution( switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -148,6 +152,7 @@ private static Distribution positionDeltaUpdateMergeDistribution( switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -180,6 +185,7 @@ private static SortOrder[] positionDeltaUpdateMergeOrdering(Table table, boolean private static Distribution positionDeltaDeleteDistribution(Table table, DistributionMode mode) { switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java index 0d68a0d8cdd0..ce20a39e40ce 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java @@ -76,6 +76,9 @@ public static SparkWriteRequirements writeRequirements( private static Distribution writeDistribution(Table table, DistributionMode mode) { switch (mode) { case NONE: + // Spark's connector API has no round-robin distribution concept, so treat it the same + // as unspecified and let Spark distribute data across tasks as it sees fit + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -111,6 +114,7 @@ private static Distribution copyOnWriteDeleteUpdateDistribution( switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -156,6 +160,7 @@ private static Distribution positionDeltaUpdateMergeDistribution( switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -188,6 +193,7 @@ private static SortOrder[] positionDeltaUpdateMergeOrdering(Table table, boolean private static Distribution positionDeltaDeleteDistribution(Table table, DistributionMode mode) { switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java index 86902c15e139..6e9a9d371046 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java @@ -106,6 +106,9 @@ public static SparkWriteRequirements writeRequirements( private static Distribution writeDistribution(Table table, DistributionMode mode) { switch (mode) { case NONE: + // Spark's connector API has no round-robin distribution concept, so treat it the same + // as unspecified and let Spark distribute data across tasks as it sees fit + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -141,6 +144,7 @@ private static Distribution copyOnWriteDeleteUpdateDistribution( switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -186,6 +190,7 @@ private static Distribution positionDeltaUpdateMergeDistribution( switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -218,6 +223,7 @@ private static SortOrder[] positionDeltaUpdateMergeOrdering(Table table, boolean private static Distribution positionDeltaDeleteDistribution(Table table, DistributionMode mode) { switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java index 86902c15e139..6e9a9d371046 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java @@ -106,6 +106,9 @@ public static SparkWriteRequirements writeRequirements( private static Distribution writeDistribution(Table table, DistributionMode mode) { switch (mode) { case NONE: + // Spark's connector API has no round-robin distribution concept, so treat it the same + // as unspecified and let Spark distribute data across tasks as it sees fit + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -141,6 +144,7 @@ private static Distribution copyOnWriteDeleteUpdateDistribution( switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -186,6 +190,7 @@ private static Distribution positionDeltaUpdateMergeDistribution( switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: @@ -218,6 +223,7 @@ private static SortOrder[] positionDeltaUpdateMergeOrdering(Table table, boolean private static Distribution positionDeltaDeleteDistribution(Table table, DistributionMode mode) { switch (mode) { case NONE: + case ROUND_ROBIN: return Distributions.unspecified(); case HASH: