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


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import java.util.stream.IntStream;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.flink.RowDataWrapper;
+
+/**
+ * A {@link KeySelector} that extracts the bucketId from a data row's bucket 
partition as the key.
+ * To be used with the {@link BucketPartitioner}.
+ */
+class BucketPartitionKeySelector implements KeySelector<RowData, Integer> {
+
+  private final Schema schema;
+  private final PartitionKey partitionKey;
+  private final RowType flinkSchema;
+  private final int bucketFieldPosition;
+
+  private transient RowDataWrapper rowDataWrapper;
+
+  BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, 
RowType flinkSchema) {

Review Comment:
   nit: usually do simple assignments like `this.schema = schema` first. Then 
start a new section with calculated values like  `bucketFieldId`, 
`bucketFieldPosition`



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * This partitioner will redirect records to writers deterministically based 
on the Bucket partition
+ * spec. It'll attempt to optimize the file size written depending on whether 
numPartitions is
+ * greater, less or equal than the maxNumBuckets. Note: The current 
implementation only supports ONE
+ * bucket in the partition spec.
+ */
+class BucketPartitioner implements Partitioner<Integer> {
+
+  static final String BUCKET_NULL_MESSAGE = "bucketId cannot be null";
+  static final String BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE =
+      "bucketId out of range: %s, must be non-negative.";
+  static final String BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE =
+      "bucketId out of range: %s, must be less than bucket limit: %s.";
+
+  private final int maxNumBuckets;
+
+  // To hold the OFFSET of the next writer to use for any bucket, only used 
when writers > the
+  // number

Review Comment:
   nit: fix formatting.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.transforms.PartitionSpecVisitor;
+
+final class BucketPartitionerUtils {
+  static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE =

Review Comment:
   nit on error msg `Unsupported partition spec: bucket partitioner expect 1 
bucket partition: %s`



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * This partitioner will redirect records to writers deterministically based 
on the Bucket partition
+ * spec. It'll attempt to optimize the file size written depending on whether 
numPartitions is
+ * greater, less or equal than the maxNumBuckets. Note: The current 
implementation only supports ONE
+ * bucket in the partition spec.
+ */
+class BucketPartitioner implements Partitioner<Integer> {
+
+  static final String BUCKET_NULL_MESSAGE = "bucketId cannot be null";
+  static final String BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE =
+      "bucketId out of range: %s, must be non-negative.";

Review Comment:
   Iceberg style on error msg. it should be readable to users (not Iceberg 
developers). Hence, variable or class name shouldn't be exposed. `bucketId` 
should be `Bucket ID`.
   
   Iceberg error msg tends to follow the style of `Invalid ...`. Hence error 
msgs can be like `Invalid bucket ID: must be non-negative: %s` and `Invalid 
bucket ID: must be less than the bucket limit of %s: %s`



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.BucketUtil;
+
+final class TestBucketPartitionerUtils {
+
+  enum TableSchemaType {
+    ONE_BUCKET,
+    IDENTITY_AND_BUCKET,
+    TWO_BUCKETS;
+
+    public static int bucketPartitionColumnPosition(TableSchemaType 
tableSchemaType) {

Review Comment:
   it is weird that we are putting those static method in the enum class 
definition. they should be overriden in the enum definition so that we can 
avoid the switch-case like the method below. these two methods should be non 
static.
   
   also `TWO_BUCKETS` should throw an exception for this method.



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE;
+import static 
org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE;
+import static 
org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE;
+
+import org.apache.iceberg.PartitionSpec;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+
+public class TestBucketPartitioner {
+
+  static final int DEFAULT_NUM_BUCKETS = 60;
+
+  @ParameterizedTest
+  @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", 
"IDENTITY_AND_BUCKET,60"})
+  public void testPartitioningParallelismGreaterThanBuckets(
+      String schemaTypeStr, String numBucketsStr) {
+    final int numPartitions = 500;

Review Comment:
   nit: no `final` for local var



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+

Review Comment:
   nit this empty line is not needed



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {

Review Comment:
   should this use `@BeforeEach`?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {

Review Comment:
   I would incline to merge this method with the method above



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+
+    return catalogExtension
+        .catalog()
+        .createTable(
+            TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            partitionSpec,
+            ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, 
format.name()));
+  }
+
+  private void testWriteRowData(List<RowData> allRows) throws Exception {
+    DataFormatConverters.RowConverter converter =
+        new 
DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes());
+
+    DataStream<RowData> dataStream =
+        env.addSource(
+                new BoundedTestSource<>(
+                    
allRows.stream().map(converter::toExternal).toArray(Row[]::new)),

Review Comment:
   why do we need convert internal and external back and forth? if this is 
about type info, can we use `TypeInformation.of(RowData.class)`?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionKeySelector {
+
+  @ParameterizedTest
+  @EnumSource(
+      value = TableSchemaType.class,
+      names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"})
+  public void testCorrectKeySelection(TableSchemaType tableSchemaType) {
+    final int numBuckets = 60;

Review Comment:
   nit: Iceberg style doesn't use `final` for local var



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+
+    return catalogExtension
+        .catalog()
+        .createTable(
+            TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            partitionSpec,
+            ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, 
format.name()));
+  }
+
+  private void testWriteRowData(List<RowData> allRows) throws Exception {
+    DataFormatConverters.RowConverter converter =
+        new 
DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes());
+
+    DataStream<RowData> dataStream =
+        env.addSource(
+                new BoundedTestSource<>(
+                    
allRows.stream().map(converter::toExternal).toArray(Row[]::new)),
+                ROW_TYPE_INFO)
+            .map(converter::toInternal, 
FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE));
+
+    FlinkSink.forRowData(dataStream)
+        .table(table)
+        .tableLoader(tableLoader)
+        .writeParallelism(parallelism)
+        .distributionMode(DistributionMode.HASH)
+        .append();
+
+    // Execute the program.
+    env.execute("Test Iceberg DataStream");
+
+    // Assert the iceberg table's records.
+    SimpleDataUtil.assertTableRows(table, allRows);
+  }
+
+  @ParameterizedTest
+  @EnumSource(
+      value = TableSchemaType.class,
+      names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"})
+  public void testSendRecordsToAllBucketsEvenly(TableSchemaType 
tableSchemaType) throws Exception {
+    setupEnvironment(tableSchemaType);
+    List<RowData> rows = generateTestDataRows();
+
+    testWriteRowData(rows);
+    TableTestStats stats = extractPartitionResults(tableSchemaType);
+
+    Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size());
+    // All 4 buckets should've been written to
+    Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets);
+    
Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets);
+    // Writer expectation (2 writers per bucket):
+    // - Bucket0 -> Writers [0, 4]
+    // - Bucket1 -> Writers [1, 5]
+    // - Bucket2 -> Writers [2, 6]
+    // - Bucket3 -> Writers [3, 7]
+    for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) {
+      
Assertions.assertThat(stats.writersPerBucket.get(i)).isEqualTo(Arrays.asList(i, 
j));
+      // 2 files per bucket (one file is created by each writer)
+      Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(2);
+      // 2 rows per file (total of 16 rows across 8 files)
+      Assertions.assertThat((long) stats.rowsPerFile.get(i)).isEqualTo(2);
+    }
+  }
+
+  /**
+   * Verifies the BucketPartitioner is not used when the PartitionSpec has 
more than 1 bucket, and
+   * that it should fallback to input.keyBy
+   */
+  @ParameterizedTest
+  @EnumSource(value = TableSchemaType.class, names = "TWO_BUCKETS")
+  public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) 
throws Exception {
+    setupEnvironment(tableSchemaType);
+    List<RowData> rows = generateTestDataRows();
+
+    testWriteRowData(rows);
+    TableTestStats stats = extractPartitionResults(tableSchemaType);
+
+    Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size());
+    for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) {
+      // Only 1 file per bucket will be created when falling back to 
input.keyBy(...)
+      Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(1);
+    }
+  }
+
+  /**
+   * Generating 16 rows to be sent uniformly to all writers (round-robin 
across 8 writers -> 4
+   * buckets)
+   */
+  @NotNull
+  private List<RowData> generateTestDataRows() {
+    int totalNumRows = parallelism * 2;
+    int numRowsPerBucket = totalNumRows / numBuckets;
+    return 
TestBucketPartitionerUtils.generateRowsForBucketIdRange(numRowsPerBucket, 
numBuckets);
+  }
+
+  private TableTestStats extractPartitionResults(TableSchemaType 
tableSchemaType)
+      throws IOException {
+    int totalRecordCount = 0;
+    Map<Integer, List<Integer>> writersPerBucket = Maps.newHashMap(); // 
<BucketId, List<WriterId>>
+    Map<Integer, Integer> filesPerBucket = Maps.newHashMap(); // <BucketId, 
NumFiles>
+    Map<Integer, Long> recordsPerFile = new TreeMap<>(); // <WriterId, 
NumRecords>

Review Comment:
   this should be `recordsPerWriter`. also why is `TreeMap` needed here?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;

Review Comment:
   wrong annotation. shouldn't use jetbrains. also looking at the usage. I 
think we can just remove the annotation usage



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.transforms.PartitionSpecVisitor;
+
+final class BucketPartitionerUtils {

Review Comment:
   nit: although I can find both `Util` and `Utils`, I think there are a lot 
more class name without the plural. so let's stay with `Util` for this and the 
other test class.



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+
+    return catalogExtension
+        .catalog()
+        .createTable(
+            TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            partitionSpec,
+            ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, 
format.name()));
+  }
+
+  private void testWriteRowData(List<RowData> allRows) throws Exception {
+    DataFormatConverters.RowConverter converter =
+        new 
DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes());
+
+    DataStream<RowData> dataStream =
+        env.addSource(
+                new BoundedTestSource<>(
+                    
allRows.stream().map(converter::toExternal).toArray(Row[]::new)),
+                ROW_TYPE_INFO)
+            .map(converter::toInternal, 
FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE));
+
+    FlinkSink.forRowData(dataStream)
+        .table(table)
+        .tableLoader(tableLoader)
+        .writeParallelism(parallelism)
+        .distributionMode(DistributionMode.HASH)
+        .append();
+
+    // Execute the program.
+    env.execute("Test Iceberg DataStream");
+
+    // Assert the iceberg table's records.
+    SimpleDataUtil.assertTableRows(table, allRows);
+  }
+
+  @ParameterizedTest
+  @EnumSource(
+      value = TableSchemaType.class,
+      names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"})
+  public void testSendRecordsToAllBucketsEvenly(TableSchemaType 
tableSchemaType) throws Exception {
+    setupEnvironment(tableSchemaType);
+    List<RowData> rows = generateTestDataRows();
+
+    testWriteRowData(rows);
+    TableTestStats stats = extractPartitionResults(tableSchemaType);
+
+    Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size());
+    // All 4 buckets should've been written to
+    Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets);
+    
Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets);
+    // Writer expectation (2 writers per bucket):
+    // - Bucket0 -> Writers [0, 4]
+    // - Bucket1 -> Writers [1, 5]
+    // - Bucket2 -> Writers [2, 6]
+    // - Bucket3 -> Writers [3, 7]
+    for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) {
+      
Assertions.assertThat(stats.writersPerBucket.get(i)).isEqualTo(Arrays.asList(i, 
j));
+      // 2 files per bucket (one file is created by each writer)
+      Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(2);
+      // 2 rows per file (total of 16 rows across 8 files)
+      Assertions.assertThat((long) stats.rowsPerFile.get(i)).isEqualTo(2);
+    }
+  }
+
+  /**
+   * Verifies the BucketPartitioner is not used when the PartitionSpec has 
more than 1 bucket, and
+   * that it should fallback to input.keyBy
+   */
+  @ParameterizedTest
+  @EnumSource(value = TableSchemaType.class, names = "TWO_BUCKETS")
+  public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) 
throws Exception {
+    setupEnvironment(tableSchemaType);
+    List<RowData> rows = generateTestDataRows();
+
+    testWriteRowData(rows);
+    TableTestStats stats = extractPartitionResults(tableSchemaType);
+
+    Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size());
+    for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) {
+      // Only 1 file per bucket will be created when falling back to 
input.keyBy(...)
+      Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(1);
+    }
+  }
+
+  /**
+   * Generating 16 rows to be sent uniformly to all writers (round-robin 
across 8 writers -> 4
+   * buckets)
+   */
+  @NotNull
+  private List<RowData> generateTestDataRows() {
+    int totalNumRows = parallelism * 2;
+    int numRowsPerBucket = totalNumRows / numBuckets;
+    return 
TestBucketPartitionerUtils.generateRowsForBucketIdRange(numRowsPerBucket, 
numBuckets);
+  }
+
+  private TableTestStats extractPartitionResults(TableSchemaType 
tableSchemaType)
+      throws IOException {
+    int totalRecordCount = 0;
+    Map<Integer, List<Integer>> writersPerBucket = Maps.newHashMap(); // 
<BucketId, List<WriterId>>
+    Map<Integer, Integer> filesPerBucket = Maps.newHashMap(); // <BucketId, 
NumFiles>
+    Map<Integer, Long> recordsPerFile = new TreeMap<>(); // <WriterId, 
NumRecords>
+
+    try (CloseableIterable<FileScanTask> fileScanTasks = 
table.newScan().planFiles()) {
+      for (FileScanTask scanTask : fileScanTasks) {
+        long recordCountInFile = scanTask.file().recordCount();
+
+        String[] splitFilePath = scanTask.file().path().toString().split("/");
+        String filename = splitFilePath[splitFilePath.length - 1];
+        int writerId = Integer.parseInt(filename.split("-")[0]);
+
+        totalRecordCount += recordCountInFile;
+        int bucketId =
+            scanTask
+                .file()
+                .partition()
+                
.get(TableSchemaType.bucketPartitionColumnPosition(tableSchemaType), 
Integer.class);
+        writersPerBucket.computeIfAbsent(bucketId, k -> Lists.newArrayList());
+        writersPerBucket.get(bucketId).add(writerId);
+        filesPerBucket.put(bucketId, filesPerBucket.getOrDefault(bucketId, 0) 
+ 1);
+        recordsPerFile.put(writerId, recordsPerFile.getOrDefault(writerId, 0L) 
+ recordCountInFile);
+      }
+    }
+
+    for (int k : writersPerBucket.keySet()) {

Review Comment:
   sort is not needed if using `hasSameElementsAs` from assertj



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+
+    return catalogExtension
+        .catalog()
+        .createTable(
+            TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            partitionSpec,
+            ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, 
format.name()));
+  }
+
+  private void testWriteRowData(List<RowData> allRows) throws Exception {
+    DataFormatConverters.RowConverter converter =
+        new 
DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes());
+
+    DataStream<RowData> dataStream =
+        env.addSource(
+                new BoundedTestSource<>(
+                    
allRows.stream().map(converter::toExternal).toArray(Row[]::new)),
+                ROW_TYPE_INFO)
+            .map(converter::toInternal, 
FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE));
+
+    FlinkSink.forRowData(dataStream)
+        .table(table)
+        .tableLoader(tableLoader)
+        .writeParallelism(parallelism)
+        .distributionMode(DistributionMode.HASH)
+        .append();
+
+    // Execute the program.
+    env.execute("Test Iceberg DataStream");
+
+    // Assert the iceberg table's records.
+    SimpleDataUtil.assertTableRows(table, allRows);
+  }
+
+  @ParameterizedTest
+  @EnumSource(
+      value = TableSchemaType.class,
+      names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"})
+  public void testSendRecordsToAllBucketsEvenly(TableSchemaType 
tableSchemaType) throws Exception {
+    setupEnvironment(tableSchemaType);
+    List<RowData> rows = generateTestDataRows();
+
+    testWriteRowData(rows);
+    TableTestStats stats = extractPartitionResults(tableSchemaType);
+
+    Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size());
+    // All 4 buckets should've been written to
+    Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets);
+    
Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets);
+    // Writer expectation (2 writers per bucket):
+    // - Bucket0 -> Writers [0, 4]
+    // - Bucket1 -> Writers [1, 5]
+    // - Bucket2 -> Writers [2, 6]
+    // - Bucket3 -> Writers [3, 7]
+    for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) {
+      
Assertions.assertThat(stats.writersPerBucket.get(i)).isEqualTo(Arrays.asList(i, 
j));
+      // 2 files per bucket (one file is created by each writer)
+      Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(2);
+      // 2 rows per file (total of 16 rows across 8 files)
+      Assertions.assertThat((long) stats.rowsPerFile.get(i)).isEqualTo(2);
+    }
+  }
+
+  /**
+   * Verifies the BucketPartitioner is not used when the PartitionSpec has 
more than 1 bucket, and
+   * that it should fallback to input.keyBy
+   */
+  @ParameterizedTest
+  @EnumSource(value = TableSchemaType.class, names = "TWO_BUCKETS")
+  public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) 
throws Exception {
+    setupEnvironment(tableSchemaType);
+    List<RowData> rows = generateTestDataRows();
+
+    testWriteRowData(rows);
+    TableTestStats stats = extractPartitionResults(tableSchemaType);
+
+    Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size());
+    for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) {
+      // Only 1 file per bucket will be created when falling back to 
input.keyBy(...)
+      Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(1);
+    }
+  }
+
+  /**
+   * Generating 16 rows to be sent uniformly to all writers (round-robin 
across 8 writers -> 4
+   * buckets)
+   */
+  @NotNull
+  private List<RowData> generateTestDataRows() {
+    int totalNumRows = parallelism * 2;
+    int numRowsPerBucket = totalNumRows / numBuckets;
+    return 
TestBucketPartitionerUtils.generateRowsForBucketIdRange(numRowsPerBucket, 
numBuckets);
+  }
+
+  private TableTestStats extractPartitionResults(TableSchemaType 
tableSchemaType)
+      throws IOException {
+    int totalRecordCount = 0;
+    Map<Integer, List<Integer>> writersPerBucket = Maps.newHashMap(); // 
<BucketId, List<WriterId>>
+    Map<Integer, Integer> filesPerBucket = Maps.newHashMap(); // <BucketId, 
NumFiles>
+    Map<Integer, Long> recordsPerFile = new TreeMap<>(); // <WriterId, 
NumRecords>
+
+    try (CloseableIterable<FileScanTask> fileScanTasks = 
table.newScan().planFiles()) {
+      for (FileScanTask scanTask : fileScanTasks) {
+        long recordCountInFile = scanTask.file().recordCount();
+
+        String[] splitFilePath = scanTask.file().path().toString().split("/");

Review Comment:
   please add an example of the file path for easier understanding of the code.
   
   this approach of parsing file name to extract writerId is a bit fragile 
(with tight coupling of file name convention in `OutputFileFactory`). But I 
don't have a better idea. Maybe we will live with this for now.



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.UUID;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.extension.AfterAllCallback;
+import org.junit.jupiter.api.extension.AfterEachCallback;
+import org.junit.jupiter.api.extension.BeforeAllCallback;
+import org.junit.jupiter.api.extension.BeforeEachCallback;
+import org.junit.jupiter.api.extension.ExtensionContext;
+
+public class HadoopCatalogExtension
+    implements BeforeAllCallback, BeforeEachCallback, AfterAllCallback, 
AfterEachCallback {
+  protected final String database;
+  protected final String tableName;
+
+  protected Path temporaryFolder;

Review Comment:
   it can be passed in similar as the junit4 resource
   
   ```
   public HadoopCatalogResource(TemporaryFolder temporaryFolder, String 
database, String tableName) {
   }
   ```



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+
+    return catalogExtension
+        .catalog()
+        .createTable(
+            TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            partitionSpec,
+            ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, 
format.name()));
+  }
+
+  private void testWriteRowData(List<RowData> allRows) throws Exception {
+    DataFormatConverters.RowConverter converter =
+        new 
DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes());
+
+    DataStream<RowData> dataStream =
+        env.addSource(
+                new BoundedTestSource<>(
+                    
allRows.stream().map(converter::toExternal).toArray(Row[]::new)),
+                ROW_TYPE_INFO)
+            .map(converter::toInternal, 
FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE));
+
+    FlinkSink.forRowData(dataStream)
+        .table(table)
+        .tableLoader(tableLoader)
+        .writeParallelism(parallelism)
+        .distributionMode(DistributionMode.HASH)
+        .append();
+
+    // Execute the program.

Review Comment:
   remove too obvious comment



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+
+    return catalogExtension
+        .catalog()
+        .createTable(
+            TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            partitionSpec,
+            ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, 
format.name()));
+  }
+
+  private void testWriteRowData(List<RowData> allRows) throws Exception {

Review Comment:
   nit: remove `test` from method name. maybe all this `appendRowsToTable`?



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+
+    return catalogExtension
+        .catalog()
+        .createTable(
+            TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            partitionSpec,
+            ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, 
format.name()));
+  }
+
+  private void testWriteRowData(List<RowData> allRows) throws Exception {
+    DataFormatConverters.RowConverter converter =
+        new 
DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes());
+
+    DataStream<RowData> dataStream =
+        env.addSource(
+                new BoundedTestSource<>(
+                    
allRows.stream().map(converter::toExternal).toArray(Row[]::new)),
+                ROW_TYPE_INFO)
+            .map(converter::toInternal, 
FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE));
+
+    FlinkSink.forRowData(dataStream)
+        .table(table)
+        .tableLoader(tableLoader)
+        .writeParallelism(parallelism)
+        .distributionMode(DistributionMode.HASH)
+        .append();
+
+    // Execute the program.
+    env.execute("Test Iceberg DataStream");
+
+    // Assert the iceberg table's records.

