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


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.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) {
+    this.schema = schema;
+    this.partitionKey = new PartitionKey(partitionSpec, schema);
+    this.flinkSchema = flinkSchema;
+    int bucketFieldId = 
BucketPartitionerUtil.getBucketFieldInfo(partitionSpec).f0;
+    this.bucketFieldPosition =

Review Comment:
   can we combine this with the line above in a method called 
`getBucketFieldPosition`?



##########
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:
   you can just add the type info. it should take care of the serialization 
part.
   ```
         DataStream<RowData> stream =
             env.fromSource(
                 IcebergSource.<RowData>builder()
                     .tableLoader(tableLoader)
                     .assignerFactory(new SimpleSplitAssignerFactory())
                     .project(projected)
                     .build(),
                 WatermarkStrategy.noWatermarks(),
                 "testBasicRead",
                 TypeInformation.of(RowData.class));
   ```



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.TestBucketPartitionerUtil.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) {
+    int numPartitions = 500;
+    TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr);
+    int numBuckets = Integer.parseInt(numBucketsStr);
+    PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets);
+    BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec);
+
+    int bucketId = 0;
+    for (int expectedIdx = 0; expectedIdx < numPartitions; expectedIdx++) {

Review Comment:
   nit: idx -> index



##########
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;
+  protected Catalog catalog;
+  protected CatalogLoader catalogLoader;
+  protected String warehouse;
+  protected TableLoader tableLoader;
+
+  public HadoopCatalogExtension(String database, String tableName) {
+    this.database = database;
+    this.tableName = tableName;
+  }
+
+  @Override
+  public void beforeAll(ExtensionContext context) throws Exception {
+    this.temporaryFolder = Files.createTempDirectory("junit5_hadoop_catalog-");
+  }
+
+  @Override
+  public void afterAll(ExtensionContext context) throws Exception {
+    FileUtils.deleteDirectory(this.temporaryFolder.toFile());

Review Comment:
   nit: iceberg style only uses `this.` when setting/changing values (not 
during read). this applies to a few other places in the main code.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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 =
+      "Invalid bucket ID %s: must be non-negative.";
+  static final String BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE =
+      "Invalid bucket ID %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 of buckets
+  private final int[] currentBucketWriterOffset;
+
+  BucketPartitioner(PartitionSpec partitionSpec) {
+    Tuple2<Integer, Integer> bucketFieldInfo =
+        BucketPartitionerUtil.getBucketFieldInfo(partitionSpec);

Review Comment:
   it would probably be a bit cleaned to expose a method 
`getBucketNumber(PartitionSpec)`. then we can avoid the tuple of `.f1`



##########
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:
   that's fair



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