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


##########
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:
   we probably should try to follow existing style where value is after the 
semicolon. maybe we can use the style as this code?
   ```
       Preconditions.checkArgument(
           numBuckets > 0, "Invalid number of buckets: %s (must be > 0)", 
numBuckets);
   ```



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.collect.Lists;
+import org.apache.iceberg.util.BucketUtil;
+
+final class TestBucketPartitionerUtil {
+
+  enum TableSchemaType {
+    ONE_BUCKET {
+      @Override
+      public int bucketPartitionColumnPosition() {
+        return 0;
+      }
+
+      @Override
+      public PartitionSpec getPartitionSpec(int numBuckets) {
+        return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", 
numBuckets).build();
+      }
+    },
+    IDENTITY_AND_BUCKET {
+      @Override
+      public PartitionSpec getPartitionSpec(int numBuckets) {
+        return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA)
+            .identity("id")
+            .bucket("data", numBuckets)
+            .build();
+      }
+    },
+    TWO_BUCKETS {
+      @Override
+      public PartitionSpec getPartitionSpec(int numBuckets) {
+        return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA)
+            .bucket("id", numBuckets)
+            .bucket("data", numBuckets)
+            .build();
+      }
+    };
+
+    public int bucketPartitionColumnPosition() {

Review Comment:
   nit: I find it easier to read to have this as abstract. I know it will be 
the same for the last 2 enums.



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