Review Comment:
   remove too obvious comment



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+
+    return catalogExtension
+        .catalog()
+        .createTable(
+            TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            partitionSpec,
+            ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, 
format.name()));
+  }
+
+  private void testWriteRowData(List<RowData> allRows) throws Exception {
+    DataFormatConverters.RowConverter converter =
+        new 
DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes());
+
+    DataStream<RowData> dataStream =
+        env.addSource(
+                new BoundedTestSource<>(
+                    
allRows.stream().map(converter::toExternal).toArray(Row[]::new)),
+                ROW_TYPE_INFO)
+            .map(converter::toInternal, 
FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE));
+
+    FlinkSink.forRowData(dataStream)
+        .table(table)
+        .tableLoader(tableLoader)
+        .writeParallelism(parallelism)
+        .distributionMode(DistributionMode.HASH)
+        .append();
+
+    // Execute the program.
+    env.execute("Test Iceberg DataStream");
+
+    // Assert the iceberg table's records.
+    SimpleDataUtil.assertTableRows(table, allRows);
+  }
+
+  @ParameterizedTest
+  @EnumSource(
+      value = TableSchemaType.class,
+      names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"})
+  public void testSendRecordsToAllBucketsEvenly(TableSchemaType 
tableSchemaType) throws Exception {
+    setupEnvironment(tableSchemaType);
+    List<RowData> rows = generateTestDataRows();
+
+    testWriteRowData(rows);
+    TableTestStats stats = extractPartitionResults(tableSchemaType);
+
+    Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size());
+    // All 4 buckets should've been written to
+    Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets);
+    
Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets);
+    // Writer expectation (2 writers per bucket):
+    // - Bucket0 -> Writers [0, 4]
+    // - Bucket1 -> Writers [1, 5]
+    // - Bucket2 -> Writers [2, 6]
+    // - Bucket3 -> Writers [3, 7]
+    for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) {
+      
Assertions.assertThat(stats.writersPerBucket.get(i)).isEqualTo(Arrays.asList(i, 
j));
+      // 2 files per bucket (one file is created by each writer)
+      Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(2);
+      // 2 rows per file (total of 16 rows across 8 files)
+      Assertions.assertThat((long) stats.rowsPerFile.get(i)).isEqualTo(2);
+    }
+  }
+
+  /**
+   * Verifies the BucketPartitioner is not used when the PartitionSpec has 
more than 1 bucket, and
+   * that it should fallback to input.keyBy
+   */
+  @ParameterizedTest
+  @EnumSource(value = TableSchemaType.class, names = "TWO_BUCKETS")
+  public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) 
throws Exception {
+    setupEnvironment(tableSchemaType);
+    List<RowData> rows = generateTestDataRows();
+
+    testWriteRowData(rows);
+    TableTestStats stats = extractPartitionResults(tableSchemaType);
+
+    Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size());
+    for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) {
+      // Only 1 file per bucket will be created when falling back to 
input.keyBy(...)
+      Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(1);
+    }
+  }
+
+  /**
+   * Generating 16 rows to be sent uniformly to all writers (round-robin 
across 8 writers -> 4
+   * buckets)
+   */
+  @NotNull
+  private List<RowData> generateTestDataRows() {
+    int totalNumRows = parallelism * 2;
+    int numRowsPerBucket = totalNumRows / numBuckets;
+    return 
TestBucketPartitionerUtils.generateRowsForBucketIdRange(numRowsPerBucket, 
numBuckets);
+  }
+
+  private TableTestStats extractPartitionResults(TableSchemaType 
tableSchemaType)
+      throws IOException {
+    int totalRecordCount = 0;
+    Map<Integer, List<Integer>> writersPerBucket = Maps.newHashMap(); // 
<BucketId, List<WriterId>>

Review Comment:
   the comments at the end of lines are unnecessary



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink;
+
+import static 
org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG;
+import static org.apache.iceberg.flink.TestFixtures.DATABASE;
+import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.flink.HadoopCatalogExtension;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.TestFixtures;
+import 
org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType;
+import org.apache.iceberg.flink.source.BoundedTestSource;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.assertj.core.api.Assertions;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+public class TestBucketPartitionerFlinkIcebergSink {
+
+  private static final int NUMBER_TASK_MANAGERS = 1;
+  private static final int SLOTS_PER_TASK_MANAGER = 8;
+
+  @RegisterExtension
+  private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+      new MiniClusterExtension(
+          new MiniClusterResourceConfiguration.Builder()
+              .setNumberTaskManagers(NUMBER_TASK_MANAGERS)
+              .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+              .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)
+              .build());
+
+  @RegisterExtension
+  private static final HadoopCatalogExtension catalogExtension =
+      new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE);
+
+  private static final TypeInformation<Row> ROW_TYPE_INFO =
+      new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes());
+
+  // Parallelism = 8 (parallelism > numBuckets) throughout the test suite
+  private final int parallelism = NUMBER_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+  private final FileFormat format = FileFormat.PARQUET;
+  private final int numBuckets = 4;
+
+  private Table table;
+  private StreamExecutionEnvironment env;
+  private TableLoader tableLoader;
+
+  private void setupEnvironment(TableSchemaType tableSchemaType) {
+    table = getTable(tableSchemaType);
+    env =
+        
StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG)
+            .enableCheckpointing(100)
+            .setParallelism(parallelism)
+            .setMaxParallelism(parallelism * 2);
+    tableLoader = catalogExtension.tableLoader();
+  }
+
+  private Table getTable(TableSchemaType tableSchemaType) {
+    PartitionSpec partitionSpec = 
TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets);
+
+    return catalogExtension
+        .catalog()
+        .createTable(
+            TABLE_IDENTIFIER,
+            SimpleDataUtil.SCHEMA,
+            partitionSpec,
+            ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, 
format.name()));
+  }
+
+  private void testWriteRowData(List<RowData> allRows) throws Exception {
+    DataFormatConverters.RowConverter converter =
+        new 
DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes());
+
+    DataStream<RowData> dataStream =
+        env.addSource(
+                new BoundedTestSource<>(
+                    
allRows.stream().map(converter::toExternal).toArray(Row[]::new)),
+                ROW_TYPE_INFO)
+            .map(converter::toInternal, 
FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE));
+
+    FlinkSink.forRowData(dataStream)
+        .table(table)
+        .tableLoader(tableLoader)
+        .writeParallelism(parallelism)
+        .distributionMode(DistributionMode.HASH)
+        .append();
+
+    // Execute the program.
+    env.execute("Test Iceberg DataStream");
+
+    // Assert the iceberg table's records.
+    SimpleDataUtil.assertTableRows(table, allRows);
+  }
+
+  @ParameterizedTest
+  @EnumSource(
+      value = TableSchemaType.class,
+      names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"})
+  public void testSendRecordsToAllBucketsEvenly(TableSchemaType 
tableSchemaType) throws Exception {
+    setupEnvironment(tableSchemaType);
+    List<RowData> rows = generateTestDataRows();
+
+    testWriteRowData(rows);
+    TableTestStats stats = extractPartitionResults(tableSchemaType);
+
+    Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size());
+    // All 4 buckets should've been written to
+    Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets);
+    
Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets);
+    // Writer expectation (2 writers per bucket):
+    // - Bucket0 -> Writers [0, 4]
+    // - Bucket1 -> Writers [1, 5]
+    // - Bucket2 -> Writers [2, 6]
+    // - Bucket3 -> Writers [3, 7]
+    for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) {
+      
Assertions.assertThat(stats.writersPerBucket.get(i)).isEqualTo(Arrays.asList(i, 
j));
+      // 2 files per bucket (one file is created by each writer)
+      Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(2);

Review Comment:
   why do we need the type cast?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to