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


##########
api/src/main/java/org/apache/iceberg/PartitionSpec.java:
##########
@@ -289,6 +289,23 @@ public Set<Integer> identitySourceIds() {
     return sourceIds;
   }
 
+  /** Determine if this PartitionSpec contains at least 1 bucket definition */
+  public boolean hasBucketPartition() {
+    List<PartitionField> partFields = fields();
+    boolean hasBucketPartition = false;
+
+    for (int i = 0; i < partFields.size(); i++) {
+      PartitionField field = partFields.get(i);
+      // Is there a more elegant way to know? (all Bucket* constructs are 
package private)
+      if (field.transform().dedupName().toLowerCase().contains("bucket")) {
+        hasBucketPartition = true;

Review Comment:
   nit: may not need this variable. just return true?



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.functions.KeySelector;
+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;
+
+/**
+ * Very similar to the {@link PartitionKeySelector}, but optimized to extract 
and return an Integer
+ * bucketId as the key.
+ */
+public class BucketPartitionKeySelector implements KeySelector<RowData, 
Integer> {
+
+  private final Schema schema;
+  private final PartitionKey partitionKey;
+  private final RowType flinkSchema;
+  private static final Pattern intPattern = Pattern.compile("\\d+");
+
+  private transient RowDataWrapper rowDataWrapper;
+
+  BucketPartitionKeySelector(PartitionSpec spec, Schema schema, RowType 
flinkSchema) {
+    this.schema = schema;
+    this.partitionKey = new PartitionKey(spec, schema);
+    this.flinkSchema = flinkSchema;
+  }
+
+  /**
+   * Construct the {@link RowDataWrapper} lazily here because few members in 
it are not
+   * serializable. In this way, we don't have to serialize them with forcing.
+   */
+  private RowDataWrapper lazyRowDataWrapper() {
+    if (rowDataWrapper == null) {
+      rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct());
+    }
+    return rowDataWrapper;
+  }
+
+  @Override
+  public Integer getKey(RowData rowData) throws Exception {
+    partitionKey.partition(lazyRowDataWrapper().wrap(rowData));
+    Optional<Integer> bucketId = 
BucketPartitionKeySelector.extractInteger(partitionKey.toPath());
+
+    if (!bucketId.isPresent()) {
+      throw new IllegalStateException("Unable to extract bucket in partition 
key: " + partitionKey);
+    }
+
+    return bucketId.get();
+  }
+
+  /**
+   * Utility method to extract a bucketId from a string value. Input examples: 
bucket[5] or
+   * bucket_data[11], would return 5 and 11 bucket Ids respectively.
+   *
+   * @param value the string with the Bucket Id
+   * @return the Bucket Id as an Integer or Empty if not found
+   */
+  public static Optional<Integer> extractInteger(String value) {

Review Comment:
   this is a very hack way to convert PartitionKey to a string and use regex 
match. we should figure out a more direct way. E.g., we can get the partition 
value from tuples. we can know which partition position is for bucketing 
partition



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Optional;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+
+/**
+ * This partitioner will redirect elements to writers deterministically so 
that each writer only
+ * targets 1 bucket. If the number of writers > number of buckets each 
partitioner will keep a state
+ * of multiple writers per bucket as evenly as possible, and will round-robin 
the requests across
+ * them.
+ */
+public class BucketPartitioner implements Partitioner<Integer> {
+
+  private final Integer maxBuckets;
+
+  private final int[] currentWriterOffset;
+
+  BucketPartitioner(PartitionSpec partitionSpec) {
+    // The current implementation redirects to writers based on the _FIRST_ 
bucket found
+    Optional<PartitionField> bucket =
+        partitionSpec.fields().stream()
+            .filter(f -> f.transform().dedupName().contains("bucket"))
+            .findFirst();
+
+    if (!bucket.isPresent()) {

Review Comment:
   Iceberg coding style uses `Preconditions` for those check



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Optional;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+
+/**
+ * This partitioner will redirect elements to writers deterministically so 
that each writer only
+ * targets 1 bucket. If the number of writers > number of buckets each 
partitioner will keep a state
+ * of multiple writers per bucket as evenly as possible, and will round-robin 
the requests across
+ * them.
+ */
+public class BucketPartitioner implements Partitioner<Integer> {
+
+  private final Integer maxBuckets;
+
+  private final int[] currentWriterOffset;
+
+  BucketPartitioner(PartitionSpec partitionSpec) {
+    // The current implementation redirects to writers based on the _FIRST_ 
bucket found
+    Optional<PartitionField> bucket =
+        partitionSpec.fields().stream()
+            .filter(f -> f.transform().dedupName().contains("bucket"))
+            .findFirst();
+
+    if (!bucket.isPresent()) {

Review Comment:
   since this impl doesn't support multiple bucket partitions, we should also 
validate and fail explicitly for that scenario. we should also update the 
document for this feature and explicitly point out this limitation.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.functions.KeySelector;
+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;
+
+/**
+ * Very similar to the {@link PartitionKeySelector}, but optimized to extract 
and return an Integer
+ * bucketId as the key.
+ */
+public class BucketPartitionKeySelector implements KeySelector<RowData, 
Integer> {

Review Comment:
   make this class package private



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Optional;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+
+/**
+ * This partitioner will redirect elements to writers deterministically so 
that each writer only

Review Comment:
   > each writer only targets 1 bucket
   
   this doesn't seem accurate. at least, it can be interpreted incorrectly. 



##########
api/src/main/java/org/apache/iceberg/PartitionSpec.java:
##########
@@ -289,6 +289,23 @@ public Set<Integer> identitySourceIds() {
     return sourceIds;
   }
 
+  /** Determine if this PartitionSpec contains at least 1 bucket definition */
+  public boolean hasBucketPartition() {

Review Comment:
   Let's not change core for now. we can keep this util method inside the flink 
module for now



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Optional;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+
+/**
+ * This partitioner will redirect elements to writers deterministically so 
that each writer only
+ * targets 1 bucket. If the number of writers > number of buckets each 
partitioner will keep a state
+ * of multiple writers per bucket as evenly as possible, and will round-robin 
the requests across
+ * them.
+ */
+public class BucketPartitioner implements Partitioner<Integer> {

Review Comment:
   please make the class package private.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Optional;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+
+/**
+ * This partitioner will redirect elements to writers deterministically so 
that each writer only
+ * targets 1 bucket. If the number of writers > number of buckets each 
partitioner will keep a state
+ * of multiple writers per bucket as evenly as possible, and will round-robin 
the requests across
+ * them.
+ */
+public class BucketPartitioner implements Partitioner<Integer> {
+
+  private final Integer maxBuckets;

Review Comment:
   nit: this can be primitive `int`



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Optional;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+
+/**
+ * This partitioner will redirect elements to writers deterministically so 
that each writer only
+ * targets 1 bucket. If the number of writers > number of buckets each 
partitioner will keep a state
+ * of multiple writers per bucket as evenly as possible, and will round-robin 
the requests across
+ * them.
+ */
+public class BucketPartitioner implements Partitioner<Integer> {
+
+  private final Integer maxBuckets;
+
+  private final int[] currentWriterOffset;
+
+  BucketPartitioner(PartitionSpec partitionSpec) {
+    // The current implementation redirects to writers based on the _FIRST_ 
bucket found
+    Optional<PartitionField> bucket =
+        partitionSpec.fields().stream()
+            .filter(f -> f.transform().dedupName().contains("bucket"))

Review Comment:
   why don't we directly check the class type is an instance of `Bucket`?



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Optional;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+
+/**
+ * This partitioner will redirect elements to writers deterministically so 
that each writer only
+ * targets 1 bucket. If the number of writers > number of buckets each 
partitioner will keep a state
+ * of multiple writers per bucket as evenly as possible, and will round-robin 
the requests across
+ * them.
+ */
+public class BucketPartitioner implements Partitioner<Integer> {
+
+  private final Integer maxBuckets;
+
+  private final int[] currentWriterOffset;
+
+  BucketPartitioner(PartitionSpec partitionSpec) {
+    // The current implementation redirects to writers based on the _FIRST_ 
bucket found
+    Optional<PartitionField> bucket =
+        partitionSpec.fields().stream()
+            .filter(f -> f.transform().dedupName().contains("bucket"))
+            .findFirst();
+
+    if (!bucket.isPresent()) {
+      throw new IllegalStateException("No buckets found on the provided 
PartitionSpec");
+    }
+
+    // Extracting the max number of buckets defined in the partition spec
+    String transformName = bucket.get().transform().dedupName();
+    Optional<Integer> maxBucketsOpt = 
BucketPartitionKeySelector.extractInteger(transformName);
+
+    if (maxBucketsOpt.isPresent()) {
+      this.maxBuckets = maxBucketsOpt.get();
+    } else {
+      throw new IllegalStateException(
+          "Could not extract the max number of buckets from the transform name 
("
+              + transformName
+              + ")");
+    }
+
+    this.currentWriterOffset = new int[this.maxBuckets];
+  }
+
+  @Override
+  public int partition(Integer bucketId, int numPartitions) {
+    if (numPartitions > maxBuckets) {
+      return getSubpartitionIdx(bucketId, numPartitions);
+    } else {
+      return bucketId % numPartitions;
+    }
+  }
+
+  private int getSubpartitionIdx(int bucketId, int numPartitions) {

Review Comment:
   nit: getPartitionIndex? Iceberg coding style typically don't use 
abbreviation like `idx`.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Optional;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+
+/**
+ * This partitioner will redirect elements to writers deterministically so 
that each writer only
+ * targets 1 bucket. If the number of writers > number of buckets each 
partitioner will keep a state
+ * of multiple writers per bucket as evenly as possible, and will round-robin 
the requests across
+ * them.
+ */
+public class BucketPartitioner implements Partitioner<Integer> {
+
+  private final Integer maxBuckets;
+
+  private final int[] currentWriterOffset;
+
+  BucketPartitioner(PartitionSpec partitionSpec) {
+    // The current implementation redirects to writers based on the _FIRST_ 
bucket found
+    Optional<PartitionField> bucket =
+        partitionSpec.fields().stream()
+            .filter(f -> f.transform().dedupName().contains("bucket"))
+            .findFirst();
+
+    if (!bucket.isPresent()) {
+      throw new IllegalStateException("No buckets found on the provided 
PartitionSpec");
+    }
+
+    // Extracting the max number of buckets defined in the partition spec
+    String transformName = bucket.get().transform().dedupName();
+    Optional<Integer> maxBucketsOpt = 
BucketPartitionKeySelector.extractInteger(transformName);
+
+    if (maxBucketsOpt.isPresent()) {
+      this.maxBuckets = maxBucketsOpt.get();
+    } else {
+      throw new IllegalStateException(

Review Comment:
   Iceberg coding style uses `Preconditions` for this type of check



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.Optional;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+
+/**
+ * This partitioner will redirect elements to writers deterministically so 
that each writer only
+ * targets 1 bucket. If the number of writers > number of buckets each 
partitioner will keep a state
+ * of multiple writers per bucket as evenly as possible, and will round-robin 
the requests across
+ * them.
+ */
+public class BucketPartitioner implements Partitioner<Integer> {
+
+  private final Integer maxBuckets;
+
+  private final int[] currentWriterOffset;
+
+  BucketPartitioner(PartitionSpec partitionSpec) {
+    // The current implementation redirects to writers based on the _FIRST_ 
bucket found
+    Optional<PartitionField> bucket =
+        partitionSpec.fields().stream()
+            .filter(f -> f.transform().dedupName().contains("bucket"))
+            .findFirst();
+
+    if (!bucket.isPresent()) {
+      throw new IllegalStateException("No buckets found on the provided 
PartitionSpec");
+    }
+
+    // Extracting the max number of buckets defined in the partition spec
+    String transformName = bucket.get().transform().dedupName();
+    Optional<Integer> maxBucketsOpt = 
BucketPartitionKeySelector.extractInteger(transformName);
+
+    if (maxBucketsOpt.isPresent()) {
+      this.maxBuckets = maxBucketsOpt.get();
+    } else {
+      throw new IllegalStateException(
+          "Could not extract the max number of buckets from the transform name 
("
+              + transformName
+              + ")");
+    }
+
+    this.currentWriterOffset = new int[this.maxBuckets];
+  }
+
+  @Override
+  public int partition(Integer bucketId, int numPartitions) {
+    if (numPartitions > maxBuckets) {
+      return getSubpartitionIdx(bucketId, numPartitions);
+    } else {
+      return bucketId % numPartitions;
+    }
+  }
+
+  private int getSubpartitionIdx(int bucketId, int numPartitions) {

Review Comment:
   also the logic in this method is not easy to understand. can you provide 
some illustration on how this works.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

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


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

Reply via email to