From 70846120b274be2edb0384786fee5e09be04fbf0 Mon Sep 17 00:00:00 2001 From: schongloo Date: Tue, 21 Mar 2023 09:56:14 -0700 Subject: [PATCH 01/23] First batch of code for a new Bucket Partitioning functionality --- .../org/apache/iceberg/PartitionSpec.java | 19 + .../sink/BucketPartitionKeySelector.java | 88 +++++ .../iceberg/flink/sink/BucketPartitioner.java | 92 +++++ .../apache/iceberg/flink/sink/FlinkSink.java | 6 +- .../flink/sink/TestBucketPartitioner.java | 326 ++++++++++++++++++ 5 files changed, 530 insertions(+), 1 deletion(-) create mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java create mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java index a31cfd76583b..884f2b19cd7f 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -304,6 +304,25 @@ public String toString() { return sb.toString(); } + /** + * Determine if this PartitionSpec contains at least 1 bucket definition + */ + public boolean hasBucketPartition() { + List 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; + break; + } + } + + return hasBucketPartition; + } + private static final PartitionSpec UNPARTITIONED_SPEC = new PartitionSpec(new Schema(), 0, ImmutableList.of(), unpartitionedLastAssignedId()); diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java new file mode 100644 index 000000000000..6e906192ca28 --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -0,0 +1,88 @@ +/* + * + * * 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.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; + +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Very similar to the {@link PartitionKeySelector}, + * but optimized to extract and return an Integer bucketId as the key. + */ +public class BucketPartitionKeySelector implements KeySelector { + + private final Schema schema; + private final PartitionKey partitionKey; + private final RowType flinkSchema; + private static final Pattern p = 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 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 name. + * Input examples: bucket[5] or bucket_data[11], would return 5 and 11 bucket Ids respectively. + * @param name + * @return + */ + public static Optional extractInteger(String name) { + Matcher m = p.matcher(name); + return m.find() ? Optional.of(Integer.parseInt(m.group())) : Optional.empty(); + } +} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java new file mode 100644 index 000000000000..0a9cc5088253 --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -0,0 +1,92 @@ +/* + * + * * 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.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * 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 { + + private final Integer maxBuckets; + + private final Map currentBucketOffset; + + BucketPartitioner(PartitionSpec partitionSpec) { + // The current implementation redirects to writers based on the _FIRST_ bucket found + Optional 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 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.currentBucketOffset = new HashMap<>(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) { + int currentOffset = -1; + + if (currentBucketOffset.containsKey(bucketId)) { + currentOffset = currentBucketOffset.get(bucketId); + int maxBucketNumOffsets = (numPartitions / maxBuckets) + (bucketId < (numPartitions % maxBuckets) ? 1 : 0); + + // Reset the offset when necessary + if (currentOffset == maxBucketNumOffsets - 1) { + currentOffset = -1; + } + } + + currentBucketOffset.put(bucketId, ++ currentOffset); + + return bucketId + (maxBuckets * currentOffset); + } +} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 445b6a6ff97a..403e1cbd3a20 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -505,7 +505,11 @@ private DataStream distributeDataStream( + "and table is unpartitioned"); return input; } else { - return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + if (partitionSpec.hasBucketPartition()) { + return input.partitionCustom(new BucketPartitioner(partitionSpec), new BucketPartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + } else { + return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + } } } else { if (partitionSpec.isUnpartitioned()) { diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java new file mode 100644 index 000000000000..521b025060a1 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -0,0 +1,326 @@ +/* + * + * * 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.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.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.*; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +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.util.BucketUtil; +import org.jetbrains.annotations.NotNull; +import org.junit.*; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.*; +import java.util.stream.Collectors; + +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; + +public class TestBucketPartitioner { + + private static final int NUMBER_TASK_MANAGERS = 1; + private static final int SLOTS_PER_TASK_MANAGER = 8; + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUMBER_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @ClassRule + public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, DATABASE, TestFixtures.TABLE); + + private static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + private static final DataFormatConverters.RowConverter CONVERTER = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + + private Table table; + private StreamExecutionEnvironment env; + private TableLoader tableLoader; + + // Simple parallelism = 8 throughout the test suite + private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; + private final FileFormat format = FileFormat.fromString("parquet"); + private final int numBuckets = 4; + + @Before + public void before() throws IOException { + table = catalogResource + .catalog() + .createTable( + TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism * 2); + + tableLoader = catalogResource.tableLoader(); + } + + private List convertToRowData(List rows) { + return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); + } + + private BoundedTestSource createBoundedSource(List rows) { + return new BoundedTestSource<>(rows.toArray(new Row[0])); + } + + private TableTestStats extractTableTestStats() throws IOException { + // Assertions: + // - Total record count in table + // - All records belong to that bucket + // - Each bucket should've been written by valid writers + // - Each file should have the expected number of records + + int totalRecordCount = 0; + Map> writersPerBucket = new HashMap<>(); // > + Map filesPerBucket = new HashMap<>(); // + Map recordsPerFile = new TreeMap<>(); // + + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + for (FileScanTask scanTask : fileScanTasks) { + Optional bucketIdOpt = + BucketPartitionKeySelector.extractInteger(scanTask.file().partition().toString()); + + 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 = bucketIdOpt.get(); + writersPerBucket.computeIfAbsent(bucketId, k -> new ArrayList<>()); + 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()) { + Collections.sort(writersPerBucket.get(k)); + } + + return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, recordsPerFile); + } + + private void testWriteRowData(List allRows) throws Exception { + DataStream dataStream = + env.addSource(createBoundedSource(allRows), 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, convertToRowData(allRows)); + } + + // Test 1A: N records sent to the same bucket across all partitions + // - 24 rows (3 batches w/overlap), 4 buckets + @Test + public void testSendToBucket0A() throws Exception { + // Data generation + // We make the number of requests a factor of the parallelism for an easier verification + int totalNumRows = parallelism * 3; // 24 rows + TableTestStats stats = sendAndAssertBucket0(totalNumRows); + // Number of records per file + Assert.assertEquals(16, stats.recordsPerFile.get(0).intValue()); + Assert.assertEquals(8, stats.recordsPerFile.get(4).intValue()); + } + + // Test 1B: N records sent to the same bucket across all partitions + // - 30 rows (4 batches w/o overlap, uneven first bucket), 4 buckets + @Test + public void testSendToBucket0B() throws Exception { + // Data generation + // We make the number of requests a factor of the parallelism for an easier verification + int totalNumRows = 30; + TableTestStats stats = sendAndAssertBucket0(totalNumRows); + // Number of records per file + Assert.assertEquals(16, stats.recordsPerFile.get(0).intValue()); + Assert.assertEquals(14, stats.recordsPerFile.get(4).intValue()); + } + + @NotNull + private TableTestStats sendAndAssertBucket0(int totalNumRows) throws Exception { + List rows = generateRowsForBucketId(totalNumRows, 0, numBuckets); + // Writing to the table + testWriteRowData(rows); + // Extracting stats + TableTestStats stats = extractTableTestStats(); + + // Assertions + // All the records were received + Assert.assertEquals(totalNumRows, stats.totalRecordCount); + // Only bucketId = 0 was written + Assert.assertEquals(1, stats.writersPerBucket.size()); + Assert.assertEquals(1, stats.filesPerBucket.size()); + Assert.assertTrue(stats.writersPerBucket.containsKey(0)); + Assert.assertTrue(stats.filesPerBucket.containsKey(0)); + // Valid writers + List validWriters = Arrays.asList(0, 4); + Assert.assertEquals(validWriters, stats.writersPerBucket.get(0)); + // Num writers = num created files + Assert.assertEquals(validWriters.size(), stats.filesPerBucket.get(0).intValue()); + // TODO: verify all the records actually hash to bucketId = 0 + return stats; + } + + // Test 2: N records of each bucket value, to all buckets, through the same partitions + // - 16 rows (2 writers per bucket, uneven last bucket), 4 buckets + @Test + public void testSendRecordsToAllBucketsEvenly() throws Exception { + // Data generation + // We make the number of requests a factor of the parallelism for an easier verification + int totalNumRows = parallelism * 2; + int numRowsPerBucket = totalNumRows / numBuckets; + List rows = new ArrayList<>(totalNumRows); + List templateRows = new ArrayList<>(numBuckets); + for (int i = 0; i < numBuckets; i++) { + templateRows.addAll(generateRowsForBucketId(1, i, numBuckets)); + } + for (int i = 0; i < numRowsPerBucket; i++) { + rows.addAll(templateRows); + } + + // Writing to the table + testWriteRowData(rows); + // Extracting stats + TableTestStats stats = extractTableTestStats(); + + // Assertions + // All the records were received + Assert.assertEquals(totalNumRows, stats.totalRecordCount); + // Only bucketId = 0 was written + Assert.assertEquals(numBuckets, stats.writersPerBucket.size()); + Assert.assertEquals(numBuckets, stats.filesPerBucket.size()); + for (int i = 0, j = 4; i < numBuckets; i++, j++) { + Assert.assertEquals(Arrays.asList(i, j), stats.writersPerBucket.get(i)); + Assert.assertTrue(stats.filesPerBucket.get(i) == 2); + } + } + + /** + * Utility method to generate rows whose values will "hash" to a desired bucketId + * @param numRows + * @param bucketId + * @return + */ + private List generateRowsForBucketId(int numRows, int bucketId, int numBuckets) { + List rows = new ArrayList<>(numRows); + for(int i = 0; i < numRows; i ++) { + String value = generateValueForBucketId(bucketId, numBuckets); + rows.add(Row.of(i, value)); + } + return rows; + } + + /** + * Utility method to generate a string value that will "hash" to a desired bucketId + * @param bucketId + * @param maxNumBuckets + * @return + */ + private String generateValueForBucketId(int bucketId, int maxNumBuckets) { + String value = ""; + while (true) { + String uuid = UUID.randomUUID().toString(); + // This is the same "hashing" mechanism used by Bucket.java + if ((BucketUtil.hash(uuid) & Integer.MAX_VALUE) % maxNumBuckets == bucketId) { + value = uuid; + break; + } + } + return value; + } + + /** + * DTO to hold Test Stats + */ + private static class TableTestStats { + final int totalRecordCount; + final Map> writersPerBucket; // > + final Map filesPerBucket; // + final Map recordsPerFile; // + + TableTestStats(int totalRecordCount, + Map> writersPerBucket, + Map filesPerBucket, + Map recordsPerFile) { + this.totalRecordCount = totalRecordCount; + this.writersPerBucket = writersPerBucket; + this.filesPerBucket = filesPerBucket; + this.recordsPerFile = recordsPerFile; + } + } + + // Tests (all with parallelism = 8): + // Test 1: N records sent to the same bucket across all partitions + // - 24 rows (3 batches w/overlap), 4 buckets + // - 30 rows (4 batches w/o overlap, uneven first bucket), 4 buckets + // Test 2: N records of each bucket value, to all buckets, through the same partitions + // - 16 rows (2 writers per bucket, uneven last bucket), 3 buckets + + // About test parameters: + // - Constant parallelism (#TMs and #Slots) + // - Parallelism evenly divisible by #buckets + // - Parallelism not evenly divisible by #buckets +} From 5eaf7180bce2bfd614e127f1e3ec4f052ef0187a Mon Sep 17 00:00:00 2001 From: schongloo Date: Wed, 22 Mar 2023 16:55:13 -0700 Subject: [PATCH 02/23] Updating code style. --- .../org/apache/iceberg/PartitionSpec.java | 34 +- .../sink/BucketPartitionKeySelector.java | 119 +++---- .../iceberg/flink/sink/BucketPartitioner.java | 126 ++++--- .../apache/iceberg/flink/sink/FlinkSink.java | 4 +- .../flink/sink/TestBucketPartitioner.java | 326 ----------------- ...TestBucketPartitionerFlinkIcebergSink.java | 336 ++++++++++++++++++ 6 files changed, 473 insertions(+), 472 deletions(-) delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java index 884f2b19cd7f..d48bb6b4fcbb 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -289,24 +289,7 @@ public Set identitySourceIds() { return sourceIds; } - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("["); - for (PartitionField field : fields) { - sb.append("\n"); - sb.append(" ").append(field); - } - if (fields.length > 0) { - sb.append("\n"); - } - sb.append("]"); - return sb.toString(); - } - - /** - * Determine if this PartitionSpec contains at least 1 bucket definition - */ + /** Determine if this PartitionSpec contains at least 1 bucket definition */ public boolean hasBucketPartition() { List partFields = fields(); boolean hasBucketPartition = false; @@ -323,6 +306,21 @@ public boolean hasBucketPartition() { return hasBucketPartition; } + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("["); + for (PartitionField field : fields) { + sb.append("\n"); + sb.append(" ").append(field); + } + if (fields.length > 0) { + sb.append("\n"); + } + sb.append("]"); + return sb.toString(); + } + private static final PartitionSpec UNPARTITIONED_SPEC = new PartitionSpec(new Schema(), 0, ImmutableList.of(), unpartitionedLastAssignedId()); diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index 6e906192ca28..e12a46320613 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -1,26 +1,26 @@ /* + * 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 * - * * 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. + * 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; @@ -29,60 +29,57 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.flink.RowDataWrapper; -import java.util.Optional; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - /** - * Very similar to the {@link PartitionKeySelector}, - * but optimized to extract and return an Integer bucketId as the key. + * Very similar to the {@link PartitionKeySelector}, but optimized to extract and return an Integer + * bucketId as the key. */ public class BucketPartitionKeySelector implements KeySelector { - private final Schema schema; - private final PartitionKey partitionKey; - private final RowType flinkSchema; - private static final Pattern p = Pattern.compile("\\d+"); + private final Schema schema; + private final PartitionKey partitionKey; + private final RowType flinkSchema; + private static final Pattern intPattern = Pattern.compile("\\d+"); - private transient RowDataWrapper rowDataWrapper; + private transient RowDataWrapper rowDataWrapper; - BucketPartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { - this.schema = schema; - this.partitionKey = new PartitionKey(spec, schema); - this.flinkSchema = flinkSchema; - } + 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; + /** + * 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 bucketId = BucketPartitionKeySelector.extractInteger(partitionKey.toPath()); + @Override + public Integer getKey(RowData rowData) throws Exception { + partitionKey.partition(lazyRowDataWrapper().wrap(rowData)); + Optional bucketId = BucketPartitionKeySelector.extractInteger(partitionKey.toPath()); - if (!bucketId.isPresent()) { - throw new IllegalStateException("Unable to extract bucket in partition key: " + partitionKey); - } - - return bucketId.get(); + if (!bucketId.isPresent()) { + throw new IllegalStateException("Unable to extract bucket in partition key: " + partitionKey); } - /** - * Utility method to extract a bucketId from a string name. - * Input examples: bucket[5] or bucket_data[11], would return 5 and 11 bucket Ids respectively. - * @param name - * @return - */ - public static Optional extractInteger(String name) { - Matcher m = p.matcher(name); - return m.find() ? Optional.of(Integer.parseInt(m.group())) : Optional.empty(); - } + 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 extractInteger(String value) { + Matcher match = intPattern.matcher(value); + return match.find() ? Optional.of(Integer.parseInt(match.group())) : Optional.empty(); + } } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 0a9cc5088253..581079869c3e 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -1,92 +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 * - * * 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. + * 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; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; - /** - * 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. + * 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 { - private final Integer maxBuckets; - - private final Map currentBucketOffset; + private final Integer maxBuckets; - BucketPartitioner(PartitionSpec partitionSpec) { - // The current implementation redirects to writers based on the _FIRST_ bucket found - Optional bucket = - partitionSpec.fields().stream().filter(f -> f.transform().dedupName().contains("bucket")).findFirst(); + private final int[] currentWriterOffset; - if (!bucket.isPresent()) { - throw new IllegalStateException("No buckets found on the provided PartitionSpec"); - } + BucketPartitioner(PartitionSpec partitionSpec) { + // The current implementation redirects to writers based on the _FIRST_ bucket found + Optional bucket = + partitionSpec.fields().stream() + .filter(f -> f.transform().dedupName().contains("bucket")) + .findFirst(); - // Extracting the max number of buckets defined in the partition spec - String transformName = bucket.get().transform().dedupName(); - Optional 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.currentBucketOffset = new HashMap<>(maxBuckets); + if (!bucket.isPresent()) { + throw new IllegalStateException("No buckets found on the provided PartitionSpec"); } - @Override - public int partition(Integer bucketId, int numPartitions) { - if (numPartitions > maxBuckets) { - return getSubpartitionIdx(bucketId, numPartitions); - } else { - return bucketId % numPartitions; - } + // Extracting the max number of buckets defined in the partition spec + String transformName = bucket.get().transform().dedupName(); + Optional 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 + + ")"); } - private int getSubpartitionIdx(int bucketId, int numPartitions) { - int currentOffset = -1; + this.currentWriterOffset = new int[this.maxBuckets]; + } - if (currentBucketOffset.containsKey(bucketId)) { - currentOffset = currentBucketOffset.get(bucketId); - int maxBucketNumOffsets = (numPartitions / maxBuckets) + (bucketId < (numPartitions % maxBuckets) ? 1 : 0); + @Override + public int partition(Integer bucketId, int numPartitions) { + if (numPartitions > maxBuckets) { + return getSubpartitionIdx(bucketId, numPartitions); + } else { + return bucketId % numPartitions; + } + } - // Reset the offset when necessary - if (currentOffset == maxBucketNumOffsets - 1) { - currentOffset = -1; - } - } + private int getSubpartitionIdx(int bucketId, int numPartitions) { + int currentOffset = currentWriterOffset[bucketId]; + // When numPartitions is not evenly divisible by maxBuckets + int extraPad = bucketId < (numPartitions % maxBuckets) ? 1 : 0; + int maxWriterNumOffsets = (numPartitions / maxBuckets) + extraPad; - currentBucketOffset.put(bucketId, ++ currentOffset); + // Reset the offset when necessary + int nextOffset = currentOffset == maxWriterNumOffsets - 1 ? 0 : currentOffset + 1; + currentWriterOffset[bucketId] = nextOffset; - return bucketId + (maxBuckets * currentOffset); - } + return bucketId + (maxBuckets * currentOffset); + } } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 403e1cbd3a20..4a13e4b8bd39 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -506,7 +506,9 @@ private DataStream distributeDataStream( return input; } else { if (partitionSpec.hasBucketPartition()) { - return input.partitionCustom(new BucketPartitioner(partitionSpec), new BucketPartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + return input.partitionCustom( + new BucketPartitioner(partitionSpec), + new BucketPartitionKeySelector(partitionSpec, iSchema, flinkRowType)); } else { return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java deleted file mode 100644 index 521b025060a1..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ /dev/null @@ -1,326 +0,0 @@ -/* - * - * * 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.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.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.*; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -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.util.BucketUtil; -import org.jetbrains.annotations.NotNull; -import org.junit.*; -import org.junit.rules.TemporaryFolder; - -import java.io.IOException; -import java.util.*; -import java.util.stream.Collectors; - -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; - -public class TestBucketPartitioner { - - private static final int NUMBER_TASK_MANAGERS = 1; - private static final int SLOTS_PER_TASK_MANAGER = 8; - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(NUMBER_TASK_MANAGERS) - .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - - @ClassRule - public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, DATABASE, TestFixtures.TABLE); - - private static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - private static final DataFormatConverters.RowConverter CONVERTER = - new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - - private Table table; - private StreamExecutionEnvironment env; - private TableLoader tableLoader; - - // Simple parallelism = 8 throughout the test suite - private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; - private final FileFormat format = FileFormat.fromString("parquet"); - private final int numBuckets = 4; - - @Before - public void before() throws IOException { - table = catalogResource - .catalog() - .createTable( - TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(), - ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism * 2); - - tableLoader = catalogResource.tableLoader(); - } - - private List convertToRowData(List rows) { - return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); - } - - private BoundedTestSource createBoundedSource(List rows) { - return new BoundedTestSource<>(rows.toArray(new Row[0])); - } - - private TableTestStats extractTableTestStats() throws IOException { - // Assertions: - // - Total record count in table - // - All records belong to that bucket - // - Each bucket should've been written by valid writers - // - Each file should have the expected number of records - - int totalRecordCount = 0; - Map> writersPerBucket = new HashMap<>(); // > - Map filesPerBucket = new HashMap<>(); // - Map recordsPerFile = new TreeMap<>(); // - - try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { - for (FileScanTask scanTask : fileScanTasks) { - Optional bucketIdOpt = - BucketPartitionKeySelector.extractInteger(scanTask.file().partition().toString()); - - 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 = bucketIdOpt.get(); - writersPerBucket.computeIfAbsent(bucketId, k -> new ArrayList<>()); - 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()) { - Collections.sort(writersPerBucket.get(k)); - } - - return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, recordsPerFile); - } - - private void testWriteRowData(List allRows) throws Exception { - DataStream dataStream = - env.addSource(createBoundedSource(allRows), 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, convertToRowData(allRows)); - } - - // Test 1A: N records sent to the same bucket across all partitions - // - 24 rows (3 batches w/overlap), 4 buckets - @Test - public void testSendToBucket0A() throws Exception { - // Data generation - // We make the number of requests a factor of the parallelism for an easier verification - int totalNumRows = parallelism * 3; // 24 rows - TableTestStats stats = sendAndAssertBucket0(totalNumRows); - // Number of records per file - Assert.assertEquals(16, stats.recordsPerFile.get(0).intValue()); - Assert.assertEquals(8, stats.recordsPerFile.get(4).intValue()); - } - - // Test 1B: N records sent to the same bucket across all partitions - // - 30 rows (4 batches w/o overlap, uneven first bucket), 4 buckets - @Test - public void testSendToBucket0B() throws Exception { - // Data generation - // We make the number of requests a factor of the parallelism for an easier verification - int totalNumRows = 30; - TableTestStats stats = sendAndAssertBucket0(totalNumRows); - // Number of records per file - Assert.assertEquals(16, stats.recordsPerFile.get(0).intValue()); - Assert.assertEquals(14, stats.recordsPerFile.get(4).intValue()); - } - - @NotNull - private TableTestStats sendAndAssertBucket0(int totalNumRows) throws Exception { - List rows = generateRowsForBucketId(totalNumRows, 0, numBuckets); - // Writing to the table - testWriteRowData(rows); - // Extracting stats - TableTestStats stats = extractTableTestStats(); - - // Assertions - // All the records were received - Assert.assertEquals(totalNumRows, stats.totalRecordCount); - // Only bucketId = 0 was written - Assert.assertEquals(1, stats.writersPerBucket.size()); - Assert.assertEquals(1, stats.filesPerBucket.size()); - Assert.assertTrue(stats.writersPerBucket.containsKey(0)); - Assert.assertTrue(stats.filesPerBucket.containsKey(0)); - // Valid writers - List validWriters = Arrays.asList(0, 4); - Assert.assertEquals(validWriters, stats.writersPerBucket.get(0)); - // Num writers = num created files - Assert.assertEquals(validWriters.size(), stats.filesPerBucket.get(0).intValue()); - // TODO: verify all the records actually hash to bucketId = 0 - return stats; - } - - // Test 2: N records of each bucket value, to all buckets, through the same partitions - // - 16 rows (2 writers per bucket, uneven last bucket), 4 buckets - @Test - public void testSendRecordsToAllBucketsEvenly() throws Exception { - // Data generation - // We make the number of requests a factor of the parallelism for an easier verification - int totalNumRows = parallelism * 2; - int numRowsPerBucket = totalNumRows / numBuckets; - List rows = new ArrayList<>(totalNumRows); - List templateRows = new ArrayList<>(numBuckets); - for (int i = 0; i < numBuckets; i++) { - templateRows.addAll(generateRowsForBucketId(1, i, numBuckets)); - } - for (int i = 0; i < numRowsPerBucket; i++) { - rows.addAll(templateRows); - } - - // Writing to the table - testWriteRowData(rows); - // Extracting stats - TableTestStats stats = extractTableTestStats(); - - // Assertions - // All the records were received - Assert.assertEquals(totalNumRows, stats.totalRecordCount); - // Only bucketId = 0 was written - Assert.assertEquals(numBuckets, stats.writersPerBucket.size()); - Assert.assertEquals(numBuckets, stats.filesPerBucket.size()); - for (int i = 0, j = 4; i < numBuckets; i++, j++) { - Assert.assertEquals(Arrays.asList(i, j), stats.writersPerBucket.get(i)); - Assert.assertTrue(stats.filesPerBucket.get(i) == 2); - } - } - - /** - * Utility method to generate rows whose values will "hash" to a desired bucketId - * @param numRows - * @param bucketId - * @return - */ - private List generateRowsForBucketId(int numRows, int bucketId, int numBuckets) { - List rows = new ArrayList<>(numRows); - for(int i = 0; i < numRows; i ++) { - String value = generateValueForBucketId(bucketId, numBuckets); - rows.add(Row.of(i, value)); - } - return rows; - } - - /** - * Utility method to generate a string value that will "hash" to a desired bucketId - * @param bucketId - * @param maxNumBuckets - * @return - */ - private String generateValueForBucketId(int bucketId, int maxNumBuckets) { - String value = ""; - while (true) { - String uuid = UUID.randomUUID().toString(); - // This is the same "hashing" mechanism used by Bucket.java - if ((BucketUtil.hash(uuid) & Integer.MAX_VALUE) % maxNumBuckets == bucketId) { - value = uuid; - break; - } - } - return value; - } - - /** - * DTO to hold Test Stats - */ - private static class TableTestStats { - final int totalRecordCount; - final Map> writersPerBucket; // > - final Map filesPerBucket; // - final Map recordsPerFile; // - - TableTestStats(int totalRecordCount, - Map> writersPerBucket, - Map filesPerBucket, - Map recordsPerFile) { - this.totalRecordCount = totalRecordCount; - this.writersPerBucket = writersPerBucket; - this.filesPerBucket = filesPerBucket; - this.recordsPerFile = recordsPerFile; - } - } - - // Tests (all with parallelism = 8): - // Test 1: N records sent to the same bucket across all partitions - // - 24 rows (3 batches w/overlap), 4 buckets - // - 30 rows (4 batches w/o overlap, uneven first bucket), 4 buckets - // Test 2: N records of each bucket value, to all buckets, through the same partitions - // - 16 rows (2 writers per bucket, uneven last bucket), 3 buckets - - // About test parameters: - // - Constant parallelism (#TMs and #Slots) - // - Parallelism evenly divisible by #buckets - // - Parallelism not evenly divisible by #buckets -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java new file mode 100644 index 000000000000..a9db9a435c5f --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -0,0 +1,336 @@ +/* + * 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.Optional; +import java.util.TreeMap; +import java.util.UUID; +import java.util.stream.Collectors; +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.util.MiniClusterWithClientResource; +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.HadoopCatalogResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +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.apache.iceberg.util.BucketUtil; +import org.jetbrains.annotations.NotNull; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestBucketPartitionerFlinkIcebergSink { + + private static final int NUMBER_TASK_MANAGERS = 1; + private static final int SLOTS_PER_TASK_MANAGER = 8; + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUMBER_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, DATABASE, TestFixtures.TABLE); + + private static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + private static final DataFormatConverters.RowConverter CONVERTER = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + + private Table table; + private StreamExecutionEnvironment env; + private TableLoader tableLoader; + + // Simple parallelism = 8 throughout the test suite + private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; + private final FileFormat format = FileFormat.fromString("parquet"); + private final int numBuckets = 4; + + @Before + public void before() throws IOException { + table = + catalogResource + .catalog() + .createTable( + TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism * 2); + + tableLoader = catalogResource.tableLoader(); + } + + private List convertToRowData(List rows) { + return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); + } + + private BoundedTestSource createBoundedSource(List rows) { + return new BoundedTestSource<>(rows.toArray(new Row[0])); + } + + private TableTestStats extractTableTestStats() throws IOException { + // Assertions: + // - Total record count in table + // - All records belong to that bucket + // - Each bucket should've been written by valid writers + // - Each file should have the expected number of records + + int totalRecordCount = 0; + Map> writersPerBucket = Maps.newHashMap(); // > + Map filesPerBucket = Maps.newHashMap(); // + Map recordsPerFile = new TreeMap<>(); // + + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + for (FileScanTask scanTask : fileScanTasks) { + Optional bucketIdOpt = + BucketPartitionKeySelector.extractInteger(scanTask.file().partition().toString()); + + 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 = bucketIdOpt.get(); + 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()) { + Collections.sort(writersPerBucket.get(k)); + } + + return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, recordsPerFile); + } + + private void testWriteRowData(List allRows) throws Exception { + DataStream dataStream = + env.addSource(createBoundedSource(allRows), 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, convertToRowData(allRows)); + } + + // Test 1A: N records sent to the same bucket across all partitions + // - 24 rows (3 batches w/overlap), 4 buckets + @Test + public void testSendToBucket0A() throws Exception { + // Data generation + // We make the number of requests a factor of the parallelism for an easier verification + int totalNumRows = parallelism * 3; // 24 rows + TableTestStats stats = sendAndAssertBucket0(totalNumRows); + // Number of records per file + Assert.assertEquals(16, stats.recordsPerFile.get(0).intValue()); + Assert.assertEquals(8, stats.recordsPerFile.get(4).intValue()); + } + + // Test 1B: N records sent to the same bucket across all partitions + // - 30 rows (4 batches w/o overlap, uneven first bucket), 4 buckets + @Test + public void testSendToBucket0B() throws Exception { + // Data generation + // We make the number of requests a factor of the parallelism for an easier verification + int totalNumRows = 30; + TableTestStats stats = sendAndAssertBucket0(totalNumRows); + // Number of records per file + Assert.assertEquals(16, stats.recordsPerFile.get(0).intValue()); + Assert.assertEquals(14, stats.recordsPerFile.get(4).intValue()); + } + + @NotNull + private TableTestStats sendAndAssertBucket0(int totalNumRows) throws Exception { + List rows = generateRowsForBucketId(totalNumRows, 0); + // Writing to the table + testWriteRowData(rows); + // Extracting stats + TableTestStats stats = extractTableTestStats(); + + // Assertions + // All the records were received + Assert.assertEquals(totalNumRows, stats.totalRecordCount); + // Only bucketId = 0 was written + Assert.assertEquals(1, stats.writersPerBucket.size()); + Assert.assertEquals(1, stats.filesPerBucket.size()); + Assert.assertTrue(stats.writersPerBucket.containsKey(0)); + Assert.assertTrue(stats.filesPerBucket.containsKey(0)); + // Valid writers + List validWriters = Arrays.asList(0, 4); + Assert.assertEquals(validWriters, stats.writersPerBucket.get(0)); + // Num writers = num created files + Assert.assertEquals(validWriters.size(), stats.filesPerBucket.get(0).intValue()); + // TODO: verify all the records actually hash to bucketId = 0 + return stats; + } + + // Test 2: N records of each bucket value, to all buckets, through the same partitions + // - 16 rows (2 writers per bucket, uneven last bucket), 4 buckets + @Test + public void testSendRecordsToAllBucketsEvenly() throws Exception { + // Data generation + // We make the number of requests a factor of the parallelism for an easier verification + int totalNumRows = parallelism * 2; + int numRowsPerBucket = totalNumRows / numBuckets; + List rows = Lists.newArrayListWithCapacity(totalNumRows); + List templateRows = Lists.newArrayListWithCapacity(numBuckets); + for (int i = 0; i < numBuckets; i++) { + templateRows.addAll(generateRowsForBucketId(1, i)); + } + for (int i = 0; i < numRowsPerBucket; i++) { + rows.addAll(templateRows); + } + + // Writing to the table + testWriteRowData(rows); + // Extracting stats + TableTestStats stats = extractTableTestStats(); + + // Assertions + // All the records were received + Assert.assertEquals(totalNumRows, stats.totalRecordCount); + // Only bucketId = 0 was written + Assert.assertEquals(numBuckets, stats.writersPerBucket.size()); + Assert.assertEquals(numBuckets, stats.filesPerBucket.size()); + for (int i = 0, j = 4; i < numBuckets; i++, j++) { + Assert.assertEquals(Arrays.asList(i, j), stats.writersPerBucket.get(i)); + Assert.assertTrue(stats.filesPerBucket.get(i) == 2); + } + } + + /** + * Utility method to generate rows whose values will "hash" to a desired bucketId + * + * @param numRows the number of rows/requests to generate + * @param bucketId the desired bucketId + * @return the list of rows whose data "hashes" to the desired bucketId + */ + private List generateRowsForBucketId(int numRows, int bucketId) { + List rows = Lists.newArrayListWithCapacity(numRows); + for (int i = 0; i < numRows; i++) { + String value = generateValueForBucketId(bucketId, numBuckets); + rows.add(Row.of(i, value)); + } + return rows; + } + + /** + * Utility method to generate a UUID string that will "hash" to a desired bucketId + * + * @param bucketId the desired bucketId + * @return the string data that "hashes" to the desired bucketId + */ + public static String generateValueForBucketId(int bucketId, int numBuckets) { + String value = ""; + while (true) { + String uuid = UUID.randomUUID().toString(); + if (hash(numBuckets, uuid) == bucketId) { + value = uuid; + break; + } + } + return value; + } + + /** + * Utility method that performs the same "hashing" mechanism used by Bucket.java + * + * @param numBuckets the maximum number of buckets + * @param uuid the UUID string value to hash + * @return the target BucketId + */ + public static int hash(int numBuckets, String uuid) { + return (BucketUtil.hash(uuid) & Integer.MAX_VALUE) % numBuckets; + } + + /** DTO to hold Test Stats */ + private static class TableTestStats { + final int totalRecordCount; + final Map> writersPerBucket; // > + final Map filesPerBucket; // + final Map recordsPerFile; // + + TableTestStats( + int totalRecordCount, + Map> writersPerBucket, + Map filesPerBucket, + Map recordsPerFile) { + this.totalRecordCount = totalRecordCount; + this.writersPerBucket = writersPerBucket; + this.filesPerBucket = filesPerBucket; + this.recordsPerFile = recordsPerFile; + } + } +} From 4d84e49fb3f1d1f6150dcd606e23b03c5c7a2246 Mon Sep 17 00:00:00 2001 From: schongloo Date: Wed, 5 Apr 2023 14:46:27 -0700 Subject: [PATCH 03/23] Addressing PR comments --- .../org/apache/iceberg/PartitionSpec.java | 17 ----------- .../sink/BucketPartitionKeySelector.java | 2 +- .../iceberg/flink/sink/BucketPartitioner.java | 28 +++++++++---------- .../apache/iceberg/flink/sink/FlinkSink.java | 10 ++++++- 4 files changed, 23 insertions(+), 34 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java index d48bb6b4fcbb..a31cfd76583b 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -289,23 +289,6 @@ public Set identitySourceIds() { return sourceIds; } - /** Determine if this PartitionSpec contains at least 1 bucket definition */ - public boolean hasBucketPartition() { - List 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; - break; - } - } - - return hasBucketPartition; - } - @Override public String toString() { StringBuilder sb = new StringBuilder(); diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index e12a46320613..af899a2b8395 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -33,7 +33,7 @@ * Very similar to the {@link PartitionKeySelector}, but optimized to extract and return an Integer * bucketId as the key. */ -public class BucketPartitionKeySelector implements KeySelector { +class BucketPartitionKeySelector implements KeySelector { private final Schema schema; private final PartitionKey partitionKey; diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 581079869c3e..90a59363baa6 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.functions.Partitioner; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** * This partitioner will redirect elements to writers deterministically so that each writer only @@ -29,9 +30,9 @@ * of multiple writers per bucket as evenly as possible, and will round-robin the requests across * them. */ -public class BucketPartitioner implements Partitioner { +class BucketPartitioner implements Partitioner { - private final Integer maxBuckets; + private final int maxBuckets; private final int[] currentWriterOffset; @@ -42,36 +43,33 @@ public class BucketPartitioner implements Partitioner { .filter(f -> f.transform().dedupName().contains("bucket")) .findFirst(); - if (!bucket.isPresent()) { - throw new IllegalStateException("No buckets found on the provided PartitionSpec"); - } + Preconditions.checkArgument( + bucket.isPresent(), "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 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 - + ")"); - } + Preconditions.checkArgument( + maxBucketsOpt.isPresent(), + "Could not extract the max number of buckets from the transform name (" + + transformName + + ")"); + this.maxBuckets = maxBucketsOpt.get(); this.currentWriterOffset = new int[this.maxBuckets]; } @Override public int partition(Integer bucketId, int numPartitions) { if (numPartitions > maxBuckets) { - return getSubpartitionIdx(bucketId, numPartitions); + return getPartitionIndex(bucketId, numPartitions); } else { return bucketId % numPartitions; } } - private int getSubpartitionIdx(int bucketId, int numPartitions) { + private int getPartitionIndex(int bucketId, int numPartitions) { int currentOffset = currentWriterOffset[bucketId]; // When numPartitions is not evenly divisible by maxBuckets int extraPad = bucketId < (numPartitions % maxBuckets) ? 1 : 0; diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 4a13e4b8bd39..6dfe35ff452d 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -505,7 +505,7 @@ private DataStream distributeDataStream( + "and table is unpartitioned"); return input; } else { - if (partitionSpec.hasBucketPartition()) { + if (canUseBucketPartitioner(partitionSpec)) { return input.partitionCustom( new BucketPartitioner(partitionSpec), new BucketPartitionKeySelector(partitionSpec, iSchema, flinkRowType)); @@ -555,6 +555,14 @@ private DataStream distributeDataStream( } } + /** Determine if this PartitionSpec contains at least 1 bucket definition */ + static boolean canUseBucketPartitioner(PartitionSpec partitionSpec) { + return partitionSpec.fields().stream() + .filter(f -> f.transform().dedupName().toLowerCase().contains("bucket")) + .count() + == 1; + } + static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { if (requestedSchema != null) { // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing From fe5437359eb4d5fb3dc3bced7ee289f84a63fd28 Mon Sep 17 00:00:00 2001 From: schongloo Date: Wed, 5 Apr 2023 14:58:08 -0700 Subject: [PATCH 04/23] Addressing PR comment: the BucketPartitioner should be provided a PartitionSpec with 1 and only 1 bucket. --- .../iceberg/flink/sink/BucketPartitioner.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 90a59363baa6..98e26231b372 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -18,7 +18,9 @@ */ package org.apache.iceberg.flink.sink; +import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; import org.apache.flink.api.common.functions.Partitioner; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; @@ -37,17 +39,17 @@ class BucketPartitioner implements Partitioner { private final int[] currentWriterOffset; BucketPartitioner(PartitionSpec partitionSpec) { - // The current implementation redirects to writers based on the _FIRST_ bucket found - Optional bucket = + // The current implementation only supports _ONE_ bucket + List bucketTransforms = partitionSpec.fields().stream() - .filter(f -> f.transform().dedupName().contains("bucket")) - .findFirst(); + .filter(f -> f.transform().dedupName().contains("bucket")).collect(Collectors.toList()); Preconditions.checkArgument( - bucket.isPresent(), "No buckets found on the provided PartitionSpec"); + bucketTransforms.size() == 1, + "Expected 1 Bucket transform in the provided PartitionSpec, found: " + bucketTransforms.size()); // Extracting the max number of buckets defined in the partition spec - String transformName = bucket.get().transform().dedupName(); + String transformName = bucketTransforms.get(0).transform().dedupName(); Optional maxBucketsOpt = BucketPartitionKeySelector.extractInteger(transformName); Preconditions.checkArgument( From c7b86d7732d7506e5f464d7f07206d7af146c840 Mon Sep 17 00:00:00 2001 From: schongloo Date: Thu, 6 Apr 2023 07:15:59 -0700 Subject: [PATCH 05/23] Addressing build check problems --- .../apache/iceberg/flink/sink/BucketPartitioner.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 98e26231b372..b4d73fa12b79 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -28,9 +28,9 @@ /** * 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. + * targets 1 bucket. If the number of writers is greater than the 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. */ class BucketPartitioner implements Partitioner { @@ -42,11 +42,13 @@ class BucketPartitioner implements Partitioner { // The current implementation only supports _ONE_ bucket List bucketTransforms = partitionSpec.fields().stream() - .filter(f -> f.transform().dedupName().contains("bucket")).collect(Collectors.toList()); + .filter(f -> f.transform().dedupName().contains("bucket")) + .collect(Collectors.toList()); Preconditions.checkArgument( bucketTransforms.size() == 1, - "Expected 1 Bucket transform in the provided PartitionSpec, found: " + bucketTransforms.size()); + "Expected 1 Bucket transform in the provided PartitionSpec, found: " + + bucketTransforms.size()); // Extracting the max number of buckets defined in the partition spec String transformName = bucketTransforms.get(0).transform().dedupName(); From 93d7c2ef23690c84973ab5670f6953b4ebccebec Mon Sep 17 00:00:00 2001 From: schongloo Date: Mon, 17 Apr 2023 10:37:56 -0700 Subject: [PATCH 06/23] Addressing PR comments: - Factoring out the new partitioning utilities into BucketPartitionerUtils.java class - Relying on the PartitionSpecVisitor pattern/approach to get Bucket information (no more regex based extraction) - Parameterized the test and adding support to evaluate vs. schemas with different Bucket scenarios --- .../sink/BucketPartitionKeySelector.java | 43 +++---- .../iceberg/flink/sink/BucketPartitioner.java | 29 ++--- .../flink/sink/BucketPartitionerUtils.java | 97 +++++++++++++++ .../apache/iceberg/flink/sink/FlinkSink.java | 10 +- ...TestBucketPartitionerFlinkIcebergSink.java | 112 +++++++++++++++--- 5 files changed, 222 insertions(+), 69 deletions(-) create mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index af899a2b8395..801332e1d659 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -18,16 +18,17 @@ */ package org.apache.iceberg.flink.sink; -import java.util.Optional; -import java.util.regex.Matcher; -import java.util.regex.Pattern; +import java.util.List; +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; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** * Very similar to the {@link PartitionKeySelector}, but optimized to extract and return an Integer @@ -38,14 +39,26 @@ class BucketPartitionKeySelector implements KeySelector { private final Schema schema; private final PartitionKey partitionKey; private final RowType flinkSchema; - private static final Pattern intPattern = Pattern.compile("\\d+"); + private final int bucketFieldPosition; private transient RowDataWrapper rowDataWrapper; BucketPartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { + List> bucketFields = BucketPartitionerUtils.getBucketFields(spec); + + // The current implementation only supports ONE bucket + Preconditions.checkArgument( + bucketFields.size() == 1, + BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE + bucketFields.size()); + + int bucketFieldId = bucketFields.get(0).f0; this.schema = schema; this.partitionKey = new PartitionKey(spec, schema); this.flinkSchema = flinkSchema; + this.bucketFieldPosition = + IntStream.range(0, spec.fields().size()) + .filter(i -> spec.fields().get(i).fieldId() == bucketFieldId) + .toArray()[0]; } /** @@ -60,26 +73,8 @@ private RowDataWrapper lazyRowDataWrapper() { } @Override - public Integer getKey(RowData rowData) throws Exception { + public Integer getKey(RowData rowData) { partitionKey.partition(lazyRowDataWrapper().wrap(rowData)); - Optional 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 extractInteger(String value) { - Matcher match = intPattern.matcher(value); - return match.find() ? Optional.of(Integer.parseInt(match.group())) : Optional.empty(); + return partitionKey.get(this.bucketFieldPosition, Integer.class); } } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index b4d73fa12b79..ff42fb6e2d95 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -19,10 +19,8 @@ package org.apache.iceberg.flink.sink; import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; import org.apache.flink.api.common.functions.Partitioner; -import org.apache.iceberg.PartitionField; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -39,28 +37,15 @@ class BucketPartitioner implements Partitioner { private final int[] currentWriterOffset; BucketPartitioner(PartitionSpec partitionSpec) { - // The current implementation only supports _ONE_ bucket - List bucketTransforms = - partitionSpec.fields().stream() - .filter(f -> f.transform().dedupName().contains("bucket")) - .collect(Collectors.toList()); + List> bucketFields = + BucketPartitionerUtils.getBucketFields(partitionSpec); + // The current implementation only supports ONE bucket Preconditions.checkArgument( - bucketTransforms.size() == 1, - "Expected 1 Bucket transform in the provided PartitionSpec, found: " - + bucketTransforms.size()); + bucketFields.size() == 1, + BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE + bucketFields.size()); - // Extracting the max number of buckets defined in the partition spec - String transformName = bucketTransforms.get(0).transform().dedupName(); - Optional maxBucketsOpt = BucketPartitionKeySelector.extractInteger(transformName); - - Preconditions.checkArgument( - maxBucketsOpt.isPresent(), - "Could not extract the max number of buckets from the transform name (" - + transformName - + ")"); - - this.maxBuckets = maxBucketsOpt.get(); + this.maxBuckets = bucketFields.get(0).f1; this.currentWriterOffset = new int[this.maxBuckets]; } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java new file mode 100644 index 000000000000..a8f373cd5db0 --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java @@ -0,0 +1,97 @@ +/* + * 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.transforms.PartitionSpecVisitor; + +final class BucketPartitionerUtils { + static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = + "Expected 1 Bucket transform in the provided PartitionSpec, found: "; + + private BucketPartitionerUtils() {} + + public static boolean hasOneBucketField(PartitionSpec partitionSpec) { + List> bucketFields = getBucketFields(partitionSpec); + return bucketFields != null && bucketFields.size() == 1; + } + + public static List> getBucketFields(PartitionSpec spec) { + return PartitionSpecVisitor.visit( + spec, + new PartitionSpecVisitor>() { + @Override + public Tuple2 identity( + int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 bucket( + int fieldId, String sourceName, int sourceId, int numBuckets) { + return new Tuple2<>(fieldId, numBuckets); + } + + @Override + public Tuple2 truncate( + int fieldId, String sourceName, int sourceId, int width) { + return null; + } + + @Override + public Tuple2 year(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 month(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 day(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 hour(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 alwaysNull( + int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 unknown( + int fieldId, String sourceName, int sourceId, String transform) { + return null; + } + }) + .stream() + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 6dfe35ff452d..5ce866e3887e 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -505,7 +505,7 @@ private DataStream distributeDataStream( + "and table is unpartitioned"); return input; } else { - if (canUseBucketPartitioner(partitionSpec)) { + if (BucketPartitionerUtils.hasOneBucketField(partitionSpec)) { return input.partitionCustom( new BucketPartitioner(partitionSpec), new BucketPartitionKeySelector(partitionSpec, iSchema, flinkRowType)); @@ -555,14 +555,6 @@ private DataStream distributeDataStream( } } - /** Determine if this PartitionSpec contains at least 1 bucket definition */ - static boolean canUseBucketPartitioner(PartitionSpec partitionSpec) { - return partitionSpec.fields().stream() - .filter(f -> f.transform().dedupName().toLowerCase().contains("bucket")) - .count() - == 1; - } - static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { if (requestedSchema != null) { // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index a9db9a435c5f..b139a1a8d63b 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -27,7 +27,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.TreeMap; import java.util.UUID; import java.util.stream.Collectors; @@ -59,12 +58,16 @@ import org.apache.iceberg.util.BucketUtil; import org.jetbrains.annotations.NotNull; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +@RunWith(Parameterized.class) public class TestBucketPartitionerFlinkIcebergSink { private static final int NUMBER_TASK_MANAGERS = 1; @@ -98,17 +101,24 @@ public class TestBucketPartitionerFlinkIcebergSink { private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; private final FileFormat format = FileFormat.fromString("parquet"); private final int numBuckets = 4; + private final TableSchemaType tableSchemaType; + + @Parameterized.Parameters(name = "TableSchemaType = {0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {TableSchemaType.ONE_BUCKET}, + new Object[] {TableSchemaType.IDENTITY_AND_BUCKET}, + new Object[] {TableSchemaType.TWO_BUCKETS}, + }; + } + + public TestBucketPartitionerFlinkIcebergSink(TableSchemaType tableSchemaType) { + this.tableSchemaType = tableSchemaType; + } @Before public void before() throws IOException { - table = - catalogResource - .catalog() - .createTable( - TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(), - ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + table = getTable(); env = StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) @@ -119,6 +129,39 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } + private Table getTable() { + PartitionSpec partitionSpec = null; + + switch (tableSchemaType) { + case ONE_BUCKET: + partitionSpec = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); + break; + case IDENTITY_AND_BUCKET: + partitionSpec = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .identity("id") + .bucket("data", numBuckets) + .build(); + break; + case TWO_BUCKETS: + partitionSpec = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .bucket("id", numBuckets) + .bucket("data", numBuckets) + .build(); + break; + } + + return catalogResource + .catalog() + .createTable( + TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitionSpec, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + } + private List convertToRowData(List rows) { return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); } @@ -141,9 +184,6 @@ private TableTestStats extractTableTestStats() throws IOException { try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { for (FileScanTask scanTask : fileScanTasks) { - Optional bucketIdOpt = - BucketPartitionKeySelector.extractInteger(scanTask.file().partition().toString()); - long recordCountInFile = scanTask.file().recordCount(); String[] splitFilePath = scanTask.file().path().toString().split("/"); @@ -151,7 +191,11 @@ private TableTestStats extractTableTestStats() throws IOException { int writerId = Integer.parseInt(filename.split("-")[0]); totalRecordCount += recordCountInFile; - int bucketId = bucketIdOpt.get(); + int bucketId = + scanTask + .file() + .partition() + .get(tableSchemaType == TableSchemaType.ONE_BUCKET ? 0 : 1, Integer.class); writersPerBucket.computeIfAbsent(bucketId, k -> Lists.newArrayList()); writersPerBucket.get(bucketId).add(writerId); filesPerBucket.put(bucketId, filesPerBucket.getOrDefault(bucketId, 0) + 1); @@ -189,6 +233,8 @@ private void testWriteRowData(List allRows) throws Exception { // - 24 rows (3 batches w/overlap), 4 buckets @Test public void testSendToBucket0A() throws Exception { + Assume.assumeFalse(tableSchemaType == TableSchemaType.TWO_BUCKETS); + // Data generation // We make the number of requests a factor of the parallelism for an easier verification int totalNumRows = parallelism * 3; // 24 rows @@ -202,6 +248,8 @@ public void testSendToBucket0A() throws Exception { // - 30 rows (4 batches w/o overlap, uneven first bucket), 4 buckets @Test public void testSendToBucket0B() throws Exception { + Assume.assumeTrue(tableSchemaType != TableSchemaType.TWO_BUCKETS); + // Data generation // We make the number of requests a factor of the parallelism for an easier verification int totalNumRows = 30; @@ -240,6 +288,8 @@ private TableTestStats sendAndAssertBucket0(int totalNumRows) throws Exception { // - 16 rows (2 writers per bucket, uneven last bucket), 4 buckets @Test public void testSendRecordsToAllBucketsEvenly() throws Exception { + Assume.assumeTrue(tableSchemaType != TableSchemaType.TWO_BUCKETS); + // Data generation // We make the number of requests a factor of the parallelism for an easier verification int totalNumRows = parallelism * 2; @@ -270,6 +320,34 @@ public void testSendRecordsToAllBucketsEvenly() throws Exception { } } + @Test + public void testBucketPartitionerFail() { + Assume.assumeTrue(tableSchemaType == TableSchemaType.TWO_BUCKETS); + + Exception exception = + Assert.assertThrows(RuntimeException.class, () -> new BucketPartitioner(table.spec())); + + Assert.assertTrue( + exception + .getMessage() + .contains(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE)); + } + + @Test + public void testBucketPartitionKeySelectorFail() { + Assume.assumeTrue(tableSchemaType == TableSchemaType.TWO_BUCKETS); + + Exception exception = + Assert.assertThrows( + RuntimeException.class, + () -> new BucketPartitionKeySelector(table.spec(), table.schema(), null)); + + Assert.assertTrue( + exception + .getMessage() + .contains(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE)); + } + /** * Utility method to generate rows whose values will "hash" to a desired bucketId * @@ -281,7 +359,7 @@ private List generateRowsForBucketId(int numRows, int bucketId) { List rows = Lists.newArrayListWithCapacity(numRows); for (int i = 0; i < numRows; i++) { String value = generateValueForBucketId(bucketId, numBuckets); - rows.add(Row.of(i, value)); + rows.add(Row.of(1, value)); } return rows; } @@ -333,4 +411,10 @@ private static class TableTestStats { this.recordsPerFile = recordsPerFile; } } + + private enum TableSchemaType { + ONE_BUCKET, + IDENTITY_AND_BUCKET, + TWO_BUCKETS; + } } From 75fad88c4f51be020733458928710e64fab4cb64 Mon Sep 17 00:00:00 2001 From: schongloo Date: Sun, 23 Apr 2023 11:34:08 -0700 Subject: [PATCH 07/23] Addressing PR comments, pending: - Logic simplification of the TestBucketPartitionerFlinkIcebergSink - Migration to Junit5 --- .../sink/BucketPartitionKeySelector.java | 24 ++-- .../iceberg/flink/sink/BucketPartitioner.java | 69 ++++++----- .../flink/sink/BucketPartitionerUtils.java | 107 ++++++++++-------- ...TestBucketPartitionerFlinkIcebergSink.java | 27 ++--- 4 files changed, 119 insertions(+), 108 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index 801332e1d659..3e21ad15c410 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import java.util.List; import java.util.stream.IntStream; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; @@ -28,11 +27,10 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** - * Very similar to the {@link PartitionKeySelector}, but optimized to extract and return an Integer - * bucketId as the key. + * Similar to the {@link PartitionKeySelector}, but extracts and returns an Integer bucketId as the + * key. To be used with the {@link BucketPartitioner}. */ class BucketPartitionKeySelector implements KeySelector { @@ -43,21 +41,17 @@ class BucketPartitionKeySelector implements KeySelector { private transient RowDataWrapper rowDataWrapper; - BucketPartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { - List> bucketFields = BucketPartitionerUtils.getBucketFields(spec); + BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, RowType flinkSchema) { + Tuple2 bucketFieldInfo = + BucketPartitionerUtils.getBucketFieldInfo(partitionSpec); - // The current implementation only supports ONE bucket - Preconditions.checkArgument( - bucketFields.size() == 1, - BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE + bucketFields.size()); - - int bucketFieldId = bucketFields.get(0).f0; + int bucketFieldId = bucketFieldInfo.f0; this.schema = schema; - this.partitionKey = new PartitionKey(spec, schema); + this.partitionKey = new PartitionKey(partitionSpec, schema); this.flinkSchema = flinkSchema; this.bucketFieldPosition = - IntStream.range(0, spec.fields().size()) - .filter(i -> spec.fields().get(i).fieldId() == bucketFieldId) + IntStream.range(0, partitionSpec.fields().size()) + .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) .toArray()[0]; } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index ff42fb6e2d95..84603e10f4fb 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -18,55 +18,70 @@ */ package org.apache.iceberg.flink.sink; -import java.util.List; 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 elements to writers deterministically so that each writer only - * targets 1 bucket. If the number of writers is greater than the 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. - */ +/** This partitioner will redirect elements to writers deterministically. */ class BucketPartitioner implements Partitioner { private final int maxBuckets; - private final int[] currentWriterOffset; + private final int[] currentWriterForBucket; BucketPartitioner(PartitionSpec partitionSpec) { - List> bucketFields = - BucketPartitionerUtils.getBucketFields(partitionSpec); - - // The current implementation only supports ONE bucket - Preconditions.checkArgument( - bucketFields.size() == 1, - BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE + bucketFields.size()); + Tuple2 bucketFieldInfo = + BucketPartitionerUtils.getBucketFieldInfo(partitionSpec); - this.maxBuckets = bucketFields.get(0).f1; - this.currentWriterOffset = new int[this.maxBuckets]; + this.maxBuckets = bucketFieldInfo.f1; + this.currentWriterForBucket = new int[this.maxBuckets]; } + /** + * If the number of writers <= the number of buckets an evenly distributed number of buckets will + * be assigned to each writer. Conversely, if the number of writers > the number of buckets the + * logic is handled by the {@link #getPartitionWritersGreaterThanBuckets + * getPartitionWritersGreaterThanBuckets} method. + * + * @param bucketId the bucketId for each request + * @param numPartitions the total number of partitions + * @return the partition index (writer) to use for each request + */ @Override public int partition(Integer bucketId, int numPartitions) { - if (numPartitions > maxBuckets) { - return getPartitionIndex(bucketId, numPartitions); - } else { + if (numPartitions <= maxBuckets) { return bucketId % numPartitions; + } else { + return getPartitionWritersGreaterThanBuckets(bucketId, numPartitions); } } - private int getPartitionIndex(int bucketId, int numPartitions) { - int currentOffset = currentWriterOffset[bucketId]; + /*- + * If the number of writers > the 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, in + * this case each writer will target no more than one bucket at all times. + * Example: numPartitions (writers) = 3, maxBuckets = 2 + * - Writers 0 and 2 -> Bucket 0 + * - Writer 1 -> Bucket 1 + * - currentWriterForBucket always has the next writer index to use for a bucket + * - maxNumWritersPerBucket determines when to reset the currentWriterForBucket to 0 + * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer, + * this is determined by extraWriter. In this example Bucket 0 has an "extra writer" to consider when + * resetting to 0. + * + * @param bucketId the bucketId for each request + * @param numPartitions the total number of partitions + * @return the partition index (writer) to use for each request + */ + private int getPartitionWritersGreaterThanBuckets(int bucketId, int numPartitions) { + int currentOffset = currentWriterForBucket[bucketId]; // When numPartitions is not evenly divisible by maxBuckets - int extraPad = bucketId < (numPartitions % maxBuckets) ? 1 : 0; - int maxWriterNumOffsets = (numPartitions / maxBuckets) + extraPad; + int extraWriter = bucketId < (numPartitions % maxBuckets) ? 1 : 0; + int maxNumWritersPerBucket = (numPartitions / maxBuckets) + extraWriter; // Reset the offset when necessary - int nextOffset = currentOffset == maxWriterNumOffsets - 1 ? 0 : currentOffset + 1; - currentWriterOffset[bucketId] = nextOffset; + int nextOffset = currentOffset == maxNumWritersPerBucket - 1 ? 0 : currentOffset + 1; + currentWriterForBucket[bucketId] = nextOffset; return bucketId + (maxBuckets * currentOffset); } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java index a8f373cd5db0..c2d3787cf25d 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java @@ -23,6 +23,7 @@ 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 { @@ -31,67 +32,73 @@ final class BucketPartitionerUtils { private BucketPartitionerUtils() {} - public static boolean hasOneBucketField(PartitionSpec partitionSpec) { + static boolean hasOneBucketField(PartitionSpec partitionSpec) { List> bucketFields = getBucketFields(partitionSpec); return bucketFields != null && bucketFields.size() == 1; } - public static List> getBucketFields(PartitionSpec spec) { - return PartitionSpecVisitor.visit( - spec, - new PartitionSpecVisitor>() { - @Override - public Tuple2 identity( - int fieldId, String sourceName, int sourceId) { - return null; - } + static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { + List> bucketFields = getBucketFields(partitionSpec); + Preconditions.checkArgument( + bucketFields.size() == 1, + BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE + bucketFields.size()); + return bucketFields.get(0); + } - @Override - public Tuple2 bucket( - int fieldId, String sourceName, int sourceId, int numBuckets) { - return new Tuple2<>(fieldId, numBuckets); - } + private static List> getBucketFields(PartitionSpec spec) { + return PartitionSpecVisitor.visit(spec, new BucketPartitionSpecVisitor()).stream() + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } - @Override - public Tuple2 truncate( - int fieldId, String sourceName, int sourceId, int width) { - return null; - } + private static class BucketPartitionSpecVisitor + implements PartitionSpecVisitor> { + @Override + public Tuple2 identity(int fieldId, String sourceName, int sourceId) { + return null; + } - @Override - public Tuple2 year(int fieldId, String sourceName, int sourceId) { - return null; - } + @Override + public Tuple2 bucket( + int fieldId, String sourceName, int sourceId, int numBuckets) { + return new Tuple2<>(fieldId, numBuckets); + } - @Override - public Tuple2 month(int fieldId, String sourceName, int sourceId) { - return null; - } + @Override + public Tuple2 truncate( + int fieldId, String sourceName, int sourceId, int width) { + return null; + } - @Override - public Tuple2 day(int fieldId, String sourceName, int sourceId) { - return null; - } + @Override + public Tuple2 year(int fieldId, String sourceName, int sourceId) { + return null; + } - @Override - public Tuple2 hour(int fieldId, String sourceName, int sourceId) { - return null; - } + @Override + public Tuple2 month(int fieldId, String sourceName, int sourceId) { + return null; + } - @Override - public Tuple2 alwaysNull( - int fieldId, String sourceName, int sourceId) { - return null; - } + @Override + public Tuple2 day(int fieldId, String sourceName, int sourceId) { + return null; + } - @Override - public Tuple2 unknown( - int fieldId, String sourceName, int sourceId, String transform) { - return null; - } - }) - .stream() - .filter(Objects::nonNull) - .collect(Collectors.toList()); + @Override + public Tuple2 hour(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 alwaysNull(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 unknown( + int fieldId, String sourceName, int sourceId, String transform) { + return null; + } } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index b139a1a8d63b..4c7b970688dc 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -70,6 +70,12 @@ @RunWith(Parameterized.class) public class TestBucketPartitionerFlinkIcebergSink { + private enum TableSchemaType { + ONE_BUCKET, + IDENTITY_AND_BUCKET, + TWO_BUCKETS; + } + private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; @@ -93,16 +99,16 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final DataFormatConverters.RowConverter CONVERTER = new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - private Table table; - private StreamExecutionEnvironment env; - private TableLoader tableLoader; - // Simple parallelism = 8 throughout the test suite private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; - private final FileFormat format = FileFormat.fromString("parquet"); + private final FileFormat format = FileFormat.PARQUET; private final int numBuckets = 4; private final TableSchemaType tableSchemaType; + private Table table; + private StreamExecutionEnvironment env; + private TableLoader tableLoader; + @Parameterized.Parameters(name = "TableSchemaType = {0}") public static Object[][] parameters() { return new Object[][] { @@ -119,13 +125,11 @@ public TestBucketPartitionerFlinkIcebergSink(TableSchemaType tableSchemaType) { @Before public void before() throws IOException { table = getTable(); - env = StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism * 2); - tableLoader = catalogResource.tableLoader(); } @@ -176,7 +180,6 @@ private TableTestStats extractTableTestStats() throws IOException { // - All records belong to that bucket // - Each bucket should've been written by valid writers // - Each file should have the expected number of records - int totalRecordCount = 0; Map> writersPerBucket = Maps.newHashMap(); // > Map filesPerBucket = Maps.newHashMap(); // @@ -267,7 +270,6 @@ private TableTestStats sendAndAssertBucket0(int totalNumRows) throws Exception { // Extracting stats TableTestStats stats = extractTableTestStats(); - // Assertions // All the records were received Assert.assertEquals(totalNumRows, stats.totalRecordCount); // Only bucketId = 0 was written @@ -308,7 +310,6 @@ public void testSendRecordsToAllBucketsEvenly() throws Exception { // Extracting stats TableTestStats stats = extractTableTestStats(); - // Assertions // All the records were received Assert.assertEquals(totalNumRows, stats.totalRecordCount); // Only bucketId = 0 was written @@ -411,10 +412,4 @@ private static class TableTestStats { this.recordsPerFile = recordsPerFile; } } - - private enum TableSchemaType { - ONE_BUCKET, - IDENTITY_AND_BUCKET, - TWO_BUCKETS; - } } From 264fc7d892f6af5e10d83cd846225633102d7a7c Mon Sep 17 00:00:00 2001 From: schongloo Date: Sun, 23 Apr 2023 11:51:24 -0700 Subject: [PATCH 08/23] Addressing PR comments, pending: - Clarifying the BucketPartitioner logic via Javadoc and better variable names. --- .../iceberg/flink/sink/BucketPartitioner.java | 39 ++++++++++--------- 1 file changed, 21 insertions(+), 18 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 84603e10f4fb..87e197762140 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -25,16 +25,16 @@ /** This partitioner will redirect elements to writers deterministically. */ class BucketPartitioner implements Partitioner { - private final int maxBuckets; + private final int maxNumBuckets; - private final int[] currentWriterForBucket; + private final int[] currentBucketWriterOffset; BucketPartitioner(PartitionSpec partitionSpec) { Tuple2 bucketFieldInfo = BucketPartitionerUtils.getBucketFieldInfo(partitionSpec); - this.maxBuckets = bucketFieldInfo.f1; - this.currentWriterForBucket = new int[this.maxBuckets]; + this.maxNumBuckets = bucketFieldInfo.f1; + this.currentBucketWriterOffset = new int[this.maxNumBuckets]; } /** @@ -49,7 +49,7 @@ class BucketPartitioner implements Partitioner { */ @Override public int partition(Integer bucketId, int numPartitions) { - if (numPartitions <= maxBuckets) { + if (numPartitions <= maxNumBuckets) { return bucketId % numPartitions; } else { return getPartitionWritersGreaterThanBuckets(bucketId, numPartitions); @@ -59,12 +59,14 @@ public int partition(Integer bucketId, int numPartitions) { /*- * If the number of writers > the 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, in - * this case each writer will target no more than one bucket at all times. - * Example: numPartitions (writers) = 3, maxBuckets = 2 - * - Writers 0 and 2 -> Bucket 0 - * - Writer 1 -> Bucket 1 - * - currentWriterForBucket always has the next writer index to use for a bucket - * - maxNumWritersPerBucket determines when to reset the currentWriterForBucket to 0 + * this case each writer will target no more than one bucket at all times. Example: + * Configuration: numPartitions (writers) = 3, maxBuckets = 2 + * Expected behavior: + * - Records for Bucket 0 will be "round robin" between Writers 0 and 2 + * - Records for Bucket 1 will always use Writer 1 + * Notes: + * - currentBucketWriterOffset always has the OFFSET of the next writer to use for any bucket + * - maxNumWritersPerBucket determines when to reset the currentWriterForBucket to 0 for bucketId * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer, * this is determined by extraWriter. In this example Bucket 0 has an "extra writer" to consider when * resetting to 0. @@ -74,15 +76,16 @@ public int partition(Integer bucketId, int numPartitions) { * @return the partition index (writer) to use for each request */ private int getPartitionWritersGreaterThanBuckets(int bucketId, int numPartitions) { - int currentOffset = currentWriterForBucket[bucketId]; - // When numPartitions is not evenly divisible by maxBuckets - int extraWriter = bucketId < (numPartitions % maxBuckets) ? 1 : 0; - int maxNumWritersPerBucket = (numPartitions / maxBuckets) + extraWriter; + int currentOffset = currentBucketWriterOffset[bucketId]; + // Determine if this bucket requires an "extra writer" + int extraWriter = bucketId < (numPartitions % maxNumBuckets) ? 1 : 0; + // The max number of writers this bucket can have + int maxNumWritersPerBucket = (numPartitions / maxNumBuckets) + extraWriter; - // Reset the offset when necessary + // Increment the writer offset or reset if it's reached the max for this bucket int nextOffset = currentOffset == maxNumWritersPerBucket - 1 ? 0 : currentOffset + 1; - currentWriterForBucket[bucketId] = nextOffset; + currentBucketWriterOffset[bucketId] = nextOffset; - return bucketId + (maxBuckets * currentOffset); + return bucketId + (maxNumBuckets * currentOffset); } } From 19fa9d6505c7b388c0418db4de9b7106f384072e Mon Sep 17 00:00:00 2001 From: schongloo Date: Sun, 23 Apr 2023 12:00:27 -0700 Subject: [PATCH 09/23] Addressing PR comments, pending: - Clarifying the BucketPartitioner logic via Javadoc and better variable names. --- .../org/apache/iceberg/flink/sink/BucketPartitioner.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 87e197762140..5906cf1daeb7 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -22,7 +22,12 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.iceberg.PartitionSpec; -/** This partitioner will redirect elements to writers deterministically. */ +/** + * 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 { private final int maxNumBuckets; From 2afa7ad8a18216808e424f873890d0b673f6729a Mon Sep 17 00:00:00 2001 From: schongloo Date: Mon, 24 Apr 2023 11:22:47 -0700 Subject: [PATCH 10/23] Addressing PR comments, pending: - Cleaning up the test and general comments/javadoc --- .../sink/BucketPartitionKeySelector.java | 4 +- .../iceberg/flink/sink/BucketPartitioner.java | 9 +- ...TestBucketPartitionerFlinkIcebergSink.java | 111 +++++------------- 3 files changed, 33 insertions(+), 91 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index 3e21ad15c410..c118fbd5f42f 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -29,8 +29,8 @@ import org.apache.iceberg.flink.RowDataWrapper; /** - * Similar to the {@link PartitionKeySelector}, but extracts and returns an Integer bucketId as the - * key. To be used with the {@link BucketPartitioner}. + * A {@link KeySelector} that extracts and returns an Integer bucketId as the key. + * To be used with the {@link BucketPartitioner}. */ class BucketPartitionKeySelector implements KeySelector { diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 5906cf1daeb7..4685ff762e9b 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -32,6 +32,7 @@ class BucketPartitioner implements Partitioner { private final int maxNumBuckets; + // This always has the OFFSET of the next writer to use for any bucket private final int[] currentBucketWriterOffset; BucketPartitioner(PartitionSpec partitionSpec) { @@ -70,11 +71,9 @@ public int partition(Integer bucketId, int numPartitions) { * - Records for Bucket 0 will be "round robin" between Writers 0 and 2 * - Records for Bucket 1 will always use Writer 1 * Notes: - * - currentBucketWriterOffset always has the OFFSET of the next writer to use for any bucket - * - maxNumWritersPerBucket determines when to reset the currentWriterForBucket to 0 for bucketId - * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer, - * this is determined by extraWriter. In this example Bucket 0 has an "extra writer" to consider when - * resetting to 0. + * - maxNumWritersPerBucket determines when to reset the currentBucketWriterOffset to 0 for this bucketId + * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter). + * In this example Bucket 0 has an "extra writer" to consider before resetting its offset to 0. * * @param bucketId the bucketId for each request * @param numPartitions the total number of partitions diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index 4c7b970688dc..a38c8089cd4e 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -56,7 +56,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.BucketUtil; -import org.jetbrains.annotations.NotNull; import org.junit.Assert; import org.junit.Assume; import org.junit.Before; @@ -175,11 +174,6 @@ private BoundedTestSource createBoundedSource(List rows) { } private TableTestStats extractTableTestStats() throws IOException { - // Assertions: - // - Total record count in table - // - All records belong to that bucket - // - Each bucket should've been written by valid writers - // - Each file should have the expected number of records int totalRecordCount = 0; Map> writersPerBucket = Maps.newHashMap(); // > Map filesPerBucket = Maps.newHashMap(); // @@ -232,92 +226,41 @@ private void testWriteRowData(List allRows) throws Exception { SimpleDataUtil.assertTableRows(table, convertToRowData(allRows)); } - // Test 1A: N records sent to the same bucket across all partitions - // - 24 rows (3 batches w/overlap), 4 buckets - @Test - public void testSendToBucket0A() throws Exception { - Assume.assumeFalse(tableSchemaType == TableSchemaType.TWO_BUCKETS); - - // Data generation - // We make the number of requests a factor of the parallelism for an easier verification - int totalNumRows = parallelism * 3; // 24 rows - TableTestStats stats = sendAndAssertBucket0(totalNumRows); - // Number of records per file - Assert.assertEquals(16, stats.recordsPerFile.get(0).intValue()); - Assert.assertEquals(8, stats.recordsPerFile.get(4).intValue()); - } - - // Test 1B: N records sent to the same bucket across all partitions - // - 30 rows (4 batches w/o overlap, uneven first bucket), 4 buckets - @Test - public void testSendToBucket0B() throws Exception { - Assume.assumeTrue(tableSchemaType != TableSchemaType.TWO_BUCKETS); - - // Data generation - // We make the number of requests a factor of the parallelism for an easier verification - int totalNumRows = 30; - TableTestStats stats = sendAndAssertBucket0(totalNumRows); - // Number of records per file - Assert.assertEquals(16, stats.recordsPerFile.get(0).intValue()); - Assert.assertEquals(14, stats.recordsPerFile.get(4).intValue()); - } - - @NotNull - private TableTestStats sendAndAssertBucket0(int totalNumRows) throws Exception { - List rows = generateRowsForBucketId(totalNumRows, 0); - // Writing to the table - testWriteRowData(rows); - // Extracting stats - TableTestStats stats = extractTableTestStats(); - - // All the records were received - Assert.assertEquals(totalNumRows, stats.totalRecordCount); - // Only bucketId = 0 was written - Assert.assertEquals(1, stats.writersPerBucket.size()); - Assert.assertEquals(1, stats.filesPerBucket.size()); - Assert.assertTrue(stats.writersPerBucket.containsKey(0)); - Assert.assertTrue(stats.filesPerBucket.containsKey(0)); - // Valid writers - List validWriters = Arrays.asList(0, 4); - Assert.assertEquals(validWriters, stats.writersPerBucket.get(0)); - // Num writers = num created files - Assert.assertEquals(validWriters.size(), stats.filesPerBucket.get(0).intValue()); - // TODO: verify all the records actually hash to bucketId = 0 - return stats; - } - - // Test 2: N records of each bucket value, to all buckets, through the same partitions - // - 16 rows (2 writers per bucket, uneven last bucket), 4 buckets @Test public void testSendRecordsToAllBucketsEvenly() throws Exception { Assume.assumeTrue(tableSchemaType != TableSchemaType.TWO_BUCKETS); - // Data generation - // We make the number of requests a factor of the parallelism for an easier verification + // Generating 16 records to be sent (8 writers -> 4 buckets), will be uniformly round-robin'd + // to all writers int totalNumRows = parallelism * 2; int numRowsPerBucket = totalNumRows / numBuckets; List rows = Lists.newArrayListWithCapacity(totalNumRows); - List templateRows = Lists.newArrayListWithCapacity(numBuckets); + List rowsPerBucket = Lists.newArrayListWithCapacity(numBuckets); for (int i = 0; i < numBuckets; i++) { - templateRows.addAll(generateRowsForBucketId(1, i)); + rowsPerBucket.addAll(generateRowsForBucketId(1, i)); } for (int i = 0; i < numRowsPerBucket; i++) { - rows.addAll(templateRows); + rows.addAll(rowsPerBucket); } - // Writing to the table testWriteRowData(rows); - // Extracting stats TableTestStats stats = extractTableTestStats(); - // All the records were received - Assert.assertEquals(totalNumRows, stats.totalRecordCount); - // Only bucketId = 0 was written + Assert.assertEquals(totalNumRows, stats.totalRowCount); + // All 4 buckets should've been written to Assert.assertEquals(numBuckets, stats.writersPerBucket.size()); - Assert.assertEquals(numBuckets, stats.filesPerBucket.size()); - for (int i = 0, j = 4; i < numBuckets; i++, j++) { + Assert.assertEquals(numBuckets, stats.numFilesPerBucket.size()); + // 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++) { Assert.assertEquals(Arrays.asList(i, j), stats.writersPerBucket.get(i)); - Assert.assertTrue(stats.filesPerBucket.get(i) == 2); + // 2 files per bucket (one file is created by each writer) + Assert.assertEquals(2, (int) stats.numFilesPerBucket.get(i)); + // 2 rows per file (total of 16 rows across 8 files) + Assert.assertEquals(2, (long) stats.rowsPerFile.get(i)); } } @@ -396,20 +339,20 @@ public static int hash(int numBuckets, String uuid) { /** DTO to hold Test Stats */ private static class TableTestStats { - final int totalRecordCount; - final Map> writersPerBucket; // > - final Map filesPerBucket; // - final Map recordsPerFile; // + final int totalRowCount; + final Map> writersPerBucket; + final Map numFilesPerBucket; + final Map rowsPerFile; TableTestStats( int totalRecordCount, Map> writersPerBucket, - Map filesPerBucket, - Map recordsPerFile) { - this.totalRecordCount = totalRecordCount; + Map numFilesPerBucket, + Map rowsPerFile) { + this.totalRowCount = totalRecordCount; this.writersPerBucket = writersPerBucket; - this.filesPerBucket = filesPerBucket; - this.recordsPerFile = recordsPerFile; + this.numFilesPerBucket = numFilesPerBucket; + this.rowsPerFile = rowsPerFile; } } } From 767a2da6ccc31ce223d6656849f72fe8f6014165 Mon Sep 17 00:00:00 2001 From: schongloo Date: Mon, 24 Apr 2023 16:34:01 -0700 Subject: [PATCH 11/23] - Creating the HadoopCatalogExtension.java for Junit5 - Migrating the sink unit test to Junit5 --- .../sink/BucketPartitionKeySelector.java | 4 +- .../iceberg/flink/HadoopCatalogExtension.java | 90 +++++++++++ ...TestBucketPartitionerFlinkIcebergSink.java | 147 ++++++++---------- 3 files changed, 160 insertions(+), 81 deletions(-) create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index c118fbd5f42f..0ddfc2d878e2 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -29,8 +29,8 @@ import org.apache.iceberg.flink.RowDataWrapper; /** - * A {@link KeySelector} that extracts and returns an Integer bucketId as the key. - * To be used with the {@link BucketPartitioner}. + * A {@link KeySelector} that extracts and returns an Integer bucketId as the key. To be used with + * the {@link BucketPartitioner}. */ class BucketPartitionKeySelector implements KeySelector { diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java new file mode 100644 index 000000000000..04fa7fbae6aa --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -0,0 +1,90 @@ +/* + * 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.Path; +import java.util.UUID; +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.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class HadoopCatalogExtension implements BeforeEachCallback, AfterEachCallback { + protected final Path temporaryFolder; + protected final String database; + protected final String tableName; + + protected Catalog catalog; + protected CatalogLoader catalogLoader; + protected String warehouse; + protected TableLoader tableLoader; + + public HadoopCatalogExtension(Path temporaryFolder, String database, String tableName) { + this.temporaryFolder = temporaryFolder; + this.database = database; + this.tableName = tableName; + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + Assertions.assertThat(temporaryFolder).exists().isDirectory(); + this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); + this.catalogLoader = + CatalogLoader.hadoop( + "hadoop", + new Configuration(), + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); + this.catalog = catalogLoader.loadCatalog(); + this.tableLoader = + TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); + } + + @Override + public void afterEach(ExtensionContext context) throws Exception { + try { + catalog.dropTable(TableIdentifier.of(database, tableName)); + ((HadoopCatalog) catalog).close(); + tableLoader.close(); + } catch (Exception e) { + throw new RuntimeException("Failed to close catalog resource"); + } + } + + public TableLoader tableLoader() { + return tableLoader; + } + + public Catalog catalog() { + return catalog; + } + + public CatalogLoader catalogLoader() { + return catalogLoader; + } + + public String warehouse() { + return warehouse; + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index a38c8089cd4e..bd55a0684b40 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -23,6 +23,7 @@ import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import java.io.IOException; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -37,7 +38,7 @@ 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.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; @@ -45,7 +46,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -56,17 +57,12 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.BucketUtil; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + public class TestBucketPartitionerFlinkIcebergSink { private enum TableSchemaType { @@ -78,61 +74,51 @@ private enum TableSchemaType { private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - new MiniClusterWithClientResource( + @RegisterExtension + public 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()); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + /** + * FOR REVIEW: I'm emulating the usage of the TemporaryFolder in the other Junit4 tests. However + * this can be completely abstracted inside the HadoopCatalogExtension via + * Files.createTempDirectory(...) in its BeforeAll. Should I pursue the latter approach? + */ + @TempDir static Path temporaryFolder; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, DATABASE, TestFixtures.TABLE); + @RegisterExtension + HadoopCatalogExtension catalogExtension = + new HadoopCatalogExtension(temporaryFolder, DATABASE, TestFixtures.TABLE); private static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); private static final DataFormatConverters.RowConverter CONVERTER = new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - // Simple parallelism = 8 throughout the test suite + // Parallelism = 8 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 final TableSchemaType tableSchemaType; private Table table; private StreamExecutionEnvironment env; private TableLoader tableLoader; - @Parameterized.Parameters(name = "TableSchemaType = {0}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {TableSchemaType.ONE_BUCKET}, - new Object[] {TableSchemaType.IDENTITY_AND_BUCKET}, - new Object[] {TableSchemaType.TWO_BUCKETS}, - }; - } - - public TestBucketPartitionerFlinkIcebergSink(TableSchemaType tableSchemaType) { - this.tableSchemaType = tableSchemaType; - } - - @Before - public void before() throws IOException { - table = getTable(); + private void setupEnvironment(TableSchemaType tableSchemaType) { + table = getTable(tableSchemaType); env = StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism * 2); - tableLoader = catalogResource.tableLoader(); + tableLoader = catalogExtension.tableLoader(); } - private Table getTable() { + private Table getTable(TableSchemaType tableSchemaType) { PartitionSpec partitionSpec = null; switch (tableSchemaType) { @@ -156,7 +142,7 @@ private Table getTable() { break; } - return catalogResource + return catalogExtension .catalog() .createTable( TABLE_IDENTIFIER, @@ -173,7 +159,7 @@ private BoundedTestSource createBoundedSource(List rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } - private TableTestStats extractTableTestStats() throws IOException { + private TableTestStats extractTableTestStats(TableSchemaType tableSchemaType) throws IOException { int totalRecordCount = 0; Map> writersPerBucket = Maps.newHashMap(); // > Map filesPerBucket = Maps.newHashMap(); // @@ -226,9 +212,12 @@ private void testWriteRowData(List allRows) throws Exception { SimpleDataUtil.assertTableRows(table, convertToRowData(allRows)); } - @Test - public void testSendRecordsToAllBucketsEvenly() throws Exception { - Assume.assumeTrue(tableSchemaType != TableSchemaType.TWO_BUCKETS); + @ParameterizedTest + @EnumSource( + value = TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) throws Exception { + setupEnvironment(tableSchemaType); // Generating 16 records to be sent (8 writers -> 4 buckets), will be uniformly round-robin'd // to all writers @@ -244,53 +233,53 @@ public void testSendRecordsToAllBucketsEvenly() throws Exception { } testWriteRowData(rows); - TableTestStats stats = extractTableTestStats(); + TableTestStats stats = extractTableTestStats(tableSchemaType); - Assert.assertEquals(totalNumRows, stats.totalRowCount); + Assertions.assertThat(stats.totalRowCount).isEqualTo(totalNumRows); // All 4 buckets should've been written to - Assert.assertEquals(numBuckets, stats.writersPerBucket.size()); - Assert.assertEquals(numBuckets, stats.numFilesPerBucket.size()); + 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++) { - Assert.assertEquals(Arrays.asList(i, j), stats.writersPerBucket.get(i)); + Assertions.assertThat(stats.writersPerBucket.get(i)).isEqualTo(Arrays.asList(i, j)); // 2 files per bucket (one file is created by each writer) - Assert.assertEquals(2, (int) stats.numFilesPerBucket.get(i)); + Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(2); // 2 rows per file (total of 16 rows across 8 files) - Assert.assertEquals(2, (long) stats.rowsPerFile.get(i)); + Assertions.assertThat((long) stats.rowsPerFile.get(i)).isEqualTo(2); } } - @Test - public void testBucketPartitionerFail() { - Assume.assumeTrue(tableSchemaType == TableSchemaType.TWO_BUCKETS); - - Exception exception = - Assert.assertThrows(RuntimeException.class, () -> new BucketPartitioner(table.spec())); - - Assert.assertTrue( - exception - .getMessage() - .contains(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE)); - } - - @Test - public void testBucketPartitionKeySelectorFail() { - Assume.assumeTrue(tableSchemaType == TableSchemaType.TWO_BUCKETS); - - Exception exception = - Assert.assertThrows( - RuntimeException.class, - () -> new BucketPartitionKeySelector(table.spec(), table.schema(), null)); - - Assert.assertTrue( - exception - .getMessage() - .contains(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE)); - } + // @Test + // public void testBucketPartitionerFail() { + // Assume.assumeTrue(tableSchemaType == TableSchemaType.TWO_BUCKETS); + // + // Exception exception = + // Assert.assertThrows(RuntimeException.class, () -> new BucketPartitioner(table.spec())); + // + // Assert.assertTrue( + // exception + // .getMessage() + // .contains(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE)); + // } + // + // @Test + // public void testBucketPartitionKeySelectorFail() { + // Assume.assumeTrue(tableSchemaType == TableSchemaType.TWO_BUCKETS); + // + // Exception exception = + // Assert.assertThrows( + // RuntimeException.class, + // () -> new BucketPartitionKeySelector(table.spec(), table.schema(), null)); + // + // Assert.assertTrue( + // exception + // .getMessage() + // .contains(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE)); + // } /** * Utility method to generate rows whose values will "hash" to a desired bucketId From 4c40ee67911c39684a29474d9ae038710ccda8fb Mon Sep 17 00:00:00 2001 From: schongloo Date: Mon, 24 Apr 2023 18:22:46 -0700 Subject: [PATCH 12/23] Removing the need for an additiona/external @TempDir for the HadoopCatalogExtension.java, now handled internally. --- .../iceberg/flink/HadoopCatalogExtension.java | 23 +++++-- ...TestBucketPartitionerFlinkIcebergSink.java | 63 ++++++++----------- 2 files changed, 44 insertions(+), 42 deletions(-) diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java index 04fa7fbae6aa..f7d23c8bda37 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -18,8 +18,9 @@ */ package org.apache.iceberg.flink; -import java.nio.file.Path; +import java.io.File;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; @@ -27,26 +28,38 @@ 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 BeforeEachCallback, AfterEachCallback { - protected final Path temporaryFolder; +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(Path temporaryFolder, String database, String tableName) { - this.temporaryFolder = temporaryFolder; + 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()); + } + @Override public void beforeEach(ExtensionContext context) throws Exception { Assertions.assertThat(temporaryFolder).exists().isDirectory(); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index bd55a0684b40..15f6e9e207e0 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -75,7 +75,7 @@ private enum TableSchemaType { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + static final MiniClusterExtension MINI_CLUSTER_RESOURCE = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) @@ -83,16 +83,9 @@ private enum TableSchemaType { .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) .build()); - /** - * FOR REVIEW: I'm emulating the usage of the TemporaryFolder in the other Junit4 tests. However - * this can be completely abstracted inside the HadoopCatalogExtension via - * Files.createTempDirectory(...) in its BeforeAll. Should I pursue the latter approach? - */ - @TempDir static Path temporaryFolder; - @RegisterExtension - HadoopCatalogExtension catalogExtension = - new HadoopCatalogExtension(temporaryFolder, DATABASE, TestFixtures.TABLE); + static HadoopCatalogExtension catalogExtension = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); @@ -253,33 +246,29 @@ public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) t } } - // @Test - // public void testBucketPartitionerFail() { - // Assume.assumeTrue(tableSchemaType == TableSchemaType.TWO_BUCKETS); - // - // Exception exception = - // Assert.assertThrows(RuntimeException.class, () -> new BucketPartitioner(table.spec())); - // - // Assert.assertTrue( - // exception - // .getMessage() - // .contains(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE)); - // } - // - // @Test - // public void testBucketPartitionKeySelectorFail() { - // Assume.assumeTrue(tableSchemaType == TableSchemaType.TWO_BUCKETS); - // - // Exception exception = - // Assert.assertThrows( - // RuntimeException.class, - // () -> new BucketPartitionKeySelector(table.spec(), table.schema(), null)); - // - // Assert.assertTrue( - // exception - // .getMessage() - // .contains(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE)); - // } + @ParameterizedTest + @EnumSource( + value = TableSchemaType.class, + names = {"TWO_BUCKETS"}) + public void testBucketPartitionerFail(TableSchemaType tableSchemaType) { + setupEnvironment(tableSchemaType); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> new BucketPartitioner(table.spec())) + .withMessageContaining(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE); + } + + @ParameterizedTest + @EnumSource( + value = TableSchemaType.class, + names = {"TWO_BUCKETS"}) + public void testBucketPartitionKeySelectorFail(TableSchemaType tableSchemaType) { + setupEnvironment(tableSchemaType); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> new BucketPartitionKeySelector(table.spec(), table.schema(), null)) + .withMessageContaining(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE); + } /** * Utility method to generate rows whose values will "hash" to a desired bucketId From 7310f9bba5d67c14d102cb8a4f7ce120a56e9c3b Mon Sep 17 00:00:00 2001 From: schongloo Date: Wed, 26 Apr 2023 09:53:51 -0700 Subject: [PATCH 13/23] - Splitting tests across TestBucketPartitionerFlinkIcebergSink (E2E), TestBucketPartitioner and TestBucketPartitionKeySelector. - Refactored and simplified the TestBucketPartitionerUtils. --- .../iceberg/flink/sink/BucketPartitioner.java | 7 + .../iceberg/flink/HadoopCatalogExtension.java | 3 +- .../sink/TestBucketPartitionKeySelector.java | 67 +++++++++ .../flink/sink/TestBucketPartitioner.java | 97 ++++++++++++ ...TestBucketPartitionerFlinkIcebergSink.java | 140 ++++-------------- .../sink/TestBucketPartitionerUtils.java | 121 +++++++++++++++ 6 files changed, 324 insertions(+), 111 deletions(-) create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 4685ff762e9b..11f9c6d7e38a 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -21,6 +21,7 @@ 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 @@ -55,6 +56,12 @@ class BucketPartitioner implements Partitioner { */ @Override public int partition(Integer bucketId, int numPartitions) { + Preconditions.checkArgument( + bucketId >= 0, "bucketId out of range: " + bucketId + " (must be >= 0)"); + Preconditions.checkArgument( + bucketId < maxNumBuckets, + "bucketId out of range: " + bucketId + " (must be >= 0), maxNumBuckets: " + maxNumBuckets); + if (numPartitions <= maxNumBuckets) { return bucketId % numPartitions; } else { diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java index f7d23c8bda37..6cbe1652bf01 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -18,7 +18,8 @@ */ package org.apache.iceberg.flink; -import java.io.File;import java.nio.file.Files;import java.nio.file.Path; +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; diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java new file mode 100644 index 000000000000..fc3d643fda5d --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -0,0 +1,67 @@ +/* + * 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.TestBucketPartitionerUtils.CONVERTER; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestBucketPartitionKeySelector { + + @ParameterizedTest + @EnumSource( + value = TestBucketPartitionerUtils.TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testCorrectKeySelection(TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { + final int numBuckets = 60; + + PartitionSpec partitionSpec = + TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, numBuckets); + BucketPartitionKeySelector keySelector = + new BucketPartitionKeySelector( + partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE); + + TestBucketPartitionerUtils.generateRowsForBucketIdRange(2, numBuckets) + .forEach( + row -> { + int bucketId = + TestBucketPartitionerUtils.computeBucketId(numBuckets, (String) row.getField(1)); + Integer key = keySelector.getKey(CONVERTER.toInternal(row)); + Assertions.assertThat(key).isEqualTo(bucketId); + }); + } + + @ParameterizedTest + @EnumSource(value = TestBucketPartitionerUtils.TableSchemaType.class, names = "TWO_BUCKETS") + public void testBucketPartitionKeySelectorFail( + TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { + PartitionSpec partitionSpec = TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, 1); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy( + () -> + new BucketPartitionKeySelector( + partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE)) + .withMessageContaining(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java new file mode 100644 index 000000000000..8b0bf28e298a --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -0,0 +1,97 @@ +/* + * 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.assertj.core.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestBucketPartitioner { + + static final int NUM_BUCKETS = 60; + + @ParameterizedTest + @EnumSource( + value = TestBucketPartitionerUtils.TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testPartitioningParallelismGreaterThanBuckets( + TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { + final int numPartitions = 500; + + PartitionSpec partitionSpec = + TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, NUM_BUCKETS); + + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + for (int expectedIdx = 0, bucketId = 0; expectedIdx < numPartitions; expectedIdx++) { + int actualIdx = bucketPartitioner.partition(bucketId, numPartitions); + Assertions.assertThat(actualIdx).isEqualTo(expectedIdx); + if (++bucketId == NUM_BUCKETS) { + bucketId = 0; + } + } + } + + @ParameterizedTest + @EnumSource( + value = TestBucketPartitionerUtils.TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testPartitioningParallelismEqualLessThanBuckets( + TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { + final int numPartitions = 30; + + PartitionSpec partitionSpec = + TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, NUM_BUCKETS); + + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + for (int bucketId = 0; bucketId < NUM_BUCKETS; bucketId++) { + int actualIdx = bucketPartitioner.partition(bucketId, numPartitions); + Assertions.assertThat(actualIdx).isEqualTo(bucketId % numPartitions); + } + } + + @ParameterizedTest + @EnumSource(value = TestBucketPartitionerUtils.TableSchemaType.class, names = "TWO_BUCKETS") + public void testPartitionerMultipleBucketsFail( + TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { + PartitionSpec partitionSpec = + TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, NUM_BUCKETS); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> new BucketPartitioner(partitionSpec)) + .withMessageContaining(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE); + } + + @ParameterizedTest + @ValueSource(ints = {-1, NUM_BUCKETS}) + public void testPartitionerBucketIdOutOfRangeFail(int bucketId) { + PartitionSpec partitionSpec = + TestBucketPartitionerUtils.getPartitionSpec( + TestBucketPartitionerUtils.TableSchemaType.ONE_BUCKET, NUM_BUCKETS); + + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> bucketPartitioner.partition(bucketId, 1)) + .withMessageContaining("out of range"); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index 15f6e9e207e0..4892d46b37e5 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -21,15 +21,14 @@ 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 static org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.CONVERTER; import java.io.IOException; -import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.TreeMap; -import java.util.UUID; import java.util.stream.Collectors; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -37,7 +36,6 @@ 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; @@ -50,32 +48,26 @@ 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.apache.iceberg.util.BucketUtil; import org.assertj.core.api.Assertions; +import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; public class TestBucketPartitionerFlinkIcebergSink { - private enum TableSchemaType { - ONE_BUCKET, - IDENTITY_AND_BUCKET, - TWO_BUCKETS; - } - private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) @@ -84,13 +76,11 @@ private enum TableSchemaType { .build()); @RegisterExtension - static HadoopCatalogExtension catalogExtension = + private static final HadoopCatalogExtension catalogExtension = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - private static final DataFormatConverters.RowConverter CONVERTER = - new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); // Parallelism = 8 throughout the test suite private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; @@ -112,28 +102,8 @@ private void setupEnvironment(TableSchemaType tableSchemaType) { } private Table getTable(TableSchemaType tableSchemaType) { - PartitionSpec partitionSpec = null; - - switch (tableSchemaType) { - case ONE_BUCKET: - partitionSpec = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); - break; - case IDENTITY_AND_BUCKET: - partitionSpec = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .identity("id") - .bucket("data", numBuckets) - .build(); - break; - case TWO_BUCKETS: - partitionSpec = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .bucket("id", numBuckets) - .bucket("data", numBuckets) - .build(); - break; - } + PartitionSpec partitionSpec = + TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, numBuckets); return catalogExtension .catalog() @@ -154,7 +124,7 @@ private BoundedTestSource createBoundedSource(List rows) { private TableTestStats extractTableTestStats(TableSchemaType tableSchemaType) throws IOException { int totalRecordCount = 0; - Map> writersPerBucket = Maps.newHashMap(); // > + Map> writersPerBucket = Maps.newHashMap(); // > Map filesPerBucket = Maps.newHashMap(); // Map recordsPerFile = new TreeMap<>(); // @@ -211,24 +181,12 @@ private void testWriteRowData(List allRows) throws Exception { names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) throws Exception { setupEnvironment(tableSchemaType); - - // Generating 16 records to be sent (8 writers -> 4 buckets), will be uniformly round-robin'd - // to all writers - int totalNumRows = parallelism * 2; - int numRowsPerBucket = totalNumRows / numBuckets; - List rows = Lists.newArrayListWithCapacity(totalNumRows); - List rowsPerBucket = Lists.newArrayListWithCapacity(numBuckets); - for (int i = 0; i < numBuckets; i++) { - rowsPerBucket.addAll(generateRowsForBucketId(1, i)); - } - for (int i = 0; i < numRowsPerBucket; i++) { - rows.addAll(rowsPerBucket); - } + List rows = generateTestDataRows(); testWriteRowData(rows); TableTestStats stats = extractTableTestStats(tableSchemaType); - Assertions.assertThat(stats.totalRowCount).isEqualTo(totalNumRows); + 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); @@ -246,73 +204,35 @@ public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) t } } + /** + * 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 testBucketPartitionerFail(TableSchemaType tableSchemaType) { + @EnumSource(value = TableSchemaType.class, names = "TWO_BUCKETS") + public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) throws Exception { setupEnvironment(tableSchemaType); + List rows = generateTestDataRows(); - Assertions.assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> new BucketPartitioner(table.spec())) - .withMessageContaining(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE); - } - - @ParameterizedTest - @EnumSource( - value = TableSchemaType.class, - names = {"TWO_BUCKETS"}) - public void testBucketPartitionKeySelectorFail(TableSchemaType tableSchemaType) { - setupEnvironment(tableSchemaType); - - Assertions.assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> new BucketPartitionKeySelector(table.spec(), table.schema(), null)) - .withMessageContaining(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE); - } - - /** - * Utility method to generate rows whose values will "hash" to a desired bucketId - * - * @param numRows the number of rows/requests to generate - * @param bucketId the desired bucketId - * @return the list of rows whose data "hashes" to the desired bucketId - */ - private List generateRowsForBucketId(int numRows, int bucketId) { - List rows = Lists.newArrayListWithCapacity(numRows); - for (int i = 0; i < numRows; i++) { - String value = generateValueForBucketId(bucketId, numBuckets); - rows.add(Row.of(1, value)); - } - return rows; - } + testWriteRowData(rows); + TableTestStats stats = extractTableTestStats(tableSchemaType); - /** - * Utility method to generate a UUID string that will "hash" to a desired bucketId - * - * @param bucketId the desired bucketId - * @return the string data that "hashes" to the desired bucketId - */ - public static String generateValueForBucketId(int bucketId, int numBuckets) { - String value = ""; - while (true) { - String uuid = UUID.randomUUID().toString(); - if (hash(numBuckets, uuid) == bucketId) { - value = uuid; - break; - } + 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); } - return value; } /** - * Utility method that performs the same "hashing" mechanism used by Bucket.java - * - * @param numBuckets the maximum number of buckets - * @param uuid the UUID string value to hash - * @return the target BucketId + * Generating 16 rows to be sent uniformly to all writers (round-robin across 8 writers -> 4 + * buckets) */ - public static int hash(int numBuckets, String uuid) { - return (BucketUtil.hash(uuid) & Integer.MAX_VALUE) % numBuckets; + @NotNull + private List generateTestDataRows() { + int totalNumRows = parallelism * 2; + int numRowsPerBucket = totalNumRows / numBuckets; + return TestBucketPartitionerUtils.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); } /** DTO to hold Test Stats */ diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java new file mode 100644 index 000000000000..db0f0b410468 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java @@ -0,0 +1,121 @@ +/* + * 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.util.DataFormatConverters; +import org.apache.flink.types.Row; +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; + } + + private TestBucketPartitionerUtils() {} + + static final DataFormatConverters.RowConverter CONVERTER = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + + static PartitionSpec getPartitionSpec(TableSchemaType tableSchemaType, int numBuckets) { + PartitionSpec partitionSpec = null; + + switch (tableSchemaType) { + case ONE_BUCKET: + partitionSpec = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); + break; + case IDENTITY_AND_BUCKET: + partitionSpec = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .identity("id") + .bucket("data", numBuckets) + .build(); + break; + case TWO_BUCKETS: + partitionSpec = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .bucket("id", numBuckets) + .bucket("data", numBuckets) + .build(); + break; + } + + Preconditions.checkNotNull( + partitionSpec, "Invalid tableSchemaType provided: " + tableSchemaType); + return partitionSpec; + } + + /** + * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to + * numBuckets) + * + * @param numRowsPerBucket how many different rows should be generated per bucket + * @param numBuckets max number of buckets to consider + * @return the list of rows whose data "hashes" to the desired bucketId + */ + public static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { + List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); + // For some of our tests, this order of the generated rows matters + for (int i = 0; i < numRowsPerBucket; i++) { + for (int bucketId = 0; bucketId < numBuckets; bucketId++) { + String value = generateValueForBucketId(bucketId, numBuckets); + rows.add(Row.of(1, value)); + } + } + return rows; + } + + /** + * Utility method to generate a UUID string that will "hash" to a desired bucketId + * + * @param bucketId the desired bucketId + * @return the string data that "hashes" to the desired bucketId + */ + private static String generateValueForBucketId(int bucketId, int numBuckets) { + String value = ""; + while (true) { + String uuid = UUID.randomUUID().toString(); + if (computeBucketId(numBuckets, uuid) == bucketId) { + value = uuid; + break; + } + } + return value; + } + + /** + * Utility that performs the same hashing/bucketing mechanism used by Bucket.java + * + * @param numBuckets max number of buckets to consider + * @param value the string to compute the bucketId from + * @return the computed bucketId + */ + static int computeBucketId(int numBuckets, String value) { + return (BucketUtil.hash(value) & Integer.MAX_VALUE) % numBuckets; + } +} From 26dae02e19855cec8a0f3afd56f25de467420646 Mon Sep 17 00:00:00 2001 From: schongloo Date: Thu, 27 Apr 2023 09:53:32 -0700 Subject: [PATCH 14/23] Additional clean up --- .../sink/BucketPartitionKeySelector.java | 4 +-- .../iceberg/flink/sink/BucketPartitioner.java | 29 ++++++++++--------- .../flink/sink/BucketPartitionerUtils.java | 10 +++++++ .../sink/TestBucketPartitionKeySelector.java | 6 ++-- .../flink/sink/TestBucketPartitioner.java | 3 +- ...TestBucketPartitionerFlinkIcebergSink.java | 2 +- .../sink/TestBucketPartitionerUtils.java | 4 +-- 7 files changed, 36 insertions(+), 22 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index 0ddfc2d878e2..3777c3d558f3 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -29,8 +29,8 @@ import org.apache.iceberg.flink.RowDataWrapper; /** - * A {@link KeySelector} that extracts and returns an Integer bucketId as the key. To be used with - * the {@link BucketPartitioner}. + * 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 { diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 11f9c6d7e38a..0c4c05d86f28 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -31,9 +31,11 @@ */ class BucketPartitioner implements Partitioner { + static final String BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX = "bucketId out of range: "; + private final int maxNumBuckets; - // This always has the OFFSET of the next writer to use for any bucket + // To hold the OFFSET of the next writer to use for any bucket private final int[] currentBucketWriterOffset; BucketPartitioner(PartitionSpec partitionSpec) { @@ -45,22 +47,23 @@ class BucketPartitioner implements Partitioner { } /** - * If the number of writers <= the number of buckets an evenly distributed number of buckets will - * be assigned to each writer. Conversely, if the number of writers > the number of buckets the - * logic is handled by the {@link #getPartitionWritersGreaterThanBuckets - * getPartitionWritersGreaterThanBuckets} method. + * Determine the partition id based on the following criteria: If the number of writers <= the + * number of buckets, an evenly distributed number of buckets will be assigned to each writer + * (one writer -> many buckets). + * Conversely, if the number of writers > the number of buckets the logic is handled by the + * {@link #getPartitionWritersGreaterThanBuckets getPartitionWritersGreaterThanBuckets} method. * * @param bucketId the bucketId for each request * @param numPartitions the total number of partitions - * @return the partition index (writer) to use for each request + * @return the partition id (writer) to use for each request */ @Override public int partition(Integer bucketId, int numPartitions) { Preconditions.checkArgument( - bucketId >= 0, "bucketId out of range: " + bucketId + " (must be >= 0)"); + bucketId >= 0, BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX + bucketId + " (must be >= 0)"); Preconditions.checkArgument( bucketId < maxNumBuckets, - "bucketId out of range: " + bucketId + " (must be >= 0), maxNumBuckets: " + maxNumBuckets); + BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX + bucketId + " (must be >= 0), maxNumBuckets: " + maxNumBuckets); if (numPartitions <= maxNumBuckets) { return bucketId % numPartitions; @@ -71,12 +74,12 @@ public int partition(Integer bucketId, int numPartitions) { /*- * If the number of writers > the 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, in - * this case each writer will target no more than one bucket at all times. Example: - * Configuration: numPartitions (writers) = 3, maxBuckets = 2 + * writers per bucket as evenly as possible, and will round-robin the requests across them, in this + * case each writer will target only one bucket at all times (many writers -> one bucket). Example: + * Configuration: numPartitions (writers) = 5, maxBuckets = 2 * Expected behavior: - * - Records for Bucket 0 will be "round robin" between Writers 0 and 2 - * - Records for Bucket 1 will always use Writer 1 + * - Records for Bucket 0 will be "round robin" between Writers 0, 2 and 4 + * - Records for Bucket 1 will always use Writer 1 and 3 * Notes: * - maxNumWritersPerBucket determines when to reset the currentBucketWriterOffset to 0 for this bucketId * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter). diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java index c2d3787cf25d..e807bc0daa78 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java @@ -32,11 +32,21 @@ final class BucketPartitionerUtils { private BucketPartitionerUtils() {} + /** + * Determines whether the PartitionSpec has one and only one Bucket definition + * @param partitionSpec the partition spec in question + * @return whether the PartitionSpec has only one Bucket + */ static boolean hasOneBucketField(PartitionSpec partitionSpec) { List> bucketFields = getBucketFields(partitionSpec); return bucketFields != null && bucketFields.size() == 1; } + /** + * Extracts the Bucket definition from a PartitionSpec. + * @param partitionSpec the partition spec in question + * @return the Bucket definition in the form of a tuple + */ static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { List> bucketFields = getBucketFields(partitionSpec); Preconditions.checkArgument( diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java index fc3d643fda5d..2f949041b368 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -44,16 +44,16 @@ public void testCorrectKeySelection(TestBucketPartitionerUtils.TableSchemaType t TestBucketPartitionerUtils.generateRowsForBucketIdRange(2, numBuckets) .forEach( row -> { - int bucketId = + int expectedBucketId = TestBucketPartitionerUtils.computeBucketId(numBuckets, (String) row.getField(1)); Integer key = keySelector.getKey(CONVERTER.toInternal(row)); - Assertions.assertThat(key).isEqualTo(bucketId); + Assertions.assertThat(key).isEqualTo(expectedBucketId); }); } @ParameterizedTest @EnumSource(value = TestBucketPartitionerUtils.TableSchemaType.class, names = "TWO_BUCKETS") - public void testBucketPartitionKeySelectorFail( + public void testKeySelectorMultipleBucketsFail( TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { PartitionSpec partitionSpec = TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, 1); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index 8b0bf28e298a..2dd58cfa2602 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -23,6 +23,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.ValueSource; +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX; public class TestBucketPartitioner { @@ -92,6 +93,6 @@ public void testPartitionerBucketIdOutOfRangeFail(int bucketId) { Assertions.assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> bucketPartitioner.partition(bucketId, 1)) - .withMessageContaining("out of range"); + .withMessageContaining(BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index 4892d46b37e5..336657a7950e 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -82,7 +82,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - // Parallelism = 8 throughout the test suite + // 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; diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java index db0f0b410468..a06b8fd83724 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java @@ -72,13 +72,13 @@ static PartitionSpec getPartitionSpec(TableSchemaType tableSchemaType, int numBu /** * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to - * numBuckets) + * numBuckets - 1) * * @param numRowsPerBucket how many different rows should be generated per bucket * @param numBuckets max number of buckets to consider * @return the list of rows whose data "hashes" to the desired bucketId */ - public static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { + static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); // For some of our tests, this order of the generated rows matters for (int i = 0; i < numRowsPerBucket; i++) { From 3bc595611c587c3ee870374687a23bc4d7bf11c9 Mon Sep 17 00:00:00 2001 From: schongloo Date: Thu, 27 Apr 2023 12:15:47 -0700 Subject: [PATCH 15/23] Checkstyle fixes --- .../iceberg/flink/sink/BucketPartitioner.java | 13 ++++++++----- .../iceberg/flink/sink/BucketPartitionerUtils.java | 4 +++- .../iceberg/flink/sink/TestBucketPartitioner.java | 3 ++- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 0c4c05d86f28..ca2e690ce58f 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -48,10 +48,10 @@ class BucketPartitioner implements Partitioner { /** * Determine the partition id based on the following criteria: If the number of writers <= the - * number of buckets, an evenly distributed number of buckets will be assigned to each writer - * (one writer -> many buckets). - * Conversely, if the number of writers > the number of buckets the logic is handled by the - * {@link #getPartitionWritersGreaterThanBuckets getPartitionWritersGreaterThanBuckets} method. + * number of buckets, an evenly distributed number of buckets will be assigned to each writer (one + * writer -> many buckets). Conversely, if the number of writers > the number of buckets the logic + * is handled by the {@link #getPartitionWritersGreaterThanBuckets + * getPartitionWritersGreaterThanBuckets} method. * * @param bucketId the bucketId for each request * @param numPartitions the total number of partitions @@ -63,7 +63,10 @@ public int partition(Integer bucketId, int numPartitions) { bucketId >= 0, BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX + bucketId + " (must be >= 0)"); Preconditions.checkArgument( bucketId < maxNumBuckets, - BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX + bucketId + " (must be >= 0), maxNumBuckets: " + maxNumBuckets); + BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX + + bucketId + + " (must be >= 0), maxNumBuckets: " + + maxNumBuckets); if (numPartitions <= maxNumBuckets) { return bucketId % numPartitions; diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java index e807bc0daa78..689fa0b715ab 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java @@ -34,6 +34,7 @@ private BucketPartitionerUtils() {} /** * Determines whether the PartitionSpec has one and only one Bucket definition + * * @param partitionSpec the partition spec in question * @return whether the PartitionSpec has only one Bucket */ @@ -44,8 +45,9 @@ static boolean hasOneBucketField(PartitionSpec partitionSpec) { /** * Extracts the Bucket definition from a PartitionSpec. + * * @param partitionSpec the partition spec in question - * @return the Bucket definition in the form of a tuple + * @return the Bucket definition in the form of a tuple (fieldId, maxNumBuckets) */ static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { List> bucketFields = getBucketFields(partitionSpec); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index 2dd58cfa2602..04d2384b32d0 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX; + import org.apache.iceberg.PartitionSpec; import org.assertj.core.api.Assertions; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.ValueSource; -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX; public class TestBucketPartitioner { From 7c118794ff3f6dcc4903d32830aabd817f3a7484 Mon Sep 17 00:00:00 2001 From: schongloo Date: Sat, 20 May 2023 08:44:44 -0700 Subject: [PATCH 16/23] Addressing PR comments --- .../sink/BucketPartitionKeySelector.java | 4 - .../iceberg/flink/sink/BucketPartitioner.java | 30 +++--- .../flink/sink/BucketPartitionerUtils.java | 5 +- .../sink/TestBucketPartitionKeySelector.java | 13 +-- .../flink/sink/TestBucketPartitioner.java | 93 +++++++++++-------- ...TestBucketPartitionerFlinkIcebergSink.java | 73 ++++++++------- .../sink/TestBucketPartitionerUtils.java | 9 +- 7 files changed, 122 insertions(+), 105 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index 3777c3d558f3..cdf88396f5f3 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -55,10 +55,6 @@ class BucketPartitionKeySelector implements KeySelector { .toArray()[0]; } - /** - * 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()); diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index ca2e690ce58f..edbde75ec95a 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -31,11 +31,17 @@ */ class BucketPartitioner implements Partitioner { - static final String BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX = "bucketId out of range: "; + 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 + // 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) { @@ -50,7 +56,7 @@ class BucketPartitioner implements Partitioner { * Determine the partition id based on the following criteria: If the number of writers <= the * number of buckets, an evenly distributed number of buckets will be assigned to each writer (one * writer -> many buckets). Conversely, if the number of writers > the number of buckets the logic - * is handled by the {@link #getPartitionWritersGreaterThanBuckets + * is handled by the {@link #getPartitionWithMoreWritersThanBuckets * getPartitionWritersGreaterThanBuckets} method. * * @param bucketId the bucketId for each request @@ -59,19 +65,15 @@ class BucketPartitioner implements Partitioner { */ @Override public int partition(Integer bucketId, int numPartitions) { + Preconditions.checkNotNull(bucketId, BUCKET_NULL_MESSAGE); + Preconditions.checkArgument(bucketId >= 0, BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, bucketId); Preconditions.checkArgument( - bucketId >= 0, BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX + bucketId + " (must be >= 0)"); - Preconditions.checkArgument( - bucketId < maxNumBuckets, - BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX - + bucketId - + " (must be >= 0), maxNumBuckets: " - + maxNumBuckets); + bucketId < maxNumBuckets, BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, bucketId, maxNumBuckets); if (numPartitions <= maxNumBuckets) { return bucketId % numPartitions; } else { - return getPartitionWritersGreaterThanBuckets(bucketId, numPartitions); + return getPartitionWithMoreWritersThanBuckets(bucketId, numPartitions); } } @@ -88,11 +90,9 @@ public int partition(Integer bucketId, int numPartitions) { * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter). * In this example Bucket 0 has an "extra writer" to consider before resetting its offset to 0. * - * @param bucketId the bucketId for each request - * @param numPartitions the total number of partitions - * @return the partition index (writer) to use for each request + * @return the destination partition index (writer subtask id) */ - private int getPartitionWritersGreaterThanBuckets(int bucketId, int numPartitions) { + private int getPartitionWithMoreWritersThanBuckets(int bucketId, int numPartitions) { int currentOffset = currentBucketWriterOffset[bucketId]; // Determine if this bucket requires an "extra writer" int extraWriter = bucketId < (numPartitions % maxNumBuckets) ? 1 : 0; diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java index 689fa0b715ab..cade5ac29de4 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java @@ -28,7 +28,7 @@ final class BucketPartitionerUtils { static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = - "Expected 1 Bucket transform in the provided PartitionSpec, found: "; + "Expected 1 Bucket transform in the provided PartitionSpec, found: %s"; private BucketPartitionerUtils() {} @@ -53,7 +53,8 @@ static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) List> bucketFields = getBucketFields(partitionSpec); Preconditions.checkArgument( bucketFields.size() == 1, - BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE + bucketFields.size()); + BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, + bucketFields.size()); return bucketFields.get(0); } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java index 2f949041b368..d97b7dbbd411 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -23,6 +23,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.flink.SimpleDataUtil; 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; @@ -51,17 +52,17 @@ public void testCorrectKeySelection(TestBucketPartitionerUtils.TableSchemaType t }); } - @ParameterizedTest - @EnumSource(value = TestBucketPartitionerUtils.TableSchemaType.class, names = "TWO_BUCKETS") - public void testKeySelectorMultipleBucketsFail( - TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { - PartitionSpec partitionSpec = TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, 1); + @Test + public void testKeySelectorMultipleBucketsFail() { + PartitionSpec partitionSpec = + TestBucketPartitionerUtils.getPartitionSpec( + TestBucketPartitionerUtils.TableSchemaType.TWO_BUCKETS, 1); Assertions.assertThatExceptionOfType(RuntimeException.class) .isThrownBy( () -> new BucketPartitionKeySelector( partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE)) - .withMessageContaining(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE); + .withMessage(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index 04d2384b32d0..e58209ba9f37 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -18,82 +18,99 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX; +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.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.CsvSource; public class TestBucketPartitioner { - static final int NUM_BUCKETS = 60; + static final int DEFAULT_NUM_BUCKETS = 60; @ParameterizedTest - @EnumSource( - value = TestBucketPartitionerUtils.TableSchemaType.class, - names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) public void testPartitioningParallelismGreaterThanBuckets( - TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { + String schemaTypeStr, String numBucketsStr) { final int numPartitions = 500; - + final TestBucketPartitionerUtils.TableSchemaType tableSchemaType = + TestBucketPartitionerUtils.TableSchemaType.valueOf(schemaTypeStr); + final int numBuckets = Integer.parseInt(numBucketsStr); PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, NUM_BUCKETS); - + TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, numBuckets); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - for (int expectedIdx = 0, bucketId = 0; expectedIdx < numPartitions; expectedIdx++) { - int actualIdx = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualIdx).isEqualTo(expectedIdx); - if (++bucketId == NUM_BUCKETS) { + int bucketId = 0; + for (int expectedIdx = 0; expectedIdx < numPartitions; expectedIdx++) { + int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); + Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIdx); + bucketId++; + if (bucketId == numBuckets) { bucketId = 0; } } } @ParameterizedTest - @EnumSource( - value = TestBucketPartitionerUtils.TableSchemaType.class, - names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) public void testPartitioningParallelismEqualLessThanBuckets( - TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { + String schemaTypeStr, String numBucketsStr) { final int numPartitions = 30; - + final TestBucketPartitionerUtils.TableSchemaType tableSchemaType = + TestBucketPartitionerUtils.TableSchemaType.valueOf(schemaTypeStr); + final int numBuckets = Integer.parseInt(numBucketsStr); PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, NUM_BUCKETS); - + TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, numBuckets); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - for (int bucketId = 0; bucketId < NUM_BUCKETS; bucketId++) { - int actualIdx = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualIdx).isEqualTo(bucketId % numPartitions); + for (int bucketId = 0; bucketId < numBuckets; bucketId++) { + int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); + Assertions.assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); } } - @ParameterizedTest - @EnumSource(value = TestBucketPartitionerUtils.TableSchemaType.class, names = "TWO_BUCKETS") - public void testPartitionerMultipleBucketsFail( - TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { + @Test + public void testPartitionerBucketIdNullFail() { PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, NUM_BUCKETS); + TestBucketPartitionerUtils.getPartitionSpec( + TestBucketPartitionerUtils.TableSchemaType.ONE_BUCKET, DEFAULT_NUM_BUCKETS); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); Assertions.assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> new BucketPartitioner(partitionSpec)) - .withMessageContaining(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE); + .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) + .withMessage(BUCKET_NULL_MESSAGE); } - @ParameterizedTest - @ValueSource(ints = {-1, NUM_BUCKETS}) - public void testPartitionerBucketIdOutOfRangeFail(int bucketId) { + @Test + public void testPartitionerMultipleBucketsFail() { PartitionSpec partitionSpec = TestBucketPartitionerUtils.getPartitionSpec( - TestBucketPartitionerUtils.TableSchemaType.ONE_BUCKET, NUM_BUCKETS); + TestBucketPartitionerUtils.TableSchemaType.TWO_BUCKETS, DEFAULT_NUM_BUCKETS); + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> new BucketPartitioner(partitionSpec)) + .withMessage(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); + } + + @Test + public void testPartitionerBucketIdOutOfRangeFail() { + PartitionSpec partitionSpec = + TestBucketPartitionerUtils.getPartitionSpec( + TestBucketPartitionerUtils.TableSchemaType.ONE_BUCKET, DEFAULT_NUM_BUCKETS); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + int negativeBucketId = -1; + Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) + .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); + + int tooBigBucketId = DEFAULT_NUM_BUCKETS; Assertions.assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> bucketPartitioner.partition(bucketId, 1)) - .withMessageContaining(BUCKET_OUT_OF_RANGE_MESSAGE_PREFIX); + .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) + .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index 336657a7950e..ebec935e4ab6 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -122,40 +122,6 @@ private BoundedTestSource createBoundedSource(List rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } - private TableTestStats extractTableTestStats(TableSchemaType tableSchemaType) throws IOException { - int totalRecordCount = 0; - Map> writersPerBucket = Maps.newHashMap(); // > - Map filesPerBucket = Maps.newHashMap(); // - Map recordsPerFile = new TreeMap<>(); // - - try (CloseableIterable 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 == TableSchemaType.ONE_BUCKET ? 0 : 1, 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()) { - Collections.sort(writersPerBucket.get(k)); - } - - return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, recordsPerFile); - } - private void testWriteRowData(List allRows) throws Exception { DataStream dataStream = env.addSource(createBoundedSource(allRows), ROW_TYPE_INFO) @@ -184,7 +150,7 @@ public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) t List rows = generateTestDataRows(); testWriteRowData(rows); - TableTestStats stats = extractTableTestStats(tableSchemaType); + TableTestStats stats = extractPartitionResults(tableSchemaType); Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); // All 4 buckets should've been written to @@ -215,7 +181,7 @@ public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) throws List rows = generateTestDataRows(); testWriteRowData(rows); - TableTestStats stats = extractTableTestStats(tableSchemaType); + TableTestStats stats = extractPartitionResults(tableSchemaType); Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { @@ -235,6 +201,41 @@ private List generateTestDataRows() { return TestBucketPartitionerUtils.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); } + private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) + throws IOException { + int totalRecordCount = 0; + Map> writersPerBucket = Maps.newHashMap(); // > + Map filesPerBucket = Maps.newHashMap(); // + Map recordsPerFile = new TreeMap<>(); // + + try (CloseableIterable 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()) { + Collections.sort(writersPerBucket.get(k)); + } + + return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, recordsPerFile); + } + /** DTO to hold Test Stats */ private static class TableTestStats { final int totalRowCount; diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java index a06b8fd83724..b981557273d0 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java @@ -34,6 +34,10 @@ enum TableSchemaType { ONE_BUCKET, IDENTITY_AND_BUCKET, TWO_BUCKETS; + + public static int bucketPartitionColumnPosition(TableSchemaType tableSchemaType) { + return tableSchemaType == ONE_BUCKET ? 0 : 1; + } } private TestBucketPartitionerUtils() {} @@ -97,15 +101,12 @@ static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBucke * @return the string data that "hashes" to the desired bucketId */ private static String generateValueForBucketId(int bucketId, int numBuckets) { - String value = ""; while (true) { String uuid = UUID.randomUUID().toString(); if (computeBucketId(numBuckets, uuid) == bucketId) { - value = uuid; - break; + return uuid; } } - return value; } /** From b44bd99bdea423395582f840ab45ea5617fae5d1 Mon Sep 17 00:00:00 2001 From: schongloo Date: Sat, 20 May 2023 09:20:28 -0700 Subject: [PATCH 17/23] Using RowData as the main type across tests. --- .../sink/TestBucketPartitionKeySelector.java | 9 +++--- ...TestBucketPartitionerFlinkIcebergSink.java | 29 +++++++++---------- .../sink/TestBucketPartitionerUtils.java | 14 ++++----- 3 files changed, 23 insertions(+), 29 deletions(-) diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java index d97b7dbbd411..dee7234f1a80 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -18,8 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.CONVERTER; - import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.flink.SimpleDataUtil; import org.assertj.core.api.Assertions; @@ -44,10 +42,11 @@ public void testCorrectKeySelection(TestBucketPartitionerUtils.TableSchemaType t TestBucketPartitionerUtils.generateRowsForBucketIdRange(2, numBuckets) .forEach( - row -> { + rowData -> { int expectedBucketId = - TestBucketPartitionerUtils.computeBucketId(numBuckets, (String) row.getField(1)); - Integer key = keySelector.getKey(CONVERTER.toInternal(row)); + TestBucketPartitionerUtils.computeBucketId( + numBuckets, rowData.getString(1).toString()); + Integer key = keySelector.getKey(rowData); Assertions.assertThat(key).isEqualTo(expectedBucketId); }); } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index ebec935e4ab6..29554da8cdf9 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -21,7 +21,6 @@ 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 static org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.CONVERTER; import java.io.IOException; import java.util.Arrays; @@ -29,13 +28,13 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; -import java.util.stream.Collectors; 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; @@ -114,18 +113,16 @@ private Table getTable(TableSchemaType tableSchemaType) { ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); } - private List convertToRowData(List rows) { - return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); - } - - private BoundedTestSource createBoundedSource(List rows) { - return new BoundedTestSource<>(rows.toArray(new Row[0])); - } + private void testWriteRowData(List allRows) throws Exception { + DataFormatConverters.RowConverter converter = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - private void testWriteRowData(List allRows) throws Exception { DataStream dataStream = - env.addSource(createBoundedSource(allRows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + 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) @@ -138,7 +135,7 @@ private void testWriteRowData(List allRows) throws Exception { env.execute("Test Iceberg DataStream"); // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(allRows)); + SimpleDataUtil.assertTableRows(table, allRows); } @ParameterizedTest @@ -147,7 +144,7 @@ private void testWriteRowData(List allRows) throws Exception { names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) throws Exception { setupEnvironment(tableSchemaType); - List rows = generateTestDataRows(); + List rows = generateTestDataRows(); testWriteRowData(rows); TableTestStats stats = extractPartitionResults(tableSchemaType); @@ -178,7 +175,7 @@ public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) t @EnumSource(value = TableSchemaType.class, names = "TWO_BUCKETS") public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) throws Exception { setupEnvironment(tableSchemaType); - List rows = generateTestDataRows(); + List rows = generateTestDataRows(); testWriteRowData(rows); TableTestStats stats = extractPartitionResults(tableSchemaType); @@ -195,7 +192,7 @@ public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) throws * buckets) */ @NotNull - private List generateTestDataRows() { + private List generateTestDataRows() { int totalNumRows = parallelism * 2; int numRowsPerBucket = totalNumRows / numBuckets; return TestBucketPartitionerUtils.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java index b981557273d0..5d7b2837c4bc 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java @@ -20,8 +20,9 @@ import java.util.List; import java.util.UUID; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.types.Row; +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; @@ -42,9 +43,6 @@ public static int bucketPartitionColumnPosition(TableSchemaType tableSchemaType) private TestBucketPartitionerUtils() {} - static final DataFormatConverters.RowConverter CONVERTER = - new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - static PartitionSpec getPartitionSpec(TableSchemaType tableSchemaType, int numBuckets) { PartitionSpec partitionSpec = null; @@ -82,13 +80,13 @@ static PartitionSpec getPartitionSpec(TableSchemaType tableSchemaType, int numBu * @param numBuckets max number of buckets to consider * @return the list of rows whose data "hashes" to the desired bucketId */ - static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { - List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); + static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { + List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); // For some of our tests, this order of the generated rows matters for (int i = 0; i < numRowsPerBucket; i++) { for (int bucketId = 0; bucketId < numBuckets; bucketId++) { String value = generateValueForBucketId(bucketId, numBuckets); - rows.add(Row.of(1, value)); + rows.add(GenericRowData.of(1, StringData.fromString(value))); } } return rows; From 16518048a4d480089ef4218c015e8fb35fe37722 Mon Sep 17 00:00:00 2001 From: schongloo Date: Wed, 24 May 2023 15:24:48 -0700 Subject: [PATCH 18/23] Refactoring the partitioning/schema utilities --- .../sink/TestBucketPartitionKeySelector.java | 12 ++--- .../flink/sink/TestBucketPartitioner.java | 22 +++----- ...TestBucketPartitionerFlinkIcebergSink.java | 3 +- .../sink/TestBucketPartitionerUtils.java | 54 +++++++++---------- 4 files changed, 41 insertions(+), 50 deletions(-) diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java index dee7234f1a80..fdb55773d64f 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -20,6 +20,7 @@ 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; @@ -29,13 +30,12 @@ public class TestBucketPartitionKeySelector { @ParameterizedTest @EnumSource( - value = TestBucketPartitionerUtils.TableSchemaType.class, + value = TableSchemaType.class, names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) - public void testCorrectKeySelection(TestBucketPartitionerUtils.TableSchemaType tableSchemaType) { + public void testCorrectKeySelection(TableSchemaType tableSchemaType) { final int numBuckets = 60; - PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, numBuckets); + PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); BucketPartitionKeySelector keySelector = new BucketPartitionKeySelector( partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE); @@ -53,9 +53,7 @@ public void testCorrectKeySelection(TestBucketPartitionerUtils.TableSchemaType t @Test public void testKeySelectorMultipleBucketsFail() { - PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec( - TestBucketPartitionerUtils.TableSchemaType.TWO_BUCKETS, 1); + PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(TableSchemaType.TWO_BUCKETS, 1); Assertions.assertThatExceptionOfType(RuntimeException.class) .isThrownBy( diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index e58209ba9f37..9841dd17e948 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -23,6 +23,7 @@ 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; @@ -37,11 +38,9 @@ public class TestBucketPartitioner { public void testPartitioningParallelismGreaterThanBuckets( String schemaTypeStr, String numBucketsStr) { final int numPartitions = 500; - final TestBucketPartitionerUtils.TableSchemaType tableSchemaType = - TestBucketPartitionerUtils.TableSchemaType.valueOf(schemaTypeStr); + final TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); final int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, numBuckets); + PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); int bucketId = 0; @@ -60,11 +59,9 @@ public void testPartitioningParallelismGreaterThanBuckets( public void testPartitioningParallelismEqualLessThanBuckets( String schemaTypeStr, String numBucketsStr) { final int numPartitions = 30; - final TestBucketPartitionerUtils.TableSchemaType tableSchemaType = - TestBucketPartitionerUtils.TableSchemaType.valueOf(schemaTypeStr); + final TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); final int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, numBuckets); + PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); for (int bucketId = 0; bucketId < numBuckets; bucketId++) { @@ -76,8 +73,7 @@ public void testPartitioningParallelismEqualLessThanBuckets( @Test public void testPartitionerBucketIdNullFail() { PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec( - TestBucketPartitionerUtils.TableSchemaType.ONE_BUCKET, DEFAULT_NUM_BUCKETS); + TableSchemaType.getPartitionSpec(TableSchemaType.ONE_BUCKET, DEFAULT_NUM_BUCKETS); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); Assertions.assertThatExceptionOfType(RuntimeException.class) @@ -88,8 +84,7 @@ public void testPartitionerBucketIdNullFail() { @Test public void testPartitionerMultipleBucketsFail() { PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec( - TestBucketPartitionerUtils.TableSchemaType.TWO_BUCKETS, DEFAULT_NUM_BUCKETS); + TableSchemaType.getPartitionSpec(TableSchemaType.TWO_BUCKETS, DEFAULT_NUM_BUCKETS); Assertions.assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> new BucketPartitioner(partitionSpec)) @@ -99,8 +94,7 @@ public void testPartitionerMultipleBucketsFail() { @Test public void testPartitionerBucketIdOutOfRangeFail() { PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec( - TestBucketPartitionerUtils.TableSchemaType.ONE_BUCKET, DEFAULT_NUM_BUCKETS); + TableSchemaType.getPartitionSpec(TableSchemaType.ONE_BUCKET, DEFAULT_NUM_BUCKETS); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); int negativeBucketId = -1; diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index 29554da8cdf9..6df08677e3e8 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -101,8 +101,7 @@ private void setupEnvironment(TableSchemaType tableSchemaType) { } private Table getTable(TableSchemaType tableSchemaType) { - PartitionSpec partitionSpec = - TestBucketPartitionerUtils.getPartitionSpec(tableSchemaType, numBuckets); + PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); return catalogExtension .catalog() diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java index 5d7b2837c4bc..910cf0c0834f 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java @@ -39,39 +39,39 @@ enum TableSchemaType { public static int bucketPartitionColumnPosition(TableSchemaType tableSchemaType) { return tableSchemaType == ONE_BUCKET ? 0 : 1; } - } - private TestBucketPartitionerUtils() {} + public static PartitionSpec getPartitionSpec(TableSchemaType tableSchemaType, int numBuckets) { + PartitionSpec partitionSpec = null; - static PartitionSpec getPartitionSpec(TableSchemaType tableSchemaType, int numBuckets) { - PartitionSpec partitionSpec = null; + switch (tableSchemaType) { + case ONE_BUCKET: + partitionSpec = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); + break; + case IDENTITY_AND_BUCKET: + partitionSpec = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .identity("id") + .bucket("data", numBuckets) + .build(); + break; + case TWO_BUCKETS: + partitionSpec = + PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .bucket("id", numBuckets) + .bucket("data", numBuckets) + .build(); + break; + } - switch (tableSchemaType) { - case ONE_BUCKET: - partitionSpec = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); - break; - case IDENTITY_AND_BUCKET: - partitionSpec = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .identity("id") - .bucket("data", numBuckets) - .build(); - break; - case TWO_BUCKETS: - partitionSpec = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .bucket("id", numBuckets) - .bucket("data", numBuckets) - .build(); - break; + Preconditions.checkNotNull( + partitionSpec, "Invalid tableSchemaType provided: " + tableSchemaType); + return partitionSpec; } - - Preconditions.checkNotNull( - partitionSpec, "Invalid tableSchemaType provided: " + tableSchemaType); - return partitionSpec; } + private TestBucketPartitionerUtils() {} + /** * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to * numBuckets - 1) From 836daa626998d4f02f20b0a396ebcc7dcb3f8726 Mon Sep 17 00:00:00 2001 From: schongloo Date: Tue, 30 May 2023 09:37:13 -0700 Subject: [PATCH 19/23] Latest round of PR comments --- .../sink/BucketPartitionKeySelector.java | 6 +- .../iceberg/flink/sink/BucketPartitioner.java | 9 ++- ...rUtils.java => BucketPartitionerUtil.java} | 8 +-- .../apache/iceberg/flink/sink/FlinkSink.java | 2 +- .../sink/TestBucketPartitionKeySelector.java | 4 +- .../flink/sink/TestBucketPartitioner.java | 14 ++--- ...TestBucketPartitionerFlinkIcebergSink.java | 58 ++++++++----------- 7 files changed, 42 insertions(+), 59 deletions(-) rename flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/{BucketPartitionerUtils.java => BucketPartitionerUtil.java} (94%) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index cdf88396f5f3..ca8e727bfe91 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -20,7 +20,6 @@ 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; @@ -42,13 +41,10 @@ class BucketPartitionKeySelector implements KeySelector { private transient RowDataWrapper rowDataWrapper; BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, RowType flinkSchema) { - Tuple2 bucketFieldInfo = - BucketPartitionerUtils.getBucketFieldInfo(partitionSpec); - - int bucketFieldId = bucketFieldInfo.f0; this.schema = schema; this.partitionKey = new PartitionKey(partitionSpec, schema); this.flinkSchema = flinkSchema; + int bucketFieldId = BucketPartitionerUtil.getBucketFieldInfo(partitionSpec).f0; this.bucketFieldPosition = IntStream.range(0, partitionSpec.fields().size()) .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index edbde75ec95a..811adbad025c 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -33,20 +33,19 @@ class BucketPartitioner implements Partitioner { 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."; + "Invalid bucket ID %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."; + "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 + // number of buckets private final int[] currentBucketWriterOffset; BucketPartitioner(PartitionSpec partitionSpec) { Tuple2 bucketFieldInfo = - BucketPartitionerUtils.getBucketFieldInfo(partitionSpec); + BucketPartitionerUtil.getBucketFieldInfo(partitionSpec); this.maxNumBuckets = bucketFieldInfo.f1; this.currentBucketWriterOffset = new int[this.maxNumBuckets]; diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java similarity index 94% rename from flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java rename to flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java index cade5ac29de4..d2042b57be59 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java @@ -26,11 +26,11 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.transforms.PartitionSpecVisitor; -final class BucketPartitionerUtils { +final class BucketPartitionerUtil { static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = - "Expected 1 Bucket transform in the provided PartitionSpec, found: %s"; + "Unsupported partition spec: bucket partitioner expects 1 bucket partition, received: %s"; - private BucketPartitionerUtils() {} + private BucketPartitionerUtil() {} /** * Determines whether the PartitionSpec has one and only one Bucket definition @@ -53,7 +53,7 @@ static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) List> bucketFields = getBucketFields(partitionSpec); Preconditions.checkArgument( bucketFields.size() == 1, - BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, + BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, bucketFields.size()); return bucketFields.get(0); } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 5ce866e3887e..abaaa81b0f21 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -505,7 +505,7 @@ private DataStream distributeDataStream( + "and table is unpartitioned"); return input; } else { - if (BucketPartitionerUtils.hasOneBucketField(partitionSpec)) { + if (BucketPartitionerUtil.hasOneBucketField(partitionSpec)) { return input.partitionCustom( new BucketPartitioner(partitionSpec), new BucketPartitionKeySelector(partitionSpec, iSchema, flinkRowType)); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java index fdb55773d64f..b2be5a0c1b64 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -33,7 +33,7 @@ public class TestBucketPartitionKeySelector { value = TableSchemaType.class, names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) public void testCorrectKeySelection(TableSchemaType tableSchemaType) { - final int numBuckets = 60; + int numBuckets = 60; PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); BucketPartitionKeySelector keySelector = @@ -60,6 +60,6 @@ public void testKeySelectorMultipleBucketsFail() { () -> new BucketPartitionKeySelector( partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE)) - .withMessage(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); + .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index 9841dd17e948..e02bcaae393a 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -37,9 +37,9 @@ public class TestBucketPartitioner { @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; - final TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); - final int numBuckets = Integer.parseInt(numBucketsStr); + int numPartitions = 500; + TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); + int numBuckets = Integer.parseInt(numBucketsStr); PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); @@ -58,9 +58,9 @@ public void testPartitioningParallelismGreaterThanBuckets( @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) public void testPartitioningParallelismEqualLessThanBuckets( String schemaTypeStr, String numBucketsStr) { - final int numPartitions = 30; - final TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); - final int numBuckets = Integer.parseInt(numBucketsStr); + int numPartitions = 30; + TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); + int numBuckets = Integer.parseInt(numBucketsStr); PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); @@ -88,7 +88,7 @@ public void testPartitionerMultipleBucketsFail() { Assertions.assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> new BucketPartitioner(partitionSpec)) - .withMessage(BucketPartitionerUtils.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); + .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); } @Test diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index 6df08677e3e8..7a0b022adcaa 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -24,10 +24,8 @@ 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; @@ -55,7 +53,6 @@ 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; @@ -91,7 +88,15 @@ public class TestBucketPartitionerFlinkIcebergSink { private TableLoader tableLoader; private void setupEnvironment(TableSchemaType tableSchemaType) { - table = getTable(tableSchemaType); + PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); + table = + catalogExtension + .catalog() + .createTable( + TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitionSpec, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); env = StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) @@ -100,19 +105,7 @@ private void setupEnvironment(TableSchemaType tableSchemaType) { 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 allRows) throws Exception { + private void appendRowsToTable(List allRows) throws Exception { DataFormatConverters.RowConverter converter = new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); @@ -130,10 +123,8 @@ private void testWriteRowData(List allRows) throws Exception { .distributionMode(DistributionMode.HASH) .append(); - // Execute the program. env.execute("Test Iceberg DataStream"); - // Assert the iceberg table's records. SimpleDataUtil.assertTableRows(table, allRows); } @@ -145,7 +136,7 @@ public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) t setupEnvironment(tableSchemaType); List rows = generateTestDataRows(); - testWriteRowData(rows); + appendRowsToTable(rows); TableTestStats stats = extractPartitionResults(tableSchemaType); Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); @@ -158,11 +149,11 @@ public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) t // - 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)); + Assertions.assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); // 2 files per bucket (one file is created by each writer) - Assertions.assertThat((int) stats.numFilesPerBucket.get(i)).isEqualTo(2); + Assertions.assertThat(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); + Assertions.assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); } } @@ -176,7 +167,7 @@ public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) throws setupEnvironment(tableSchemaType); List rows = generateTestDataRows(); - testWriteRowData(rows); + appendRowsToTable(rows); TableTestStats stats = extractPartitionResults(tableSchemaType); Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); @@ -190,7 +181,6 @@ public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) throws * Generating 16 rows to be sent uniformly to all writers (round-robin across 8 writers -> 4 * buckets) */ - @NotNull private List generateTestDataRows() { int totalNumRows = parallelism * 2; int numRowsPerBucket = totalNumRows / numBuckets; @@ -202,13 +192,15 @@ private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) int totalRecordCount = 0; Map> writersPerBucket = Maps.newHashMap(); // > Map filesPerBucket = Maps.newHashMap(); // - Map recordsPerFile = new TreeMap<>(); // + Map rowsPerWriter = Maps.newHashMap(); // try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { for (FileScanTask scanTask : fileScanTasks) { long recordCountInFile = scanTask.file().recordCount(); String[] splitFilePath = scanTask.file().path().toString().split("/"); + // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet + // Writer ID: .......^^^^^ String filename = splitFilePath[splitFilePath.length - 1]; int writerId = Integer.parseInt(filename.split("-")[0]); @@ -221,15 +213,11 @@ private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) 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); + rowsPerWriter.put(writerId, rowsPerWriter.getOrDefault(writerId, 0L) + recordCountInFile); } } - for (int k : writersPerBucket.keySet()) { - Collections.sort(writersPerBucket.get(k)); - } - - return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, recordsPerFile); + return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, rowsPerWriter); } /** DTO to hold Test Stats */ @@ -237,17 +225,17 @@ private static class TableTestStats { final int totalRowCount; final Map> writersPerBucket; final Map numFilesPerBucket; - final Map rowsPerFile; + final Map rowsPerWriter; TableTestStats( int totalRecordCount, Map> writersPerBucket, Map numFilesPerBucket, - Map rowsPerFile) { + Map rowsPerWriter) { this.totalRowCount = totalRecordCount; this.writersPerBucket = writersPerBucket; this.numFilesPerBucket = numFilesPerBucket; - this.rowsPerFile = rowsPerFile; + this.rowsPerWriter = rowsPerWriter; } } } From a488d129620ab4799265290b3d40f39e9a4f476e Mon Sep 17 00:00:00 2001 From: schongloo Date: Tue, 30 May 2023 16:15:16 -0700 Subject: [PATCH 20/23] Refactoring enum static methods --- .../sink/TestBucketPartitionKeySelector.java | 10 +-- .../flink/sink/TestBucketPartitioner.java | 15 ++-- ...TestBucketPartitionerFlinkIcebergSink.java | 8 +-- ...ls.java => TestBucketPartitionerUtil.java} | 68 +++++++++---------- 4 files changed, 48 insertions(+), 53 deletions(-) rename flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/{TestBucketPartitionerUtils.java => TestBucketPartitionerUtil.java} (69%) diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java index b2be5a0c1b64..5ebcc6361c7b 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -20,7 +20,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.sink.TestBucketPartitionerUtils.TableSchemaType; +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; @@ -35,16 +35,16 @@ public class TestBucketPartitionKeySelector { public void testCorrectKeySelection(TableSchemaType tableSchemaType) { int numBuckets = 60; - PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); BucketPartitionKeySelector keySelector = new BucketPartitionKeySelector( partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE); - TestBucketPartitionerUtils.generateRowsForBucketIdRange(2, numBuckets) + TestBucketPartitionerUtil.generateRowsForBucketIdRange(2, numBuckets) .forEach( rowData -> { int expectedBucketId = - TestBucketPartitionerUtils.computeBucketId( + TestBucketPartitionerUtil.computeBucketId( numBuckets, rowData.getString(1).toString()); Integer key = keySelector.getKey(rowData); Assertions.assertThat(key).isEqualTo(expectedBucketId); @@ -53,7 +53,7 @@ public void testCorrectKeySelection(TableSchemaType tableSchemaType) { @Test public void testKeySelectorMultipleBucketsFail() { - PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(TableSchemaType.TWO_BUCKETS, 1); + PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); Assertions.assertThatExceptionOfType(RuntimeException.class) .isThrownBy( diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index e02bcaae393a..432ea1e0b1b7 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -23,7 +23,7 @@ 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.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -40,7 +40,7 @@ public void testPartitioningParallelismGreaterThanBuckets( int numPartitions = 500; TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); int bucketId = 0; @@ -61,7 +61,7 @@ public void testPartitioningParallelismEqualLessThanBuckets( int numPartitions = 30; TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); for (int bucketId = 0; bucketId < numBuckets; bucketId++) { @@ -72,8 +72,7 @@ public void testPartitioningParallelismEqualLessThanBuckets( @Test public void testPartitionerBucketIdNullFail() { - PartitionSpec partitionSpec = - TableSchemaType.getPartitionSpec(TableSchemaType.ONE_BUCKET, DEFAULT_NUM_BUCKETS); + PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); Assertions.assertThatExceptionOfType(RuntimeException.class) @@ -83,8 +82,7 @@ public void testPartitionerBucketIdNullFail() { @Test public void testPartitionerMultipleBucketsFail() { - PartitionSpec partitionSpec = - TableSchemaType.getPartitionSpec(TableSchemaType.TWO_BUCKETS, DEFAULT_NUM_BUCKETS); + PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); Assertions.assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> new BucketPartitioner(partitionSpec)) @@ -93,8 +91,7 @@ public void testPartitionerMultipleBucketsFail() { @Test public void testPartitionerBucketIdOutOfRangeFail() { - PartitionSpec partitionSpec = - TableSchemaType.getPartitionSpec(TableSchemaType.ONE_BUCKET, DEFAULT_NUM_BUCKETS); + PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); int negativeBucketId = -1; diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index 7a0b022adcaa..29a0898a1b76 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -45,7 +45,7 @@ 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.sink.TestBucketPartitionerUtil.TableSchemaType; import org.apache.iceberg.flink.source.BoundedTestSource; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.io.CloseableIterable; @@ -88,7 +88,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private TableLoader tableLoader; private void setupEnvironment(TableSchemaType tableSchemaType) { - PartitionSpec partitionSpec = TableSchemaType.getPartitionSpec(tableSchemaType, numBuckets); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); table = catalogExtension .catalog() @@ -184,7 +184,7 @@ public void testMultipleBucketsFallback(TableSchemaType tableSchemaType) throws private List generateTestDataRows() { int totalNumRows = parallelism * 2; int numRowsPerBucket = totalNumRows / numBuckets; - return TestBucketPartitionerUtils.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); + return TestBucketPartitionerUtil.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); } private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) @@ -209,7 +209,7 @@ private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) scanTask .file() .partition() - .get(TableSchemaType.bucketPartitionColumnPosition(tableSchemaType), Integer.class); + .get(tableSchemaType.bucketPartitionColumnPosition(), Integer.class); writersPerBucket.computeIfAbsent(bucketId, k -> Lists.newArrayList()); writersPerBucket.get(bucketId).add(writerId); filesPerBucket.put(bucketId, filesPerBucket.getOrDefault(bucketId, 0) + 1); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java similarity index 69% rename from flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java rename to flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java index 910cf0c0834f..30252aba834c 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtils.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java @@ -25,52 +25,50 @@ 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 { +final class TestBucketPartitionerUtil { enum TableSchemaType { - ONE_BUCKET, - IDENTITY_AND_BUCKET, - TWO_BUCKETS; - - public static int bucketPartitionColumnPosition(TableSchemaType tableSchemaType) { - return tableSchemaType == ONE_BUCKET ? 0 : 1; - } - - public static PartitionSpec getPartitionSpec(TableSchemaType tableSchemaType, int numBuckets) { - PartitionSpec partitionSpec = null; + ONE_BUCKET { + @Override + public int bucketPartitionColumnPosition() { + return 0; + } - switch (tableSchemaType) { - case ONE_BUCKET: - partitionSpec = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); - break; - case IDENTITY_AND_BUCKET: - partitionSpec = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .identity("id") - .bucket("data", numBuckets) - .build(); - break; - case TWO_BUCKETS: - partitionSpec = - PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .bucket("id", numBuckets) - .bucket("data", numBuckets) - .build(); - break; + @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(); + } + }; - Preconditions.checkNotNull( - partitionSpec, "Invalid tableSchemaType provided: " + tableSchemaType); - return partitionSpec; + public int bucketPartitionColumnPosition() { + return 1; } + + public abstract PartitionSpec getPartitionSpec(int numBuckets); } - private TestBucketPartitionerUtils() {} + private TestBucketPartitionerUtil() {} /** * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to From dc4b6f479b570078ce0a2561f403225f7dff449d Mon Sep 17 00:00:00 2001 From: schongloo Date: Thu, 13 Jul 2023 09:44:26 -0700 Subject: [PATCH 21/23] Latest round of PR comments --- .../flink/sink/BucketPartitionKeySelector.java | 15 +++++++++------ .../iceberg/flink/sink/BucketPartitioner.java | 8 ++------ .../iceberg/flink/sink/BucketPartitionerUtil.java | 10 +++++++++- .../iceberg/flink/HadoopCatalogExtension.java | 2 +- .../iceberg/flink/sink/TestBucketPartitioner.java | 4 ++-- 5 files changed, 23 insertions(+), 16 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index ca8e727bfe91..acd985291bc9 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -44,11 +44,14 @@ class BucketPartitionKeySelector implements KeySelector { this.schema = schema; this.partitionKey = new PartitionKey(partitionSpec, schema); this.flinkSchema = flinkSchema; - int bucketFieldId = BucketPartitionerUtil.getBucketFieldInfo(partitionSpec).f0; - this.bucketFieldPosition = - IntStream.range(0, partitionSpec.fields().size()) - .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) - .toArray()[0]; + this.bucketFieldPosition = getBucketFieldPosition(partitionSpec); + } + + private int getBucketFieldPosition(PartitionSpec partitionSpec) { + int bucketFieldId = BucketPartitionerUtil.getBucketFieldId(partitionSpec); + return IntStream.range(0, partitionSpec.fields().size()) + .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) + .toArray()[0]; } private RowDataWrapper lazyRowDataWrapper() { @@ -61,6 +64,6 @@ private RowDataWrapper lazyRowDataWrapper() { @Override public Integer getKey(RowData rowData) { partitionKey.partition(lazyRowDataWrapper().wrap(rowData)); - return partitionKey.get(this.bucketFieldPosition, Integer.class); + return partitionKey.get(bucketFieldPosition, Integer.class); } } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java index 811adbad025c..9c9a117906e2 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java @@ -19,7 +19,6 @@ 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; @@ -44,11 +43,8 @@ class BucketPartitioner implements Partitioner { private final int[] currentBucketWriterOffset; BucketPartitioner(PartitionSpec partitionSpec) { - Tuple2 bucketFieldInfo = - BucketPartitionerUtil.getBucketFieldInfo(partitionSpec); - - this.maxNumBuckets = bucketFieldInfo.f1; - this.currentBucketWriterOffset = new int[this.maxNumBuckets]; + this.maxNumBuckets = BucketPartitionerUtil.getMaxNumBuckets(partitionSpec); + this.currentBucketWriterOffset = new int[maxNumBuckets]; } /** diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java index d2042b57be59..b87633985fd4 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java @@ -49,7 +49,7 @@ static boolean hasOneBucketField(PartitionSpec partitionSpec) { * @param partitionSpec the partition spec in question * @return the Bucket definition in the form of a tuple (fieldId, maxNumBuckets) */ - static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { + private static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { List> bucketFields = getBucketFields(partitionSpec); Preconditions.checkArgument( bucketFields.size() == 1, @@ -58,6 +58,14 @@ static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) return bucketFields.get(0); } + static int getBucketFieldId(PartitionSpec partitionSpec) { + return getBucketFieldInfo(partitionSpec).f0; + } + + static int getMaxNumBuckets(PartitionSpec partitionSpec) { + return getBucketFieldInfo(partitionSpec).f1; + } + private static List> getBucketFields(PartitionSpec spec) { return PartitionSpecVisitor.visit(spec, new BucketPartitionSpecVisitor()).stream() .filter(Objects::nonNull) diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java index 6cbe1652bf01..d8e1325254d9 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -58,7 +58,7 @@ public void beforeAll(ExtensionContext context) throws Exception { @Override public void afterAll(ExtensionContext context) throws Exception { - FileUtils.deleteDirectory(this.temporaryFolder.toFile()); + FileUtils.deleteDirectory(temporaryFolder.toFile()); } @Override diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index 432ea1e0b1b7..835713e6b417 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -44,9 +44,9 @@ public void testPartitioningParallelismGreaterThanBuckets( BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); int bucketId = 0; - for (int expectedIdx = 0; expectedIdx < numPartitions; expectedIdx++) { + for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIdx); + Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIndex); bucketId++; if (bucketId == numBuckets) { bucketId = 0; From 0cb7f371bafbc560ec454f03af1bd8709e7f6940 Mon Sep 17 00:00:00 2001 From: schongloo Date: Fri, 14 Jul 2023 13:43:38 -0700 Subject: [PATCH 22/23] Latest round of PR comments --- .../iceberg/flink/sink/BucketPartitionerUtil.java | 2 +- .../flink/sink/TestBucketPartitionerUtil.java | 14 +++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java index b87633985fd4..c33207728d3e 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java @@ -28,7 +28,7 @@ final class BucketPartitionerUtil { static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = - "Unsupported partition spec: bucket partitioner expects 1 bucket partition, received: %s"; + "Invalid number of buckets: %s (must be 1)"; private BucketPartitionerUtil() {} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java index 30252aba834c..e1309bfac6d5 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java @@ -43,6 +43,11 @@ public PartitionSpec getPartitionSpec(int numBuckets) { } }, IDENTITY_AND_BUCKET { + @Override + public int bucketPartitionColumnPosition() { + return 1; + } + @Override public PartitionSpec getPartitionSpec(int numBuckets) { return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) @@ -52,6 +57,11 @@ public PartitionSpec getPartitionSpec(int numBuckets) { } }, TWO_BUCKETS { + @Override + public int bucketPartitionColumnPosition() { + return 1; + } + @Override public PartitionSpec getPartitionSpec(int numBuckets) { return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) @@ -61,9 +71,7 @@ public PartitionSpec getPartitionSpec(int numBuckets) { } }; - public int bucketPartitionColumnPosition() { - return 1; - } + public abstract int bucketPartitionColumnPosition(); public abstract PartitionSpec getPartitionSpec(int numBuckets); } From c42ce135d577ae1dd0c316619cba6728666b58d5 Mon Sep 17 00:00:00 2001 From: schongloo Date: Wed, 9 Aug 2023 15:12:07 -0700 Subject: [PATCH 23/23] Last PR comment --- .../apache/iceberg/flink/sink/BucketPartitionKeySelector.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java index acd985291bc9..1cb6e013bd2c 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -58,6 +58,7 @@ private RowDataWrapper lazyRowDataWrapper() { if (rowDataWrapper == null) { rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); } + return rowDataWrapper; }