From 6d6fd0b5e967327967142758c999483ae50f95c5 Mon Sep 17 00:00:00 2001 From: Abid Mohammed Date: Fri, 29 Mar 2024 10:32:01 -0700 Subject: [PATCH 01/11] [core] fix #9997 - Handle s3a file upload interrupt which results in table metadata pointing to files that doesn't exist (#9998) Co-authored-by: Abid Mohammed --- .../apache/iceberg/hadoop/HadoopStreams.java | 11 +++ .../hadoop/fs/s3a/S3ABlockOutputStream.java | 67 +++++++++++++++++++ .../iceberg/hadoop/TestHadoopStreams.java | 53 +++++++++++++++ 3 files changed, 131 insertions(+) create mode 100644 core/src/test/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java create mode 100644 core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java index 44023326a0cf..f9b43b684666 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java @@ -187,6 +187,17 @@ public void flush() throws IOException { public void close() throws IOException { stream.close(); this.closed = true; + // {@link org.apache.hadoop.fs.s3a.S3ABlockOutputStream#close()} calls {@link + // org.apache.hadoop.fs.s3a.S3ABlockOutputStream#putObject()} + // which doesn't throw an exception when interrupted. + // Need to check the interrupted flag to detect failed object upload + // and propagate the error up. + if (Thread.interrupted() + && "org.apache.hadoop.fs.s3a.S3ABlockOutputStream" + .equals(stream.getWrappedStream().getClass().getName())) { + throw new IOException( + "S3ABlockOutputStream failed to upload object after stream was closed"); + } } @SuppressWarnings("checkstyle:NoFinalizer") diff --git a/core/src/test/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/core/src/test/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java new file mode 100644 index 000000000000..63bddf7b79c2 --- /dev/null +++ b/core/src/test/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.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.hadoop.fs.s3a; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +/** mock class for testing hadoop s3a writer */ +public class S3ABlockOutputStream extends OutputStream { + public ExecutorService mockCloseService; + public Future mockUploadOnClose; + + public S3ABlockOutputStream() { + mockCloseService = Executors.newSingleThreadExecutor(); + } + + @Override + public void write(int b) throws IOException { + throw new IOException("mocked class, do not use"); + } + + @Override + public void close() throws IOException { + try { + mockUploadOnClose = + mockCloseService.submit( + () -> { + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + // ignore + } + }); + mockUploadOnClose.get(); + } catch (CancellationException | InterruptedException e) { + // mock interrupt in S3ABlockOutputStream#putObject + Thread.currentThread().interrupt(); + } catch (Exception e) { + throw new RuntimeException(e); + } + super.close(); + } + + public void interruptClose() { + mockUploadOnClose.cancel(true); + } +} diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java new file mode 100644 index 000000000000..09b478e4a6c1 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java @@ -0,0 +1,53 @@ +/* + * 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.hadoop; + +import java.io.IOException; +import java.util.concurrent.Executors; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.s3a.S3ABlockOutputStream; +import org.apache.iceberg.io.PositionOutputStream; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +class TestHadoopStreams { + + @Test + void closeShouldThrowIOExceptionWhenInterrupted() throws Exception { + + S3ABlockOutputStream s3ABlockOutputStream = new S3ABlockOutputStream(); + FSDataOutputStream fsDataOutputStream = new FSDataOutputStream(s3ABlockOutputStream, null); + PositionOutputStream wrap = HadoopStreams.wrap(fsDataOutputStream); + // interrupt mock upload on close after a delay + Executors.newSingleThreadExecutor() + .execute( + () -> { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + s3ABlockOutputStream.interruptClose(); + }); + + Assertions.assertThatThrownBy(wrap::close) + .isInstanceOf(IOException.class) + .hasMessage("S3ABlockOutputStream failed to upload object after stream was closed"); + } +} From 793c8d05cee9e4a95ffe2b94e31bef5a617e8c85 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sat, 30 Mar 2024 09:07:45 +0100 Subject: [PATCH 02/11] Spark: Clarify schema behavior when working with branches (#10055) --- docs/docs/branching.md | 91 +++++++++++++++++++ docs/docs/spark-queries.md | 23 ++++- docs/docs/spark-writes.md | 4 + .../apache/iceberg/spark/sql/TestSelect.java | 59 ++++++++++++ 4 files changed, 176 insertions(+), 1 deletion(-) diff --git a/docs/docs/branching.md b/docs/docs/branching.md index e944e4eb3bc2..3379264d8a5f 100644 --- a/docs/docs/branching.md +++ b/docs/docs/branching.md @@ -113,3 +113,94 @@ Creating, querying and writing to branches and tags are supported in the Iceberg - [Spark Branch Writes](spark-writes.md#writing-to-branches) - [Flink Reads](flink-queries.md#reading-branches-and-tags-with-SQL) - [Flink Branch Writes](flink-writes.md#branch-writes) + + +## Schema selection with branches and tags + +It is important to understand that the schema tracked for a table is valid across all branches. +When working with branches, the table's schema is used as that's the schema being validated when writing data to a branch. +On the other hands, querying a tag uses the snapshot's schema, which is the schema id that snapshot pointed to when the snapshot was created. + +The below examples show which schema is being used when working with branches. + +Create a table and insert some data: + +```sql +CREATE TABLE db.table (id bigint, data string, col float); +INSERT INTO db.table values (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', 3.0); +SELECT * FROM db.table; +1 a 1.0 +2 b 2.0 +3 c 3.0 +``` + +Create a branch `test_branch` that points to the current snapshot and read data from the branch: + +```sql +ALTER TABLE db.table CREATE BRANCH test_branch; + +SELECT * FROM db.table.branch_test_branch; +1 a 1.0 +2 b 2.0 +3 c 3.0 +``` + +Modify the table's schema by dropping the `col` column and adding a new column named `new_col`: + +```sql +ALTER TABLE db.table drop column float; + +ALTER TABLE db.table add column new_col date; + +INSERT INTO db.table values (4, 'd', date('2024-04-04')), (5, 'e', date('2024-05-05')); + +SELECT * FROM db.table; +1 a NULL +2 b NULL +3 c NULL +4 d 2024-04-04 +5 e 2024-05-05 +``` + +Querying the head of the branch using one of the below statements will return data using the **table's schema**: + +```sql +SELECT * FROM db.table.branch_test_branch; +1 a NULL +2 b NULL +3 c NULL + +SELECT * FROM db.table VERSION AS OF 'test_branch'; +1 a NULL +2 b NULL +3 c NULL +``` + +Performing a time travel query using the snapshot id uses the **snapshot's schema**: + +```sql + +SELECT * FROM db.table.refs; +test_branch BRANCH 8109744798576441359 NULL NULL NULL +main BRANCH 6910357365743665710 NULL NULL NULL + + +SELECT * FROM db.table VERSION AS OF 8109744798576441359; +1 a 1.0 +2 b 2.0 +3 c 3.0 +``` + +When writing to the branch, the **table's schema** is used for validation: + +```sql + +INSERT INTO db.table.branch_test_branch values (6, 'e', date('2024-06-06')), (7, 'g', date('2024-07-07')); + +SELECT * FROM db.table.branch_test_branch; +6 e 2024-06-06 +7 g 2024-07-07 +1 a NULL +2 b NULL +3 c NULL +``` diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index 536c136d7e55..b606d849a692 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -82,7 +82,7 @@ The `VERSION AS OF` clause can contain a long snapshot ID or a string branch or If this is not desired, rename the tag or branch with a well-defined prefix such as 'snapshot-1'. -```sql +```sql -- time travel to October 26, 1986 at 01:21:00 SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00'; @@ -124,6 +124,27 @@ SELECT * FROM prod.db.table.`tag_historical-snapshot`; Note that the identifier with branch or tag may not be used in combination with `VERSION AS OF`. + +#### Schema selection in time travel queries + +The different time travel queries mentioned in the previous section can use either the snapshot's schema or the table's schema: + +```sql +-- time travel to October 26, 1986 at 01:21:00 -> uses the snapshot's schema +SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00'; + +-- time travel to snapshot with id 10963874102873L -> uses the snapshot's schema +SELECT * FROM prod.db.table VERSION AS OF 10963874102873; + +-- time travel to the head of audit-branch -> uses the table's schema +SELECT * FROM prod.db.table VERSION AS OF 'audit-branch'; +SELECT * FROM prod.db.table.`branch_audit-branch`; + +-- time travel to the snapshot referenced by the tag historical-snapshot -> uses the snapshot's schema +SELECT * FROM prod.db.table VERSION AS OF 'historical-snapshot'; +SELECT * FROM prod.db.table.`tag_historical-snapshot`; +``` + #### DataFrame To select a specific table snapshot or the snapshot at some time in the DataFrame API, Iceberg supports four Spark read options: diff --git a/docs/docs/spark-writes.md b/docs/docs/spark-writes.md index 626dee6c96e6..96fcc5f7ce32 100644 --- a/docs/docs/spark-writes.md +++ b/docs/docs/spark-writes.md @@ -201,6 +201,10 @@ Note WAP branch and branch identifier cannot both be specified. Also, the branch must exist before performing the write. The operation does **not** create the branch if it does not exist. For more information on branches please refer to [branches](branching.md). + +!!! info + Note: When writing to a branch, the current schema of the table will be used for validation. + ```sql -- INSERT (1,' a') (2, 'b') into the audit branch. diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 80d7d8787e27..4c99a38d29fc 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.text.SimpleDateFormat; +import java.util.Arrays; import java.util.Date; import java.util.List; import java.util.concurrent.TimeUnit; @@ -348,6 +349,64 @@ public void testBranchReference() { assertEquals("Snapshot at specific branch reference name", expected, fromDF); } + @TestTemplate + public void readAndWriteWithBranchAfterSchemaChange() { + Table table = validationCatalog.loadTable(tableIdent); + String branchName = "test_branch"; + table.manageSnapshots().createBranch(branchName, table.currentSnapshot().snapshotId()).commit(); + + List expected = + Arrays.asList(row(1L, "a", 1.0f), row(2L, "b", 2.0f), row(3L, "c", Float.NaN)); + assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyElementsOf(expected); + + // change schema on the table and add more data + sql("ALTER TABLE %s DROP COLUMN float", tableName); + sql("ALTER TABLE %s ADD COLUMN new_col date", tableName); + sql( + "INSERT INTO %s VALUES (4, 'd', date('2024-04-04')), (5, 'e', date('2024-05-05'))", + tableName); + + // time-travel query using snapshot id should return the snapshot's schema + long branchSnapshotId = table.refs().get(branchName).snapshotId(); + assertThat(sql("SELECT * FROM %s VERSION AS OF %s", tableName, branchSnapshotId)) + .containsExactlyElementsOf(expected); + + // querying the head of the branch should return the table's schema + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); + + if (!"spark_catalog".equals(catalogName)) { + // querying the head of the branch using 'branch_' should return the table's schema + assertThat(sql("SELECT * FROM %s.branch_%s", tableName, branchName)) + .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); + } + + // writing to a branch uses the table's schema + sql( + "INSERT INTO %s.branch_%s VALUES (6L, 'f', cast('2023-06-06' as date)), (7L, 'g', cast('2023-07-07' as date))", + tableName, branchName); + + // querying the head of the branch returns the table's schema + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + .containsExactlyInAnyOrder( + row(1L, "a", null), + row(2L, "b", null), + row(3L, "c", null), + row(6L, "f", java.sql.Date.valueOf("2023-06-06")), + row(7L, "g", java.sql.Date.valueOf("2023-07-07"))); + + // using DataFrameReader with the 'branch' option should return the table's schema + Dataset df = + spark.read().format("iceberg").option(SparkReadOptions.BRANCH, branchName).load(tableName); + assertThat(rowsToJava(df.collectAsList())) + .containsExactlyInAnyOrder( + row(1L, "a", null), + row(2L, "b", null), + row(3L, "c", null), + row(6L, "f", java.sql.Date.valueOf("2023-06-06")), + row(7L, "g", java.sql.Date.valueOf("2023-07-07"))); + } + @TestTemplate public void testUnknownReferenceAsOf() { assertThatThrownBy(() -> sql("SELECT * FROM %s VERSION AS OF 'test_unknown'", tableName)) From a86e1b3bbd4101bd1ecb3cc8551590022f0211ac Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Sat, 30 Mar 2024 13:01:28 -0700 Subject: [PATCH 03/11] Flink: backport PR #9321 for range partitioner on map statistics (#10061) --- .../shuffle/MapRangePartitionerBenchmark.java | 199 ++++++++ .../sink/shuffle/MapRangePartitioner.java | 381 +++++++++++++++ .../sink/shuffle/TestMapRangePartitioner.java | 448 ++++++++++++++++++ .../shuffle/MapRangePartitionerBenchmark.java | 199 ++++++++ .../sink/shuffle/MapRangePartitioner.java | 381 +++++++++++++++ .../sink/shuffle/TestMapRangePartitioner.java | 448 ++++++++++++++++++ jmh.gradle | 8 + 7 files changed, 2064 insertions(+) create mode 100644 flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java create mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java create mode 100644 flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java diff --git a/flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java new file mode 100644 index 000000000000..c3917165753d --- /dev/null +++ b/flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -0,0 +1,199 @@ +/* + * 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.shuffle; + +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class MapRangePartitionerBenchmark { + private static final String CHARS = + "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; + private static final int SAMPLE_SIZE = 100_000; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + + private MapRangePartitioner partitioner; + private RowData[] rows; + + @Setup + public void setupBenchmark() { + NavigableMap weights = longTailDistribution(100_000, 24, 240, 100, 2.0); + Map mapStatistics = Maps.newHashMapWithExpectedSize(weights.size()); + weights.forEach( + (id, weight) -> { + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, id); + mapStatistics.put(sortKey, weight); + }); + + MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + this.partitioner = + new MapRangePartitioner( + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + + List keys = Lists.newArrayList(weights.keySet().iterator()); + long[] weightsCDF = new long[keys.size()]; + long totalWeight = 0; + for (int i = 0; i < keys.size(); ++i) { + totalWeight += weights.get(keys.get(i)); + weightsCDF[i] = totalWeight; + } + + // pre-calculate the samples for benchmark run + this.rows = new GenericRowData[SAMPLE_SIZE]; + for (int i = 0; i < SAMPLE_SIZE; ++i) { + long weight = ThreadLocalRandom.current().nextLong(totalWeight); + int index = binarySearchIndex(weightsCDF, weight); + rows[i] = + GenericRowData.of( + keys.get(index), + randomString("name2-"), + randomString("name3-"), + randomString("name4-"), + randomString("name5-"), + randomString("name6-"), + randomString("name7-"), + randomString("name8-"), + randomString("name9-")); + } + } + + @TearDown + public void tearDownBenchmark() {} + + @Benchmark + @Threads(1) + public void testPartitionerLongTailDistribution(Blackhole blackhole) { + for (int i = 0; i < SAMPLE_SIZE; ++i) { + blackhole.consume(partitioner.partition(rows[i], 128)); + } + } + + private static String randomString(String prefix) { + int length = ThreadLocalRandom.current().nextInt(200); + byte[] buffer = new byte[length]; + + for (int i = 0; i < length; i += 1) { + buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); + } + + return prefix + new String(buffer); + } + + /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ + private static int binarySearchIndex(long[] weightsUDF, long target) { + Preconditions.checkArgument( + target < weightsUDF[weightsUDF.length - 1], + "weight is out of range: total weight = %s, search target = %s", + weightsUDF[weightsUDF.length - 1], + target); + int start = 0; + int end = weightsUDF.length - 1; + while (start < end) { + int mid = (start + end) / 2; + if (weightsUDF[mid] < target && weightsUDF[mid + 1] >= target) { + return mid; + } + + if (weightsUDF[mid] >= target) { + end = mid - 1; + } else if (weightsUDF[mid + 1] < target) { + start = mid + 1; + } + } + return start; + } + + /** Key is the id string and value is the weight in long value. */ + private static NavigableMap longTailDistribution( + long startingWeight, + int longTailStartingIndex, + int longTailLength, + long longTailBaseWeight, + double weightRandomJitterPercentage) { + + NavigableMap weights = Maps.newTreeMap(); + + // first part just decays the weight by half + long currentWeight = startingWeight; + for (int index = 0; index < longTailStartingIndex; ++index) { + double jitter = ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage / 100); + long weight = (long) (currentWeight * (1.0 + jitter)); + weight = weight > 0 ? weight : 1; + weights.put(index, weight); + if (currentWeight > longTailBaseWeight) { + currentWeight = currentWeight / 2; + } + } + + // long tail part + for (int index = longTailStartingIndex; + index < longTailStartingIndex + longTailLength; + ++index) { + long longTailWeight = + (long) + (longTailBaseWeight + * ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage)); + longTailWeight = longTailWeight > 0 ? longTailWeight : 1; + weights.put(index, longTailWeight); + } + + return weights; + } +} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java new file mode 100644 index 000000000000..fb1a8f03a65c --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -0,0 +1,381 @@ +/* + * 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.shuffle; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Internal partitioner implementation that supports MapDataStatistics, which is typically used for + * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used + * for high-cardinality use cases. Otherwise, the memory footprint is too high. + * + *

It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always + * precise when calculating close cost for every file, target weight per subtask, padding residual + * weight, assigned weight without close cost. + * + *

All actions should be executed in a single Flink mailbox thread. So there is no need to make + * it thread safe. + */ +class MapRangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); + + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final Comparator comparator; + private final Map mapStatistics; + private final double closeFileCostInWeightPercentage; + + // Counter that tracks how many times a new key encountered + // where there is no traffic statistics learned about it. + private long newSortKeyCounter; + private long lastNewSortKeyLogTimeMilli; + + // lazily computed due to the need of numPartitions + private Map assignment; + private NavigableMap sortedStatsWithCloseFileCost; + + MapRangePartitioner( + Schema schema, + SortOrder sortOrder, + MapDataStatistics dataStatistics, + double closeFileCostInWeightPercentage) { + dataStatistics + .statistics() + .entrySet() + .forEach( + entry -> + Preconditions.checkArgument( + entry.getValue() > 0, + "Invalid statistics: weight is 0 for key %s", + entry.getKey())); + + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.mapStatistics = dataStatistics.statistics(); + this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.newSortKeyCounter = 0; + this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); + } + + @Override + public int partition(RowData row, int numPartitions) { + // assignment table can only be built lazily when first referenced here, + // because number of partitions (downstream subtasks) is needed. + // the numPartitions is not available in the constructor. + Map assignmentMap = assignment(numPartitions); + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + KeyAssignment keyAssignment = assignmentMap.get(sortKey); + if (keyAssignment == null) { + LOG.trace( + "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", + sortKey); + // Ideally unknownKeyCounter should be published as a counter metric. + // It seems difficult to pass in MetricGroup into the partitioner. + // Just log an INFO message every minute. + newSortKeyCounter += 1; + long now = System.currentTimeMillis(); + if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { + LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + lastNewSortKeyLogTimeMilli = now; + } + return (int) (newSortKeyCounter % numPartitions); + } + + return keyAssignment.select(); + } + + @VisibleForTesting + Map assignment(int numPartitions) { + if (assignment == null) { + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostInWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); + + this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + this.assignment = + buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostInWeight); + } + + return assignment; + } + + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + assignment.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { + int subtaskId = keyAssignment.assignedSubtasks[i]; + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + private Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostInWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostInWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostInWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks.clear(); + subtaskWeights.clear(); + currentKey = null; + } + } + + return assignmentMap; + } + + /** Subtask assignment for a key */ + @VisibleForTesting + static class KeyAssignment { + private final int[] assignedSubtasks; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It + * could also be multiple subtasks if the key has heavy weight that should be handled by + * multiple subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the + * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain + * values as [10, 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostInWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostInWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostInWeight)); + + this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + /** @return subtask id */ + int select() { + if (assignedSubtasks.length == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks[0]; + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.length, + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks[position]; + } + } + + @Override + public int hashCode() { + return 31 * Arrays.hashCode(assignedSubtasks) + + Arrays.hashCode(subtaskWeightsExcludingCloseCost); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Arrays.equals(assignedSubtasks, that.assignedSubtasks) + && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) + .toString(); + } + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java new file mode 100644 index 000000000000..92eb71acc834 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -0,0 +1,448 @@ +/* + * 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.shuffle; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapRangePartitioner { + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final SortKey[] SORT_KEYS = initSortKeys(); + + private static SortKey[] initSortKeys() { + SortKey[] sortKeys = new SortKey[10]; + for (int i = 0; i < 10; ++i) { + RowData rowData = + GenericRowData.of(StringData.fromString("k" + i), i, StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(keyWrapper); + sortKeys[i] = sortKey; + } + return sortKeys; + } + + // Total weight is 800 + private final MapDataStatistics mapDataStatistics = + new MapDataStatistics( + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L)); + + @Test + public void testEvenlyDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 8; + + // each task should get targeted weight of 100 (=800/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(100L, 1), + 1, + Pair.of(100L, 1), + 2, + Pair.of(100L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(100L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 1), + 7, + Pair.of(100L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testEvenlyDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 8; + + // target subtask weight is 100 before close file cost factored in. + // close file cost is 5 = 5% * 100. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight with close cost per subtask is 110 (880/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight (excluding close file cost) for the subtask, + // second is the number of keys assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(105L, 1), + 1, + Pair.of(105L, 1), + 2, + Pair.of(105L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(105L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 2), + 7, + Pair.of(75L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 9; + + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // each task should get targeted weight of 89 = ceiling(800/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(89L, 1), + 1, + Pair.of(89L, 1), + 2, + Pair.of(89L, 1), + 3, + Pair.of(89L, 2), + 4, + Pair.of(89L, 1), + 5, + Pair.of(89L, 1), + 6, + Pair.of(89L, 2), + 7, + Pair.of(89L, 2), + 8, + Pair.of(88L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 9; + + // target subtask weight is 89 before close file cost factored in. + // close file cost is 5 (= 5% * 89) per file. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight per subtask is 98 ceiling(880/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight for the subtask, second is the number of keys + // assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(93L, 1), + 1, + Pair.of(93L, 1), + 2, + Pair.of(93L, 1), + 3, + Pair.of(88L, 2), + 4, + Pair.of(93L, 1), + 5, + Pair.of(93L, 1), + 6, + Pair.of(88L, 2), + 7, + Pair.of(88L, 2), + 8, + Pair.of(61L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + // drift threshold is high for non-dividable scenario with close cost + validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); + } + + private static Map>> runPartitioner( + MapRangePartitioner partitioner, int numPartitions) { + // The Map key is the subtaskId. + // For the map value pair, the first element is the count of assigned and + // the second element of Set is for the set of assigned keys. + Map>> partitionResults = Maps.newHashMap(); + partitioner + .mapStatistics() + .forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); + return partitionResults; + } + + /** @param expectedAssignmentInfo excluding closing cost */ + private void validatePartitionResults( + Map> expectedAssignmentInfo, + Map>> partitionResults, + double maxDriftPercentage) { + + Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + + List expectedAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + List expectedNormalizedWeights = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualNormalizedWeights = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + + long expectedTotalWeight = + expectedAssignmentInfo.values().stream().mapToLong(Pair::first).sum(); + expectedAssignmentInfo.forEach( + (subtaskId, pair) -> { + expectedAssignedKeyCounts.add(pair.second()); + expectedNormalizedWeights.add(pair.first().doubleValue() / expectedTotalWeight); + }); + + long actualTotalWeight = + partitionResults.values().stream().mapToLong(pair -> pair.first().longValue()).sum(); + partitionResults.forEach( + (subtaskId, pair) -> { + actualAssignedKeyCounts.add(pair.second().size()); + actualNormalizedWeights.add(pair.first().doubleValue() / actualTotalWeight); + }); + + // number of assigned keys should match exactly + Assertions.assertThat(actualAssignedKeyCounts) + .as("the number of assigned keys should match for every subtask") + .isEqualTo(expectedAssignedKeyCounts); + + // weight for every subtask shouldn't differ for more than some threshold relative to the + // expected weight + for (int subtaskId = 0; subtaskId < expectedNormalizedWeights.size(); ++subtaskId) { + double expectedWeight = expectedNormalizedWeights.get(subtaskId); + double min = expectedWeight * (1 - maxDriftPercentage / 100); + double max = expectedWeight * (1 + maxDriftPercentage / 100); + Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) + .as( + "Subtask %d weight should within %.1f percent of the expected range %s", + subtaskId, maxDriftPercentage, expectedWeight) + .isBetween(min, max); + } + } +} diff --git a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java new file mode 100644 index 000000000000..c3917165753d --- /dev/null +++ b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -0,0 +1,199 @@ +/* + * 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.shuffle; + +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class MapRangePartitionerBenchmark { + private static final String CHARS = + "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; + private static final int SAMPLE_SIZE = 100_000; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + + private MapRangePartitioner partitioner; + private RowData[] rows; + + @Setup + public void setupBenchmark() { + NavigableMap weights = longTailDistribution(100_000, 24, 240, 100, 2.0); + Map mapStatistics = Maps.newHashMapWithExpectedSize(weights.size()); + weights.forEach( + (id, weight) -> { + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, id); + mapStatistics.put(sortKey, weight); + }); + + MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + this.partitioner = + new MapRangePartitioner( + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + + List keys = Lists.newArrayList(weights.keySet().iterator()); + long[] weightsCDF = new long[keys.size()]; + long totalWeight = 0; + for (int i = 0; i < keys.size(); ++i) { + totalWeight += weights.get(keys.get(i)); + weightsCDF[i] = totalWeight; + } + + // pre-calculate the samples for benchmark run + this.rows = new GenericRowData[SAMPLE_SIZE]; + for (int i = 0; i < SAMPLE_SIZE; ++i) { + long weight = ThreadLocalRandom.current().nextLong(totalWeight); + int index = binarySearchIndex(weightsCDF, weight); + rows[i] = + GenericRowData.of( + keys.get(index), + randomString("name2-"), + randomString("name3-"), + randomString("name4-"), + randomString("name5-"), + randomString("name6-"), + randomString("name7-"), + randomString("name8-"), + randomString("name9-")); + } + } + + @TearDown + public void tearDownBenchmark() {} + + @Benchmark + @Threads(1) + public void testPartitionerLongTailDistribution(Blackhole blackhole) { + for (int i = 0; i < SAMPLE_SIZE; ++i) { + blackhole.consume(partitioner.partition(rows[i], 128)); + } + } + + private static String randomString(String prefix) { + int length = ThreadLocalRandom.current().nextInt(200); + byte[] buffer = new byte[length]; + + for (int i = 0; i < length; i += 1) { + buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); + } + + return prefix + new String(buffer); + } + + /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ + private static int binarySearchIndex(long[] weightsUDF, long target) { + Preconditions.checkArgument( + target < weightsUDF[weightsUDF.length - 1], + "weight is out of range: total weight = %s, search target = %s", + weightsUDF[weightsUDF.length - 1], + target); + int start = 0; + int end = weightsUDF.length - 1; + while (start < end) { + int mid = (start + end) / 2; + if (weightsUDF[mid] < target && weightsUDF[mid + 1] >= target) { + return mid; + } + + if (weightsUDF[mid] >= target) { + end = mid - 1; + } else if (weightsUDF[mid + 1] < target) { + start = mid + 1; + } + } + return start; + } + + /** Key is the id string and value is the weight in long value. */ + private static NavigableMap longTailDistribution( + long startingWeight, + int longTailStartingIndex, + int longTailLength, + long longTailBaseWeight, + double weightRandomJitterPercentage) { + + NavigableMap weights = Maps.newTreeMap(); + + // first part just decays the weight by half + long currentWeight = startingWeight; + for (int index = 0; index < longTailStartingIndex; ++index) { + double jitter = ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage / 100); + long weight = (long) (currentWeight * (1.0 + jitter)); + weight = weight > 0 ? weight : 1; + weights.put(index, weight); + if (currentWeight > longTailBaseWeight) { + currentWeight = currentWeight / 2; + } + } + + // long tail part + for (int index = longTailStartingIndex; + index < longTailStartingIndex + longTailLength; + ++index) { + long longTailWeight = + (long) + (longTailBaseWeight + * ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage)); + longTailWeight = longTailWeight > 0 ? longTailWeight : 1; + weights.put(index, longTailWeight); + } + + return weights; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java new file mode 100644 index 000000000000..fb1a8f03a65c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -0,0 +1,381 @@ +/* + * 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.shuffle; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Internal partitioner implementation that supports MapDataStatistics, which is typically used for + * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used + * for high-cardinality use cases. Otherwise, the memory footprint is too high. + * + *

It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always + * precise when calculating close cost for every file, target weight per subtask, padding residual + * weight, assigned weight without close cost. + * + *

All actions should be executed in a single Flink mailbox thread. So there is no need to make + * it thread safe. + */ +class MapRangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); + + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final Comparator comparator; + private final Map mapStatistics; + private final double closeFileCostInWeightPercentage; + + // Counter that tracks how many times a new key encountered + // where there is no traffic statistics learned about it. + private long newSortKeyCounter; + private long lastNewSortKeyLogTimeMilli; + + // lazily computed due to the need of numPartitions + private Map assignment; + private NavigableMap sortedStatsWithCloseFileCost; + + MapRangePartitioner( + Schema schema, + SortOrder sortOrder, + MapDataStatistics dataStatistics, + double closeFileCostInWeightPercentage) { + dataStatistics + .statistics() + .entrySet() + .forEach( + entry -> + Preconditions.checkArgument( + entry.getValue() > 0, + "Invalid statistics: weight is 0 for key %s", + entry.getKey())); + + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.mapStatistics = dataStatistics.statistics(); + this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.newSortKeyCounter = 0; + this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); + } + + @Override + public int partition(RowData row, int numPartitions) { + // assignment table can only be built lazily when first referenced here, + // because number of partitions (downstream subtasks) is needed. + // the numPartitions is not available in the constructor. + Map assignmentMap = assignment(numPartitions); + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + KeyAssignment keyAssignment = assignmentMap.get(sortKey); + if (keyAssignment == null) { + LOG.trace( + "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", + sortKey); + // Ideally unknownKeyCounter should be published as a counter metric. + // It seems difficult to pass in MetricGroup into the partitioner. + // Just log an INFO message every minute. + newSortKeyCounter += 1; + long now = System.currentTimeMillis(); + if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { + LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + lastNewSortKeyLogTimeMilli = now; + } + return (int) (newSortKeyCounter % numPartitions); + } + + return keyAssignment.select(); + } + + @VisibleForTesting + Map assignment(int numPartitions) { + if (assignment == null) { + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostInWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); + + this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + this.assignment = + buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostInWeight); + } + + return assignment; + } + + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + assignment.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { + int subtaskId = keyAssignment.assignedSubtasks[i]; + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + private Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostInWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostInWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostInWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks.clear(); + subtaskWeights.clear(); + currentKey = null; + } + } + + return assignmentMap; + } + + /** Subtask assignment for a key */ + @VisibleForTesting + static class KeyAssignment { + private final int[] assignedSubtasks; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It + * could also be multiple subtasks if the key has heavy weight that should be handled by + * multiple subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the + * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain + * values as [10, 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostInWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostInWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostInWeight)); + + this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + /** @return subtask id */ + int select() { + if (assignedSubtasks.length == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks[0]; + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.length, + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks[position]; + } + } + + @Override + public int hashCode() { + return 31 * Arrays.hashCode(assignedSubtasks) + + Arrays.hashCode(subtaskWeightsExcludingCloseCost); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Arrays.equals(assignedSubtasks, that.assignedSubtasks) + && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) + .toString(); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java new file mode 100644 index 000000000000..92eb71acc834 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -0,0 +1,448 @@ +/* + * 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.shuffle; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapRangePartitioner { + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final SortKey[] SORT_KEYS = initSortKeys(); + + private static SortKey[] initSortKeys() { + SortKey[] sortKeys = new SortKey[10]; + for (int i = 0; i < 10; ++i) { + RowData rowData = + GenericRowData.of(StringData.fromString("k" + i), i, StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(keyWrapper); + sortKeys[i] = sortKey; + } + return sortKeys; + } + + // Total weight is 800 + private final MapDataStatistics mapDataStatistics = + new MapDataStatistics( + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L)); + + @Test + public void testEvenlyDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 8; + + // each task should get targeted weight of 100 (=800/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(100L, 1), + 1, + Pair.of(100L, 1), + 2, + Pair.of(100L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(100L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 1), + 7, + Pair.of(100L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testEvenlyDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 8; + + // target subtask weight is 100 before close file cost factored in. + // close file cost is 5 = 5% * 100. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight with close cost per subtask is 110 (880/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight (excluding close file cost) for the subtask, + // second is the number of keys assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(105L, 1), + 1, + Pair.of(105L, 1), + 2, + Pair.of(105L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(105L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 2), + 7, + Pair.of(75L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 9; + + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // each task should get targeted weight of 89 = ceiling(800/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(89L, 1), + 1, + Pair.of(89L, 1), + 2, + Pair.of(89L, 1), + 3, + Pair.of(89L, 2), + 4, + Pair.of(89L, 1), + 5, + Pair.of(89L, 1), + 6, + Pair.of(89L, 2), + 7, + Pair.of(89L, 2), + 8, + Pair.of(88L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 9; + + // target subtask weight is 89 before close file cost factored in. + // close file cost is 5 (= 5% * 89) per file. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight per subtask is 98 ceiling(880/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight for the subtask, second is the number of keys + // assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(93L, 1), + 1, + Pair.of(93L, 1), + 2, + Pair.of(93L, 1), + 3, + Pair.of(88L, 2), + 4, + Pair.of(93L, 1), + 5, + Pair.of(93L, 1), + 6, + Pair.of(88L, 2), + 7, + Pair.of(88L, 2), + 8, + Pair.of(61L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + // drift threshold is high for non-dividable scenario with close cost + validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); + } + + private static Map>> runPartitioner( + MapRangePartitioner partitioner, int numPartitions) { + // The Map key is the subtaskId. + // For the map value pair, the first element is the count of assigned and + // the second element of Set is for the set of assigned keys. + Map>> partitionResults = Maps.newHashMap(); + partitioner + .mapStatistics() + .forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); + return partitionResults; + } + + /** @param expectedAssignmentInfo excluding closing cost */ + private void validatePartitionResults( + Map> expectedAssignmentInfo, + Map>> partitionResults, + double maxDriftPercentage) { + + Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + + List expectedAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + List expectedNormalizedWeights = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualNormalizedWeights = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + + long expectedTotalWeight = + expectedAssignmentInfo.values().stream().mapToLong(Pair::first).sum(); + expectedAssignmentInfo.forEach( + (subtaskId, pair) -> { + expectedAssignedKeyCounts.add(pair.second()); + expectedNormalizedWeights.add(pair.first().doubleValue() / expectedTotalWeight); + }); + + long actualTotalWeight = + partitionResults.values().stream().mapToLong(pair -> pair.first().longValue()).sum(); + partitionResults.forEach( + (subtaskId, pair) -> { + actualAssignedKeyCounts.add(pair.second().size()); + actualNormalizedWeights.add(pair.first().doubleValue() / actualTotalWeight); + }); + + // number of assigned keys should match exactly + Assertions.assertThat(actualAssignedKeyCounts) + .as("the number of assigned keys should match for every subtask") + .isEqualTo(expectedAssignedKeyCounts); + + // weight for every subtask shouldn't differ for more than some threshold relative to the + // expected weight + for (int subtaskId = 0; subtaskId < expectedNormalizedWeights.size(); ++subtaskId) { + double expectedWeight = expectedNormalizedWeights.get(subtaskId); + double min = expectedWeight * (1 - maxDriftPercentage / 100); + double max = expectedWeight * (1 + maxDriftPercentage / 100); + Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) + .as( + "Subtask %d weight should within %.1f percent of the expected range %s", + subtaskId, maxDriftPercentage, expectedWeight) + .isBetween(min, max); + } + } +} diff --git a/jmh.gradle b/jmh.gradle index ea317cc2eea1..de50162cb099 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -26,10 +26,18 @@ def sparkVersions = (System.getProperty("sparkVersions") != null ? System.getPro def scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") def jmhProjects = [project(":iceberg-core"), project(":iceberg-data")] +if (flinkVersions.contains("1.16")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.16")) +} + if (flinkVersions.contains("1.17")) { jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.17")) } +if (flinkVersions.contains("1.18")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.18")) +} + if (sparkVersions.contains("3.3")) { jmhProjects.add(project(":iceberg-spark:iceberg-spark-3.3_${scalaVersion}")) jmhProjects.add(project(":iceberg-spark:iceberg-spark-extensions-3.3_${scalaVersion}")) From d28fcf2f746945d93d42d63268564d08c7b13a72 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sun, 31 Mar 2024 12:55:18 +0200 Subject: [PATCH 04/11] Spark: Don't allow branch_ usage with TIMESTAMP AS OF (#10059) --- .../main/java/org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../test/java/org/apache/iceberg/spark/sql/TestSelect.java | 6 ++++++ .../main/java/org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../test/java/org/apache/iceberg/spark/sql/TestSelect.java | 6 ++++++ .../main/java/org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../test/java/org/apache/iceberg/spark/sql/TestSelect.java | 5 +++++ 6 files changed, 20 insertions(+), 3 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 48decf995fbb..5638af071ad7 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -205,7 +205,7 @@ public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); // convert the timestamp to milliseconds as Spark passes microseconds diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 54d8dd01b92e..2265dec4763c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -453,6 +453,12 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + + // using branch_b1 in the table identifier and TIMESTAMP AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } @Test diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index d72a281fe825..7894adee410d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -215,7 +215,7 @@ public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); // convert the timestamp to milliseconds as Spark passes microseconds diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 93bb21b41a2b..85d340c8b27b 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -448,6 +448,12 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + + // using branch_b1 in the table identifier and TIMESTAMP AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } @Test diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 7357a4683bed..0be9fd948466 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -215,7 +215,7 @@ public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); // convert the timestamp to milliseconds as Spark passes microseconds diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 4c99a38d29fc..3ecfc60b49b4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -529,6 +529,11 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + + // using branch_b1 in the table identifier and TIMESTAMP AS OF + assertThatThrownBy(() -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } @TestTemplate From 78128283534f13d7fa5146743c0761c14a06354b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 31 Mar 2024 23:08:23 +0200 Subject: [PATCH 05/11] Build: Bump software.amazon.awssdk:bom from 2.25.18 to 2.25.21 (#10072) Bumps software.amazon.awssdk:bom from 2.25.18 to 2.25.21. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index a0fb79bcd363..68e9ddee6dde 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.25.18" +awssdk-bom = "2.25.21" azuresdk-bom = "1.2.21" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From a7f87c7e05b4127c0c0094576896fdf9ecae903e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 31 Mar 2024 23:08:51 +0200 Subject: [PATCH 06/11] Build: Bump org.glassfish.jaxb:jaxb-runtime from 2.3.3 to 2.3.9 (#9988) Bumps org.glassfish.jaxb:jaxb-runtime from 2.3.3 to 2.3.9. --- updated-dependencies: - dependency-name: org.glassfish.jaxb:jaxb-runtime dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 68e9ddee6dde..261fd983d386 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -57,7 +57,7 @@ jackson214 = { strictly = "2.14.2"} jackson215 = { strictly = "2.15.2"} jakarta-el-api = "3.0.3" jaxb-api = "2.3.1" -jaxb-runtime = "2.3.3" +jaxb-runtime = "2.3.9" jetty = "9.4.54.v20240208" junit = "5.10.1" kafka = "3.7.0" From ededfcb78f75940265cba24e61c5f9c218d45b95 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 2 Apr 2024 08:42:44 +0200 Subject: [PATCH 07/11] Build: Bump com.azure:azure-sdk-bom from 1.2.21 to 1.2.22 (#10071) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.2.21 to 1.2.22. - [Release notes](https://github.com/azure/azure-sdk-for-java/releases) - [Commits](https://github.com/azure/azure-sdk-for-java/compare/azure-sdk-bom_1.2.21...azure-sdk-bom_1.2.22) --- updated-dependencies: - dependency-name: com.azure:azure-sdk-bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 261fd983d386..41b87a22faf4 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" awssdk-bom = "2.25.21" -azuresdk-bom = "1.2.21" +azuresdk-bom = "1.2.22" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" From 815b2c649fa51b0ec837a7fa761961820eadfef0 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Tue, 2 Apr 2024 20:12:29 +0900 Subject: [PATCH 08/11] Core, Flink, Spark: Migrate remaining subclasses of TableTestBase to JUnit5 (#10063) --- .../org/apache/iceberg/TableTestBase.java | 753 ------------------ .../apache/iceberg/TestContentFileParser.java | 36 +- .../iceberg/TestFileScanTaskParser.java | 13 +- .../iceberg/util/TestTableScanUtil.java | 10 +- .../data/TestGenericFileWriterFactory.java | 5 - .../iceberg/io/TestFileWriterFactory.java | 189 ++--- .../iceberg/io/TestPartitioningWriters.java | 85 +- .../iceberg/io/TestPositionDeltaWriters.java | 49 +- .../iceberg/io/TestRollingFileWriters.java | 64 +- .../org/apache/iceberg/io/WriterTestBase.java | 8 +- .../flink/sink/TestDeltaTaskWriter.java | 62 +- .../sink/TestFlinkFileWriterFactory.java | 5 - .../sink/TestFlinkPartitioningWriters.java | 5 - .../sink/TestFlinkPositionDeltaWriters.java | 5 - .../sink/TestFlinkRollingFileWriters.java | 5 - .../flink/sink/TestIcebergFilesCommitter.java | 87 +- .../source/TestStreamingMonitorFunction.java | 43 +- .../source/TestStreamingReaderOperator.java | 38 +- .../flink/sink/TestDeltaTaskWriter.java | 62 +- .../sink/TestFlinkFileWriterFactory.java | 5 - .../sink/TestFlinkPartitioningWriters.java | 5 - .../sink/TestFlinkPositionDeltaWriters.java | 5 - .../sink/TestFlinkRollingFileWriters.java | 5 - .../flink/sink/TestIcebergFilesCommitter.java | 87 +- .../source/TestStreamingMonitorFunction.java | 43 +- .../source/TestStreamingReaderOperator.java | 38 +- .../flink/sink/TestDeltaTaskWriter.java | 62 +- .../sink/TestFlinkFileWriterFactory.java | 5 - .../sink/TestFlinkPartitioningWriters.java | 10 +- .../sink/TestFlinkPositionDeltaWriters.java | 5 - .../sink/TestFlinkRollingFileWriters.java | 5 - .../flink/sink/TestIcebergFilesCommitter.java | 87 +- .../source/TestStreamingMonitorFunction.java | 43 +- .../source/TestStreamingReaderOperator.java | 38 +- .../source/TestSparkFileWriterFactory.java | 5 - .../source/TestSparkPartitioningWriters.java | 5 - .../source/TestSparkPositionDeltaWriters.java | 5 - .../source/TestSparkRollingFileWriters.java | 5 - .../source/TestSparkFileWriterFactory.java | 5 - .../source/TestSparkPartitioningWriters.java | 5 - .../source/TestSparkPositionDeltaWriters.java | 5 - .../source/TestSparkRollingFileWriters.java | 5 - .../source/TestSparkFileWriterFactory.java | 5 - .../source/TestSparkPartitioningWriters.java | 5 - .../source/TestSparkPositionDeltaWriters.java | 5 - .../source/TestSparkRollingFileWriters.java | 5 - 46 files changed, 571 insertions(+), 1456 deletions(-) delete mode 100644 core/src/test/java/org/apache/iceberg/TableTestBase.java diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java deleted file mode 100644 index a92497f88e87..000000000000 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ /dev/null @@ -1,753 +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; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.File; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Iterators; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.relocated.com.google.common.io.Files; -import org.apache.iceberg.types.Conversions; -import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; - -public class TableTestBase { - // Schema passed to create tables - public static final Schema SCHEMA = - new Schema( - required(3, "id", Types.IntegerType.get()), required(4, "data", Types.StringType.get())); - - protected static final int BUCKETS_NUMBER = 16; - - // Partition spec used to create tables - public static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).bucket("data", BUCKETS_NUMBER).build(); - - static final DataFile FILE_A = - DataFiles.builder(SPEC) - .withPath("/path/to/data-a.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - static final DataFile FILE_A2 = - DataFiles.builder(SPEC) - .withPath("/path/to/data-a-2.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - static final DeleteFile FILE_A_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofPositionDeletes() - .withPath("/path/to/data-a-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - // Equality delete files. - static final DeleteFile FILE_A2_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofEqualityDeletes(1) - .withPath("/path/to/data-a2-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=0") - .withRecordCount(1) - .build(); - static final DataFile FILE_B = - DataFiles.builder(SPEC) - .withPath("/path/to/data-b.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=1") // easy way to set partition data for now - .withRecordCount(1) - .withSplitOffsets(ImmutableList.of(1L)) - .build(); - static final DeleteFile FILE_B_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofPositionDeletes() - .withPath("/path/to/data-b-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=1") // easy way to set partition data for now - .withRecordCount(1) - .build(); - static final DataFile FILE_C = - DataFiles.builder(SPEC) - .withPath("/path/to/data-c.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=2") // easy way to set partition data for now - .withRecordCount(1) - .withSplitOffsets(ImmutableList.of(2L, 8L)) - .build(); - static final DeleteFile FILE_C2_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofEqualityDeletes(1) - .withPath("/path/to/data-c-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=2") // easy way to set partition data for now - .withRecordCount(1) - .build(); - static final DataFile FILE_D = - DataFiles.builder(SPEC) - .withPath("/path/to/data-d.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=3") // easy way to set partition data for now - .withRecordCount(1) - .withSplitOffsets(ImmutableList.of(0L, 3L, 6L)) - .build(); - static final DeleteFile FILE_D2_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofEqualityDeletes(1) - .withPath("/path/to/data-d-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=3") // easy way to set partition data for now - .withRecordCount(1) - .build(); - static final DataFile FILE_WITH_STATS = - DataFiles.builder(SPEC) - .withPath("/path/to/data-with-stats.parquet") - .withMetrics( - new Metrics( - 10L, - ImmutableMap.of(3, 100L, 4, 200L), // column sizes - ImmutableMap.of(3, 90L, 4, 180L), // value counts - ImmutableMap.of(3, 10L, 4, 20L), // null value counts - ImmutableMap.of(3, 0L, 4, 0L), // nan value counts - ImmutableMap.of( - 3, - Conversions.toByteBuffer(Types.IntegerType.get(), 1), - 4, - Conversions.toByteBuffer(Types.IntegerType.get(), 2)), // lower bounds - ImmutableMap.of( - 3, - Conversions.toByteBuffer(Types.IntegerType.get(), 5), - 4, - Conversions.toByteBuffer(Types.IntegerType.get(), 10)) // upperbounds - )) - .withFileSizeInBytes(350) - .build(); - - static final FileIO FILE_IO = new TestTables.LocalFileIO(); - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - protected File tableDir = null; - protected File metadataDir = null; - public TestTables.TestTable table = null; - - protected final int formatVersion; - - @SuppressWarnings("checkstyle:MemberName") - protected final TableAssertions V1Assert; - - @SuppressWarnings("checkstyle:MemberName") - protected final TableAssertions V2Assert; - - public TableTestBase(int formatVersion) { - this.formatVersion = formatVersion; - this.V1Assert = new TableAssertions(1, formatVersion); - this.V2Assert = new TableAssertions(2, formatVersion); - } - - @Before - public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); - tableDir.delete(); // created by table create - - this.metadataDir = new File(tableDir, "metadata"); - this.table = create(SCHEMA, SPEC); - } - - @After - public void cleanupTables() { - TestTables.clearTables(); - } - - List listManifestFiles() { - return listManifestFiles(tableDir); - } - - List listManifestFiles(File tableDirToList) { - return Lists.newArrayList( - new File(tableDirToList, "metadata") - .listFiles( - (dir, name) -> - !name.startsWith("snap") - && Files.getFileExtension(name).equalsIgnoreCase("avro"))); - } - - List listManifestLists(String tableDirToList) { - return Lists.newArrayList( - new File(tableDirToList, "metadata") - .listFiles( - (dir, name) -> - name.startsWith("snap") - && Files.getFileExtension(name).equalsIgnoreCase("avro"))); - } - - public static long countAllMetadataFiles(File tableDir) { - return Arrays.stream(new File(tableDir, "metadata").listFiles()) - .filter(f -> f.isFile()) - .count(); - } - - protected TestTables.TestTable create(Schema schema, PartitionSpec spec) { - return TestTables.create(tableDir, "test", schema, spec, formatVersion); - } - - TestTables.TestTable load() { - return TestTables.load(tableDir, "test"); - } - - Integer version() { - return TestTables.metadataVersion("test"); - } - - public TableMetadata readMetadata() { - return TestTables.readMetadata("test"); - } - - ManifestFile writeManifest(DataFile... files) throws IOException { - return writeManifest(null, files); - } - - ManifestFile writeManifest(Long snapshotId, DataFile... files) throws IOException { - File manifestFile = temp.newFile("input.m0.avro"); - Assert.assertTrue(manifestFile.delete()); - OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); - - ManifestWriter writer = - ManifestFiles.write(formatVersion, table.spec(), outputFile, snapshotId); - try { - for (DataFile file : files) { - writer.add(file); - } - } finally { - writer.close(); - } - - return writer.toManifestFile(); - } - - ManifestFile writeManifest(String fileName, ManifestEntry... entries) throws IOException { - return writeManifest(null, fileName, entries); - } - - ManifestFile writeManifest(Long snapshotId, ManifestEntry... entries) throws IOException { - return writeManifest(snapshotId, "input.m0.avro", entries); - } - - @SuppressWarnings("unchecked") - > ManifestFile writeManifest( - Long snapshotId, String fileName, ManifestEntry... entries) throws IOException { - File manifestFile = temp.newFile(fileName); - Assert.assertTrue(manifestFile.delete()); - OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); - - ManifestWriter writer; - if (entries[0].file() instanceof DataFile) { - writer = - (ManifestWriter) - ManifestFiles.write(formatVersion, table.spec(), outputFile, snapshotId); - } else { - writer = - (ManifestWriter) - ManifestFiles.writeDeleteManifest( - formatVersion, table.spec(), outputFile, snapshotId); - } - try { - for (ManifestEntry entry : entries) { - writer.addEntry((ManifestEntry) entry); - } - } finally { - writer.close(); - } - - return writer.toManifestFile(); - } - - ManifestFile writeDeleteManifest(int newFormatVersion, Long snapshotId, DeleteFile... deleteFiles) - throws IOException { - OutputFile manifestFile = - org.apache.iceberg.Files.localOutput( - FileFormat.AVRO.addExtension(temp.newFile().toString())); - ManifestWriter writer = - ManifestFiles.writeDeleteManifest(newFormatVersion, SPEC, manifestFile, snapshotId); - try { - for (DeleteFile deleteFile : deleteFiles) { - writer.add(deleteFile); - } - } finally { - writer.close(); - } - return writer.toManifestFile(); - } - - ManifestFile writeManifestWithName(String name, DataFile... files) throws IOException { - File manifestFile = temp.newFile(name + ".avro"); - Assert.assertTrue(manifestFile.delete()); - OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); - - ManifestWriter writer = - ManifestFiles.write(formatVersion, table.spec(), outputFile, null); - try { - for (DataFile file : files) { - writer.add(file); - } - } finally { - writer.close(); - } - - return writer.toManifestFile(); - } - - > ManifestEntry manifestEntry( - ManifestEntry.Status status, Long snapshotId, F file) { - return manifestEntry(status, snapshotId, 0L, 0L, file); - } - - > ManifestEntry manifestEntry( - ManifestEntry.Status status, - Long snapshotId, - Long dataSequenceNumber, - Long fileSequenceNumber, - F file) { - - GenericManifestEntry entry = new GenericManifestEntry<>(table.spec().partitionType()); - switch (status) { - case ADDED: - if (dataSequenceNumber != null && dataSequenceNumber != 0) { - return entry.wrapAppend(snapshotId, dataSequenceNumber, file); - } else { - return entry.wrapAppend(snapshotId, file); - } - case EXISTING: - return entry.wrapExisting(snapshotId, dataSequenceNumber, fileSequenceNumber, file); - case DELETED: - return entry.wrapDelete(snapshotId, dataSequenceNumber, fileSequenceNumber, file); - default: - throw new IllegalArgumentException("Unexpected entry status: " + status); - } - } - - void validateSnapshot(Snapshot old, Snapshot snap, DataFile... newFiles) { - validateSnapshot(old, snap, null, newFiles); - } - - void validateSnapshot(Snapshot old, Snapshot snap, long sequenceNumber, DataFile... newFiles) { - validateSnapshot(old, snap, (Long) sequenceNumber, newFiles); - } - - @SuppressWarnings("checkstyle:HiddenField") - Snapshot commit(Table table, SnapshotUpdate snapshotUpdate, String branch) { - Snapshot snapshot; - if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - snapshotUpdate.commit(); - snapshot = table.currentSnapshot(); - } else { - ((SnapshotProducer) snapshotUpdate.toBranch(branch)).commit(); - snapshot = table.snapshot(branch); - } - - return snapshot; - } - - Snapshot apply(SnapshotUpdate snapshotUpdate, String branch) { - if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - return ((SnapshotProducer) snapshotUpdate).apply(); - } else { - return ((SnapshotProducer) snapshotUpdate.toBranch(branch)).apply(); - } - } - - void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber, DataFile... newFiles) { - Assert.assertEquals( - "Should not change delete manifests", - old != null ? Sets.newHashSet(old.deleteManifests(FILE_IO)) : ImmutableSet.of(), - Sets.newHashSet(snap.deleteManifests(FILE_IO))); - List oldManifests = old != null ? old.dataManifests(FILE_IO) : ImmutableList.of(); - - // copy the manifests to a modifiable list and remove the existing manifests - List newManifests = Lists.newArrayList(snap.dataManifests(FILE_IO)); - for (ManifestFile oldManifest : oldManifests) { - Assert.assertTrue( - "New snapshot should contain old manifests", newManifests.remove(oldManifest)); - } - - Assert.assertEquals( - "Should create 1 new manifest and reuse old manifests", 1, newManifests.size()); - ManifestFile manifest = newManifests.get(0); - - long id = snap.snapshotId(); - Iterator newPaths = paths(newFiles).iterator(); - - for (ManifestEntry entry : ManifestFiles.read(manifest, FILE_IO).entries()) { - DataFile file = entry.file(); - if (sequenceNumber != null) { - V1Assert.assertEquals( - "Data sequence number should default to 0", 0, entry.dataSequenceNumber().longValue()); - V1Assert.assertEquals( - "Data sequence number should default to 0", - 0, - entry.file().dataSequenceNumber().longValue()); - - V2Assert.assertEquals( - "Data sequence number should match expected", - sequenceNumber, - entry.dataSequenceNumber()); - V2Assert.assertEquals( - "Data sequence number should match expected", - sequenceNumber, - entry.file().dataSequenceNumber()); - V2Assert.assertEquals( - "Sequence number should match expected", - snap.sequenceNumber(), - entry.dataSequenceNumber().longValue()); - - V2Assert.assertEquals( - "File sequence number should match expected", - sequenceNumber, - entry.file().fileSequenceNumber()); - V2Assert.assertEquals( - "File sequence number should match expected", - snap.sequenceNumber(), - entry.file().fileSequenceNumber().longValue()); - } - Assert.assertEquals("Path should match expected", newPaths.next(), file.path().toString()); - Assert.assertEquals("File's snapshot ID should match", id, (long) entry.snapshotId()); - } - - Assert.assertFalse("Should find all files in the manifest", newPaths.hasNext()); - - Assert.assertEquals("Schema ID should match", table.schema().schemaId(), (int) snap.schemaId()); - } - - void validateTableFiles(Table tbl, DataFile... expectedFiles) { - Set expectedFilePaths = Sets.newHashSet(); - for (DataFile file : expectedFiles) { - expectedFilePaths.add(file.path()); - } - Set actualFilePaths = Sets.newHashSet(); - for (FileScanTask task : tbl.newScan().planFiles()) { - actualFilePaths.add(task.file().path()); - } - Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); - } - - void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) { - Set expectedFilePaths = Sets.newHashSet(); - for (DataFile file : expectedFiles) { - expectedFilePaths.add(file.path()); - } - Set actualFilePaths = Sets.newHashSet(); - for (FileScanTask task : tbl.newScan().useRef(ref).planFiles()) { - actualFilePaths.add(task.file().path()); - } - Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); - } - - void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedFiles) { - Set expectedFilePaths = Sets.newHashSet(); - for (DeleteFile file : expectedFiles) { - expectedFilePaths.add(file.path()); - } - Set actualFilePaths = Sets.newHashSet(); - for (FileScanTask task : tbl.newScan().useRef(branch).planFiles()) { - for (DeleteFile file : task.deletes()) { - actualFilePaths.add(file.path()); - } - } - Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths); - } - - List paths(DataFile... dataFiles) { - List paths = Lists.newArrayListWithExpectedSize(dataFiles.length); - for (DataFile file : dataFiles) { - paths.add(file.path().toString()); - } - return paths; - } - - void validateManifest( - ManifestFile manifest, Iterator ids, Iterator expectedFiles) { - validateManifest(manifest, null, null, ids, expectedFiles, null); - } - - void validateManifest( - ManifestFile manifest, - Iterator dataSeqs, - Iterator fileSeqs, - Iterator ids, - Iterator expectedFiles) { - validateManifest(manifest, dataSeqs, fileSeqs, ids, expectedFiles, null); - } - - void validateManifest( - ManifestFile manifest, - Iterator dataSeqs, - Iterator fileSeqs, - Iterator ids, - Iterator expectedFiles, - Iterator statuses) { - for (ManifestEntry entry : ManifestFiles.read(manifest, FILE_IO).entries()) { - DataFile file = entry.file(); - DataFile expected = expectedFiles.next(); - - validateManifestSequenceNumbers(entry, dataSeqs, fileSeqs); - - Assert.assertEquals( - "Path should match expected", expected.path().toString(), file.path().toString()); - Assert.assertEquals("Snapshot ID should match expected ID", ids.next(), entry.snapshotId()); - if (statuses != null) { - Assert.assertEquals("Status should match expected", statuses.next(), entry.status()); - } - } - - Assert.assertFalse("Should find all files in the manifest", expectedFiles.hasNext()); - } - - void validateDeleteManifest( - ManifestFile manifest, - Iterator dataSeqs, - Iterator fileSeqs, - Iterator ids, - Iterator expectedFiles, - Iterator statuses) { - for (ManifestEntry entry : - ManifestFiles.readDeleteManifest(manifest, FILE_IO, null).entries()) { - DeleteFile file = entry.file(); - DeleteFile expected = expectedFiles.next(); - - validateManifestSequenceNumbers(entry, dataSeqs, fileSeqs); - - Assert.assertEquals( - "Path should match expected", expected.path().toString(), file.path().toString()); - Assert.assertEquals("Snapshot ID should match expected ID", ids.next(), entry.snapshotId()); - Assert.assertEquals("Status should match expected", statuses.next(), entry.status()); - } - - Assert.assertFalse("Should find all files in the manifest", expectedFiles.hasNext()); - } - - private > void validateManifestSequenceNumbers( - ManifestEntry entry, Iterator dataSeqs, Iterator fileSeqs) { - if (dataSeqs != null) { - V1Assert.assertEquals( - "Data sequence number should default to 0", 0, entry.dataSequenceNumber().longValue()); - V1Assert.assertEquals( - "Data sequence number should default to 0", - 0, - entry.file().dataSequenceNumber().longValue()); - - Long expectedSequenceNumber = dataSeqs.next(); - V2Assert.assertEquals( - "Data sequence number should match expected", - expectedSequenceNumber, - entry.dataSequenceNumber()); - V2Assert.assertEquals( - "Data sequence number should match expected", - expectedSequenceNumber, - entry.file().dataSequenceNumber()); - } - - if (fileSeqs != null) { - V1Assert.assertEquals( - "File sequence number should default to 0", (Long) 0L, entry.fileSequenceNumber()); - V1Assert.assertEquals( - "File sequence number should default to 0", (Long) 0L, entry.file().fileSequenceNumber()); - - Long expectedFileSequenceNumber = fileSeqs.next(); - V2Assert.assertEquals( - "File sequence number should match", - expectedFileSequenceNumber, - entry.fileSequenceNumber()); - V2Assert.assertEquals( - "File sequence number should match", - expectedFileSequenceNumber, - entry.file().fileSequenceNumber()); - } - } - - protected DataFile newDataFile(String partitionPath) { - return DataFiles.builder(table.spec()) - .withPath("/path/to/data-" + UUID.randomUUID() + ".parquet") - .withFileSizeInBytes(10) - .withPartitionPath(partitionPath) - .withRecordCount(1) - .build(); - } - - protected DeleteFile newDeleteFile(int specId, String partitionPath) { - PartitionSpec spec = table.specs().get(specId); - return FileMetadata.deleteFileBuilder(spec) - .ofPositionDeletes() - .withPath("/path/to/delete-" + UUID.randomUUID() + ".parquet") - .withFileSizeInBytes(10) - .withPartitionPath(partitionPath) - .withRecordCount(1) - .build(); - } - - protected DeleteFile newEqualityDeleteFile(int specId, String partitionPath, int... fieldIds) { - PartitionSpec spec = table.specs().get(specId); - return FileMetadata.deleteFileBuilder(spec) - .ofEqualityDeletes(fieldIds) - .withPath("/path/to/delete-" + UUID.randomUUID() + ".parquet") - .withFileSizeInBytes(10) - .withPartitionPath(partitionPath) - .withRecordCount(1) - .build(); - } - - protected PositionDelete positionDelete(CharSequence path, long pos, T row) { - PositionDelete positionDelete = PositionDelete.create(); - return positionDelete.set(path, pos, row); - } - - protected void withUnavailableLocations(Iterable locations, Action action) { - for (String location : locations) { - move(location, location + "_temp"); - } - - try { - action.invoke(); - } finally { - for (String location : locations) { - move(location + "_temp", location); - } - } - } - - private void move(String location, String newLocation) { - Path path = Paths.get(location); - Path tempPath = Paths.get(newLocation); - - try { - java.nio.file.Files.move(path, tempPath); - } catch (IOException e) { - throw new UncheckedIOException("Failed to move: " + location, e); - } - } - - static void validateManifestEntries( - ManifestFile manifest, - Iterator ids, - Iterator expectedFiles, - Iterator expectedStatuses) { - for (ManifestEntry entry : ManifestFiles.read(manifest, FILE_IO).entries()) { - DataFile file = entry.file(); - DataFile expected = expectedFiles.next(); - final ManifestEntry.Status expectedStatus = expectedStatuses.next(); - Assert.assertEquals( - "Path should match expected", expected.path().toString(), file.path().toString()); - Assert.assertEquals("Snapshot ID should match expected ID", ids.next(), entry.snapshotId()); - Assert.assertEquals("Entry status should match expected ID", expectedStatus, entry.status()); - } - - Assert.assertFalse("Should find all files in the manifest", expectedFiles.hasNext()); - } - - static Iterator statuses(ManifestEntry.Status... statuses) { - return Iterators.forArray(statuses); - } - - static Iterator dataSeqs(Long... seqs) { - return Iterators.forArray(seqs); - } - - static Iterator fileSeqs(Long... seqs) { - return Iterators.forArray(seqs); - } - - static Iterator ids(Long... ids) { - return Iterators.forArray(ids); - } - - static Iterator files(DataFile... files) { - return Iterators.forArray(files); - } - - static Iterator files(DeleteFile... files) { - return Iterators.forArray(files); - } - - static Iterator files(ManifestFile manifest) { - return ManifestFiles.read(manifest, FILE_IO).iterator(); - } - - /** Used for assertions that only apply if the table version is v2. */ - protected static class TableAssertions { - private boolean enabled; - - private TableAssertions(int validForVersion, int formatVersion) { - this.enabled = validForVersion == formatVersion; - } - - void disable() { - this.enabled = false; - } - - void enable() { - this.enabled = true; - } - - void assertEquals(String context, int expected, int actual) { - if (enabled) { - Assert.assertEquals(context, expected, actual); - } - } - - void assertEquals(String context, long expected, long actual) { - if (enabled) { - Assert.assertEquals(context, expected, actual); - } - } - - void assertEquals(String context, Object expected, Object actual) { - if (enabled) { - Assert.assertEquals(context, expected, actual); - } - } - } - - @FunctionalInterface - protected interface Action { - void invoke(); - } -} diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index 4fda388159ec..1e185b6544e1 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -37,24 +37,24 @@ public class TestContentFileParser { @Test public void testNullArguments() throws Exception { - Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(null, TableTestBase.SPEC)) + Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(null, TestBase.SPEC)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid content file: null"); - Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(TableTestBase.FILE_A, null)) + Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(TestBase.FILE_A, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid partition spec: null"); Assertions.assertThatThrownBy( - () -> ContentFileParser.toJson(TableTestBase.FILE_A, TableTestBase.SPEC, null)) + () -> ContentFileParser.toJson(TestBase.FILE_A, TestBase.SPEC, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid JSON generator: null"); - Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(null, TableTestBase.SPEC)) + Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(null, TestBase.SPEC)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid JSON node for content file: null"); - String jsonStr = ContentFileParser.toJson(TableTestBase.FILE_A, TableTestBase.SPEC); + String jsonStr = ContentFileParser.toJson(TestBase.FILE_A, TestBase.SPEC); JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(jsonNode, null)) .isInstanceOf(IllegalArgumentException.class) @@ -96,13 +96,13 @@ private static Stream provideSpecAndDataFile() { dataFileWithAllOptional(PartitionSpec.unpartitioned()), dataFileJsonWithAllOptional(PartitionSpec.unpartitioned())), Arguments.of( - TableTestBase.SPEC, - dataFileWithRequiredOnly(TableTestBase.SPEC), - dataFileJsonWithRequiredOnly(TableTestBase.SPEC)), + TestBase.SPEC, + dataFileWithRequiredOnly(TestBase.SPEC), + dataFileJsonWithRequiredOnly(TestBase.SPEC)), Arguments.of( - TableTestBase.SPEC, - dataFileWithAllOptional(TableTestBase.SPEC), - dataFileJsonWithAllOptional(TableTestBase.SPEC))); + TestBase.SPEC, + dataFileWithAllOptional(TestBase.SPEC), + dataFileJsonWithAllOptional(TestBase.SPEC))); } private static DataFile dataFileWithRequiredOnly(PartitionSpec spec) { @@ -182,7 +182,7 @@ private static DataFile dataFileWithAllOptional(PartitionSpec spec) { .withSplitOffsets(Arrays.asList(128L, 256L)) .withEncryptionKeyMetadata(ByteBuffer.wrap(new byte[16])) .withSortOrder( - SortOrder.builderFor(TableTestBase.SCHEMA) + SortOrder.builderFor(TestBase.SCHEMA) .withOrderId(1) .sortBy("id", SortDirection.ASC, NullOrder.NULLS_FIRST) .build()); @@ -206,13 +206,13 @@ private static Stream provideSpecAndDeleteFile() { deleteFileWithAllOptional(PartitionSpec.unpartitioned()), deleteFileJsonWithAllOptional(PartitionSpec.unpartitioned())), Arguments.of( - TableTestBase.SPEC, - deleteFileWithRequiredOnly(TableTestBase.SPEC), - deleteFileJsonWithRequiredOnly(TableTestBase.SPEC)), + TestBase.SPEC, + deleteFileWithRequiredOnly(TestBase.SPEC), + deleteFileJsonWithRequiredOnly(TestBase.SPEC)), Arguments.of( - TableTestBase.SPEC, - deleteFileWithAllOptional(TableTestBase.SPEC), - deleteFileJsonWithAllOptional(TableTestBase.SPEC))); + TestBase.SPEC, + deleteFileWithAllOptional(TestBase.SPEC), + deleteFileJsonWithAllOptional(TestBase.SPEC))); } private static DeleteFile deleteFileWithRequiredOnly(PartitionSpec spec) { diff --git a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java index 42785c7a778e..73bfba996ebf 100644 --- a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java @@ -41,7 +41,7 @@ public void testNullArguments() { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testParser(boolean caseSensitive) { - PartitionSpec spec = TableTestBase.SPEC; + PartitionSpec spec = TestBase.SPEC; FileScanTask fileScanTask = createScanTask(spec, caseSensitive); String jsonStr = FileScanTaskParser.toJson(fileScanTask); Assertions.assertThat(jsonStr).isEqualTo(expectedFileScanTaskJson()); @@ -58,9 +58,9 @@ private FileScanTask createScanTask(PartitionSpec spec, boolean caseSensitive) { } return new BaseFileScanTask( - TableTestBase.FILE_A, - new DeleteFile[] {TableTestBase.FILE_A_DELETES, TableTestBase.FILE_A2_DELETES}, - SchemaParser.toJson(TableTestBase.SCHEMA), + TestBase.FILE_A, + new DeleteFile[] {TestBase.FILE_A_DELETES, TestBase.FILE_A2_DELETES}, + SchemaParser.toJson(TestBase.SCHEMA), PartitionSpecParser.toJson(spec), residualEvaluator); } @@ -99,10 +99,7 @@ private static void assertFileScanTaskEquals( Assertions.assertThat(actual.spec()).isEqualTo(expected.spec()); Assertions.assertThat( ExpressionUtil.equivalent( - expected.residual(), - actual.residual(), - TableTestBase.SCHEMA.asStruct(), - caseSensitive)) + expected.residual(), actual.residual(), TestBase.SCHEMA.asStruct(), caseSensitive)) .as("Residual expression should match") .isTrue(); } diff --git a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java index cfe3cb625e4f..d868fba90190 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java @@ -42,7 +42,7 @@ import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SplittableScanTask; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.io.CloseableIterable; @@ -136,7 +136,7 @@ public void testTaskGroupPlanning() { @Test public void testTaskGroupPlanningCorruptedOffset() { DataFile dataFile = - DataFiles.builder(TableTestBase.SPEC) + DataFiles.builder(TestBase.SPEC) .withPath("/path/to/data-a.parquet") .withFileSizeInBytes(10) .withPartitionPath("data_bucket=0") @@ -146,14 +146,14 @@ public void testTaskGroupPlanningCorruptedOffset() { .build(); ResidualEvaluator residualEvaluator = - ResidualEvaluator.of(TableTestBase.SPEC, Expressions.equal("id", 1), false); + ResidualEvaluator.of(TestBase.SPEC, Expressions.equal("id", 1), false); BaseFileScanTask baseFileScanTask = new BaseFileScanTask( dataFile, null, - SchemaParser.toJson(TableTestBase.SCHEMA), - PartitionSpecParser.toJson(TableTestBase.SPEC), + SchemaParser.toJson(TestBase.SCHEMA), + PartitionSpecParser.toJson(TestBase.SPEC), residualEvaluator); List baseFileScanTasks = ImmutableList.of(baseFileScanTask); diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericFileWriterFactory.java b/data/src/test/java/org/apache/iceberg/data/TestGenericFileWriterFactory.java index 6752234f5ef5..2ab908c8c701 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericFileWriterFactory.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericFileWriterFactory.java @@ -19,7 +19,6 @@ package org.apache.iceberg.data; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestFileWriterFactory; @@ -28,10 +27,6 @@ public class TestGenericFileWriterFactory extends TestFileWriterFactory { - public TestGenericFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java index 152679abbde5..996a434f97f3 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java @@ -21,14 +21,21 @@ import static org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH; import static org.apache.iceberg.MetadataColumns.DELETE_FILE_POS; import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; import java.util.List; import java.util.Map; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; @@ -50,43 +57,35 @@ import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestFileWriterFactory extends WriterTestBase { - @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO, false}, - new Object[] {FileFormat.AVRO, true}, - new Object[] {FileFormat.PARQUET, false}, - new Object[] {FileFormat.PARQUET, true}, - new Object[] {FileFormat.ORC, false}, - new Object[] {FileFormat.ORC, true} - }; + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, Partitioned = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO, false}, + new Object[] {2, FileFormat.AVRO, true}, + new Object[] {2, FileFormat.PARQUET, false}, + new Object[] {2, FileFormat.PARQUET, true}, + new Object[] {2, FileFormat.ORC, false}, + new Object[] {2, FileFormat.ORC, true}); } - private static final int TABLE_FORMAT_VERSION = 2; private static final String PARTITION_VALUE = "aaa"; - private final FileFormat fileFormat; - private final boolean partitioned; + @Parameter(index = 1) + private FileFormat fileFormat; + + @Parameter(index = 2) + private boolean partitioned; private StructLike partition = null; private OutputFileFactory fileFactory = null; private List dataRows; - public TestFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(TABLE_FORMAT_VERSION); - this.fileFormat = fileFormat; - this.partitioned = partitioned; - } - protected abstract StructLikeSet toSet(Iterable records); protected FileFormat format() { @@ -94,10 +93,10 @@ protected FileFormat format() { } @Override - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); // created during table creation + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -116,7 +115,7 @@ public void setupTable() throws Exception { toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "aaa"), toRow(4, "aaa"), toRow(5, "aaa")); } - @Test + @TestTemplate public void testDataWriter() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -124,10 +123,10 @@ public void testDataWriter() throws IOException { table.newRowDelta().addRows(dataFile).commit(); - Assert.assertEquals("Records should match", toSet(dataRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(dataRows)); } - @Test + @TestTemplate public void testEqualityDeleteWriter() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -151,19 +150,19 @@ public void testEqualityDeleteWriter() throws IOException { deleteRecord.copy("id", 1), deleteRecord.copy("id", 3), deleteRecord.copy("id", 5)); InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); List actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile); - Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes); + assertThat(actualDeletes).isEqualTo(expectedDeletes); // commit the written delete file table.newRowDelta().addDeletes(deleteFile).commit(); // verify the delete file is applied correctly List expectedRows = ImmutableList.of(toRow(2, "aaa"), toRow(4, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { - Assume.assumeFalse("Table must start unpartitioned", partitioned); + assumeThat(partitioned).isFalse(); List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -172,8 +171,9 @@ public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { // write an unpartitioned data file DataFile firstDataFile = writeData(writerFactory, dataRows, table.spec(), partition); - Assert.assertEquals( - "First data file must be unpartitioned", 0, firstDataFile.partition().size()); + assertThat(firstDataFile.partition().size()) + .as("First data file must be unpartitioned") + .isEqualTo(0); List deletes = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "aaa"), toRow(4, "aaa")); @@ -181,8 +181,9 @@ public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { // write an unpartitioned delete file DeleteFile firstDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition); - Assert.assertEquals( - "First delete file must be unpartitioned", 0, firstDeleteFile.partition().size()); + assertThat(firstDeleteFile.partition().size()) + .as("First delete file must be unpartitioned") + .isEqualTo(0); // commit the first data and delete files table.newAppend().appendFile(firstDataFile).commit(); @@ -195,14 +196,16 @@ public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { // write a partitioned data file DataFile secondDataFile = writeData(writerFactory, dataRows, table.spec(), partition); - Assert.assertEquals( - "Second data file must be partitioned", 1, secondDataFile.partition().size()); + assertThat(secondDataFile.partition().size()) + .as("Second data file must be partitioned") + .isEqualTo(1); // write a partitioned delete file DeleteFile secondDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition); - Assert.assertEquals( - "Second delete file must be artitioned", 1, secondDeleteFile.partition().size()); + assertThat(secondDeleteFile.partition().size()) + .as("Second delete file must be partitioned") + .isEqualTo(1); // commit the second data and delete files table.newAppend().appendFile(secondDataFile).commit(); @@ -210,10 +213,10 @@ public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { // verify both delete files are applied correctly List expectedRows = ImmutableList.of(toRow(5, "aaa"), toRow(5, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testPositionDeleteWriter() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -232,21 +235,19 @@ public void testPositionDeleteWriter() throws IOException { CharSequenceSet referencedDataFiles = result.second(); if (fileFormat == FileFormat.AVRO) { - Assert.assertNull(deleteFile.lowerBounds()); - Assert.assertNull(deleteFile.upperBounds()); - Assert.assertNull(deleteFile.columnSizes()); + assertThat(deleteFile.lowerBounds()).isNull(); + assertThat(deleteFile.upperBounds()).isNull(); + assertThat(deleteFile.columnSizes()).isNull(); } else { - Assert.assertEquals(1, referencedDataFiles.size()); - Assert.assertEquals(2, deleteFile.lowerBounds().size()); - Assert.assertTrue(deleteFile.lowerBounds().containsKey(DELETE_FILE_PATH.fieldId())); - Assert.assertEquals(2, deleteFile.upperBounds().size()); - Assert.assertTrue(deleteFile.upperBounds().containsKey(DELETE_FILE_PATH.fieldId())); - Assert.assertEquals(2, deleteFile.columnSizes().size()); + assertThat(referencedDataFiles).hasSize(1); + assertThat(deleteFile.lowerBounds()).hasSize(2).containsKey(DELETE_FILE_PATH.fieldId()); + assertThat(deleteFile.upperBounds()).hasSize(2).containsKey(DELETE_FILE_PATH.fieldId()); + assertThat(deleteFile.columnSizes()).hasSize(2); } - Assert.assertNull(deleteFile.valueCounts()); - Assert.assertNull(deleteFile.nullValueCounts()); - Assert.assertNull(deleteFile.nanValueCounts()); + assertThat(deleteFile.valueCounts()).isNull(); + assertThat(deleteFile.nullValueCounts()).isNull(); + assertThat(deleteFile.nanValueCounts()).isNull(); // verify the written delete file GenericRecord deleteRecord = GenericRecord.create(DeleteSchemaUtil.pathPosSchema()); @@ -258,7 +259,7 @@ public void testPositionDeleteWriter() throws IOException { DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 4L)); InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); List actualDeletes = readFile(DeleteSchemaUtil.pathPosSchema(), inputDeleteFile); - Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes); + assertThat(actualDeletes).isEqualTo(expectedDeletes); // commit the data and delete files table @@ -271,10 +272,10 @@ public void testPositionDeleteWriter() throws IOException { // verify the delete file is applied correctly List expectedRows = ImmutableList.of(toRow(2, "aaa"), toRow(4, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testPositionDeleteWriterWithRow() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema(), table.schema()); @@ -290,31 +291,33 @@ public void testPositionDeleteWriterWithRow() throws IOException { CharSequenceSet referencedDataFiles = result.second(); if (fileFormat == FileFormat.AVRO) { - Assert.assertNull(deleteFile.lowerBounds()); - Assert.assertNull(deleteFile.upperBounds()); - Assert.assertNull(deleteFile.columnSizes()); - Assert.assertNull(deleteFile.valueCounts()); - Assert.assertNull(deleteFile.nullValueCounts()); - Assert.assertNull(deleteFile.nanValueCounts()); + assertThat(deleteFile.lowerBounds()).isNull(); + assertThat(deleteFile.upperBounds()).isNull(); + assertThat(deleteFile.columnSizes()).isNull(); + assertThat(deleteFile.valueCounts()).isNull(); + assertThat(deleteFile.nullValueCounts()).isNull(); + assertThat(deleteFile.nanValueCounts()).isNull(); } else { - Assert.assertEquals(1, referencedDataFiles.size()); - Assert.assertEquals(4, deleteFile.lowerBounds().size()); - Assert.assertTrue(deleteFile.lowerBounds().containsKey(DELETE_FILE_PATH.fieldId())); - Assert.assertTrue(deleteFile.lowerBounds().containsKey(DELETE_FILE_POS.fieldId())); + assertThat(referencedDataFiles).hasSize(1); + assertThat(deleteFile.lowerBounds()) + .hasSize(4) + .containsKey(DELETE_FILE_PATH.fieldId()) + .containsKey(DELETE_FILE_POS.fieldId()); for (Types.NestedField column : table.schema().columns()) { - Assert.assertTrue(deleteFile.lowerBounds().containsKey(column.fieldId())); + assertThat(deleteFile.lowerBounds()).containsKey(column.fieldId()); } - Assert.assertEquals(4, deleteFile.upperBounds().size()); - Assert.assertTrue(deleteFile.upperBounds().containsKey(DELETE_FILE_PATH.fieldId())); - Assert.assertTrue(deleteFile.upperBounds().containsKey(DELETE_FILE_POS.fieldId())); + assertThat(deleteFile.upperBounds()) + .hasSize(4) + .containsKey(DELETE_FILE_PATH.fieldId()) + .containsKey(DELETE_FILE_POS.fieldId()); for (Types.NestedField column : table.schema().columns()) { - Assert.assertTrue(deleteFile.upperBounds().containsKey(column.fieldId())); + assertThat(deleteFile.upperBounds()).containsKey(column.fieldId()); } // ORC also contains metrics for the deleted row struct, not just actual data fields - Assert.assertTrue(deleteFile.columnSizes().size() >= 4); - Assert.assertTrue(deleteFile.valueCounts().size() >= 2); - Assert.assertTrue(deleteFile.nullValueCounts().size() >= 2); - Assert.assertNull(deleteFile.nanValueCounts()); + assertThat(deleteFile.columnSizes()).hasSizeGreaterThanOrEqualTo(4); + assertThat(deleteFile.valueCounts()).hasSizeGreaterThanOrEqualTo(2); + assertThat(deleteFile.nullValueCounts()).hasSizeGreaterThanOrEqualTo(2); + assertThat(deleteFile.nanValueCounts()).isNull(); } // verify the written delete file @@ -332,7 +335,7 @@ public void testPositionDeleteWriterWithRow() throws IOException { List expectedDeletes = ImmutableList.of(deleteRecord.copy(deleteRecordColumns)); InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); List actualDeletes = readFile(positionDeleteSchema, inputDeleteFile); - Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes); + assertThat(actualDeletes).isEqualTo(expectedDeletes); // commit the data and delete files table @@ -346,10 +349,10 @@ public void testPositionDeleteWriterWithRow() throws IOException { // verify the delete file is applied correctly List expectedRows = ImmutableList.of(toRow(2, "aaa"), toRow(3, "aaa"), toRow(4, "aaa"), toRow(5, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testPositionDeleteWriterMultipleDataFiles() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -369,17 +372,17 @@ public void testPositionDeleteWriterMultipleDataFiles() throws IOException { CharSequenceSet referencedDataFiles = result.second(); // verify the written delete file has NO lower and upper bounds - Assert.assertEquals(2, referencedDataFiles.size()); - Assert.assertNull(deleteFile.lowerBounds()); - Assert.assertNull(deleteFile.upperBounds()); - Assert.assertNull(deleteFile.valueCounts()); - Assert.assertNull(deleteFile.nullValueCounts()); - Assert.assertNull(deleteFile.nanValueCounts()); + assertThat(referencedDataFiles).hasSize(2); + assertThat(deleteFile.lowerBounds()).isNull(); + assertThat(deleteFile.upperBounds()).isNull(); + assertThat(deleteFile.valueCounts()).isNull(); + assertThat(deleteFile.nullValueCounts()).isNull(); + assertThat(deleteFile.nanValueCounts()).isNull(); if (fileFormat == FileFormat.AVRO) { - Assert.assertNull(deleteFile.columnSizes()); + assertThat(deleteFile.columnSizes()).isNull(); } else { - Assert.assertEquals(2, deleteFile.columnSizes().size()); + assertThat(deleteFile.columnSizes()).hasSize(2); } // commit the data and delete files @@ -402,7 +405,7 @@ public void testPositionDeleteWriterMultipleDataFiles() throws IOException { toRow(2, "aaa"), toRow(3, "aaa"), toRow(4, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } private DataFile writeData( diff --git a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java index d76774326272..88d3c16c2dca 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java @@ -22,9 +22,14 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; import java.util.List; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; @@ -35,33 +40,27 @@ import org.apache.iceberg.util.StructLikeSet; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestPartitioningWriters extends WriterTestBase { - @Parameterized.Parameters(name = "FileFormat={0}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO}, - new Object[] {FileFormat.PARQUET}, - new Object[] {FileFormat.ORC}, - }; + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.PARQUET}, + new Object[] {2, FileFormat.ORC}); } - private static final int TABLE_FORMAT_VERSION = 2; private static final long TARGET_FILE_SIZE = 128L * 1024 * 1024; - private final FileFormat fileFormat; - private OutputFileFactory fileFactory = null; + @Parameter(index = 1) + private FileFormat fileFormat; - public TestPartitioningWriters(FileFormat fileFormat) { - super(TABLE_FORMAT_VERSION); - this.fileFormat = fileFormat; - } + private OutputFileFactory fileFactory = null; protected abstract StructLikeSet toSet(Iterable records); @@ -70,9 +69,9 @@ protected FileFormat format() { } @Override - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -80,7 +79,7 @@ public void setupTable() throws Exception { this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); } - @Test + @TestTemplate public void testClusteredDataWriterNoRecords() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); ClusteredDataWriter writer = @@ -93,7 +92,7 @@ public void testClusteredDataWriterNoRecords() throws IOException { Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); } - @Test + @TestTemplate public void testClusteredDataWriterMultiplePartitions() throws IOException { table.updateSpec().addField(Expressions.ref("data")).commit(); @@ -124,7 +123,7 @@ public void testClusteredDataWriterMultiplePartitions() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testClusteredDataWriterOutOfOrderPartitions() throws IOException { table.updateSpec().addField(Expressions.ref("data")).commit(); @@ -149,7 +148,7 @@ public void testClusteredDataWriterOutOfOrderPartitions() throws IOException { writer.close(); } - @Test + @TestTemplate public void testClusteredEqualityDeleteWriterNoRecords() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -170,7 +169,7 @@ public void testClusteredEqualityDeleteWriterNoRecords() throws IOException { Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testClusteredEqualityDeleteWriterMultipleSpecs() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -236,7 +235,7 @@ public void testClusteredEqualityDeleteWriterMultipleSpecs() throws IOException Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testClusteredEqualityDeleteWriterOutOfOrderSpecsAndPartitions() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -280,12 +279,12 @@ public void testClusteredEqualityDeleteWriterOutOfOrderSpecsAndPartitions() thro writer.close(); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterNoRecordsPartitionGranularity() throws IOException { checkClusteredPositionDeleteWriterNoRecords(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterNoRecordsFileGranularity() throws IOException { checkClusteredPositionDeleteWriterNoRecords(DeleteGranularity.FILE); } @@ -308,13 +307,13 @@ private void checkClusteredPositionDeleteWriterNoRecords(DeleteGranularity delet Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterMultipleSpecsPartitionGranularity() throws IOException { checkClusteredPositionDeleteWriterMultipleSpecs(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterMultipleSpecsFileGranularity() throws IOException { checkClusteredPositionDeleteWriterMultipleSpecs(DeleteGranularity.FILE); } @@ -387,13 +386,13 @@ private void checkClusteredPositionDeleteWriterMultipleSpecs(DeleteGranularity d Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitionsPartitionGranularity() throws IOException { checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitionsFileGranularity() throws IOException { checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions(DeleteGranularity.FILE); @@ -455,12 +454,12 @@ private void checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions( writer.close(); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterPartitionGranularity() throws IOException { checkClusteredPositionDeleteWriterGranularity(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterFileGranularity() throws IOException { checkClusteredPositionDeleteWriterGranularity(DeleteGranularity.FILE); } @@ -510,7 +509,7 @@ private void checkClusteredPositionDeleteWriterGranularity(DeleteGranularity del assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testFanoutDataWriterNoRecords() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); FanoutDataWriter writer = @@ -523,7 +522,7 @@ public void testFanoutDataWriterNoRecords() throws IOException { Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); } - @Test + @TestTemplate public void testFanoutDataWriterMultiplePartitions() throws IOException { table.updateSpec().addField(Expressions.ref("data")).commit(); @@ -554,12 +553,12 @@ public void testFanoutDataWriterMultiplePartitions() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterNoRecordsPartitionGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterNoRecords(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterNoRecordsFileGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterNoRecords(DeleteGranularity.FILE); } @@ -582,13 +581,13 @@ private void checkFanoutPositionOnlyDeleteWriterNoRecords(DeleteGranularity dele Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterOutOfOrderRecordsPartitionGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterOutOfOrderRecords(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterOutOfOrderRecordsFileGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterOutOfOrderRecords(DeleteGranularity.FILE); @@ -671,12 +670,12 @@ private void checkFanoutPositionOnlyDeleteWriterOutOfOrderRecords( Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterPartitionGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterGranularity(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterFileGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterGranularity(DeleteGranularity.FILE); } diff --git a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java index aa1118306a4c..2de7bf7324c0 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java @@ -20,43 +20,42 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; import java.util.List; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.StructLikeSet; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestPositionDeltaWriters extends WriterTestBase { - @Parameterized.Parameters(name = "FileFormat={0}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO}, - new Object[] {FileFormat.ORC}, - new Object[] {FileFormat.PARQUET} - }; + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); } - private static final int TABLE_FORMAT_VERSION = 2; private static final long TARGET_FILE_SIZE = 128L * 1024 * 1024; - private final FileFormat fileFormat; - private OutputFileFactory fileFactory = null; + @Parameter(index = 1) + private FileFormat fileFormat; - public TestPositionDeltaWriters(FileFormat fileFormat) { - super(TABLE_FORMAT_VERSION); - this.fileFormat = fileFormat; - } + private OutputFileFactory fileFactory = null; protected abstract StructLikeSet toSet(Iterable records); @@ -65,9 +64,9 @@ protected FileFormat format() { } @Override - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -75,7 +74,7 @@ public void setupTable() throws Exception { this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); } - @Test + @TestTemplate public void testPositionDeltaWithOneDataWriter() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -100,7 +99,7 @@ public void testPositionDeltaWithOneDataWriter() throws IOException { Assert.assertEquals("Must not reference data files", 0, referencedDataFiles.length); } - @Test + @TestTemplate public void testPositionDeltaInsertOnly() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -136,7 +135,7 @@ public void testPositionDeltaInsertOnly() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testPositionDeltaDeleteOnly() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -192,7 +191,7 @@ public void testPositionDeltaDeleteOnly() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testPositionDeltaMultipleSpecs() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); diff --git a/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java b/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java index b229d4871f55..f6d5d41b72b3 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java @@ -20,8 +20,13 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; import java.util.List; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; @@ -29,51 +34,46 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestRollingFileWriters extends WriterTestBase { - @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO, false}, - new Object[] {FileFormat.AVRO, true}, - new Object[] {FileFormat.PARQUET, false}, - new Object[] {FileFormat.PARQUET, true}, - new Object[] {FileFormat.ORC, false}, - new Object[] {FileFormat.ORC, true} - }; + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, Partitioned = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO, false}, + new Object[] {2, FileFormat.AVRO, true}, + new Object[] {2, FileFormat.PARQUET, false}, + new Object[] {2, FileFormat.PARQUET, true}, + new Object[] {2, FileFormat.ORC, false}, + new Object[] {2, FileFormat.ORC, true}); } - private static final int TABLE_FORMAT_VERSION = 2; private static final int FILE_SIZE_CHECK_ROWS_DIVISOR = 1000; private static final long DEFAULT_FILE_SIZE = 128L * 1024 * 1024; private static final long SMALL_FILE_SIZE = 2L; private static final String PARTITION_VALUE = "aaa"; - private final FileFormat fileFormat; - private final boolean partitioned; + @Parameter(index = 1) + private FileFormat fileFormat; + + @Parameter(index = 2) + private boolean partitioned; + private StructLike partition = null; private OutputFileFactory fileFactory = null; - public TestRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(TABLE_FORMAT_VERSION); - this.fileFormat = fileFormat; - this.partitioned = partitioned; - } - protected FileFormat format() { return fileFormat; } @Override - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -89,7 +89,7 @@ public void setupTable() throws Exception { this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); } - @Test + @TestTemplate public void testRollingDataWriterNoRecords() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); RollingDataWriter writer = @@ -103,7 +103,7 @@ public void testRollingDataWriterNoRecords() throws IOException { Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); } - @Test + @TestTemplate public void testRollingDataWriterSplitData() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); RollingDataWriter writer = @@ -125,7 +125,7 @@ public void testRollingDataWriterSplitData() throws IOException { Assert.assertEquals(4, writer.result().dataFiles().size()); } - @Test + @TestTemplate public void testRollingEqualityDeleteWriterNoRecords() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -146,7 +146,7 @@ public void testRollingEqualityDeleteWriterNoRecords() throws IOException { Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testRollingEqualityDeleteWriterSplitDeletes() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -174,7 +174,7 @@ public void testRollingEqualityDeleteWriterSplitDeletes() throws IOException { Assert.assertFalse(result.referencesDataFiles()); } - @Test + @TestTemplate public void testRollingPositionDeleteWriterNoRecords() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); RollingPositionDeleteWriter writer = @@ -192,7 +192,7 @@ public void testRollingPositionDeleteWriterNoRecords() throws IOException { Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testRollingPositionDeleteWriterSplitDeletes() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); RollingPositionDeleteWriter writer = diff --git a/data/src/test/java/org/apache/iceberg/io/WriterTestBase.java b/data/src/test/java/org/apache/iceberg/io/WriterTestBase.java index b8e99515598a..52e052b711e5 100644 --- a/data/src/test/java/org/apache/iceberg/io/WriterTestBase.java +++ b/data/src/test/java/org/apache/iceberg/io/WriterTestBase.java @@ -25,7 +25,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; import org.apache.iceberg.data.Record; @@ -33,11 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.util.StructLikeSet; -public abstract class WriterTestBase extends TableTestBase { - - public WriterTestBase(int formatVersion) { - super(formatVersion); - } +public abstract class WriterTestBase extends TestBase { protected abstract FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 4ecbd1c12921..f780c6135bee 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -45,12 +45,15 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.TestTables; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; @@ -65,31 +68,28 @@ import org.apache.iceberg.util.StructLikeSet; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestDeltaTaskWriter extends TableTestBase { - private static final int FORMAT_V2 = 2; - - private final FileFormat format; - - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; - } - - public TestDeltaTaskWriter(String fileFormat) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDeltaTaskWriter extends TestBase { + + @Parameter(index = 1) + private FileFormat format; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created by table create this.metadataDir = new File(tableDir, "metadata"); @@ -170,13 +170,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitioned() throws IOException { createAndInitTable(false); testCdcEvents(false); } - @Test + @TestTemplate public void testPartitioned() throws IOException { createAndInitTable(true); testCdcEvents(true); @@ -201,12 +201,12 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(false); } - @Test + @TestTemplate public void testPartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(true); } @@ -243,17 +243,17 @@ private void testAbort(boolean partitioned) throws IOException { } } - @Test + @TestTemplate public void testUnpartitionedAbort() throws IOException { testAbort(false); } - @Test + @TestTemplate public void testPartitionedAbort() throws IOException { testAbort(true); } - @Test + @TestTemplate public void testPartitionedTableWithDataAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId()); @@ -298,7 +298,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testPartitionedTableWithDataAndIdAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); @@ -323,7 +323,7 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); } - @Test + @TestTemplate public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { Schema tableSchema = new Schema( diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java index da45241256f5..414ee40d1357 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkFileWriterFactory extends TestFileWriterFactory { - public TestFlinkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java index 3951c2e70f65..f79a3e634071 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPartitioningWriters extends TestPartitioningWriters { - public TestFlinkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java index 9e846efe6fc9..3050752d1c24 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestFlinkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java index 07716b9c3e60..03051b69cf87 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.FileWriterFactory; @@ -29,10 +28,6 @@ public class TestFlinkRollingFileWriters extends TestRollingFileWriters { - public TestFlinkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index a2fe092b7cf6..4459f224afe6 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.NavigableMap; @@ -56,10 +57,13 @@ import org.apache.iceberg.GenericManifestFile; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestHelpers; @@ -73,44 +77,39 @@ import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestIcebergFilesCommitter extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergFilesCommitter extends TestBase { private static final Configuration CONF = new Configuration(); private File flinkManifestFolder; - private final FileFormat format; - private final String branch; - - @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion = {1}, branch = {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, "main"}, - new Object[] {"avro", 2, "test-branch"}, - new Object[] {"parquet", 1, "main"}, - new Object[] {"parquet", 2, "test-branch"}, - new Object[] {"orc", 1, "main"}, - new Object[] {"orc", 2, "test-branch"} - }; - } - - public TestIcebergFilesCommitter(String format, int formatVersion, String branch) { - super(formatVersion); - this.format = FileFormat.fromString(format); - this.branch = branch; + @Parameter(index = 1) + private FileFormat format; + + @Parameter(index = 2) + private String branch; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {2, FileFormat.AVRO, "test-branch"}, + new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {2, FileFormat.PARQUET, "test-branch"}, + new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {2, FileFormat.ORC, "test-branch"}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - flinkManifestFolder = temp.newFolder(); + flinkManifestFolder = Files.createTempDirectory(temp, "flink").toFile(); - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -125,7 +124,7 @@ public void setupTable() throws IOException { .commit(); } - @Test + @TestTemplate public void testCommitTxnWithoutDataFiles() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -156,7 +155,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { } } - @Test + @TestTemplate public void testMaxContinuousEmptyCommits() throws Exception { table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); @@ -182,7 +181,7 @@ private WriteResult of(DataFile dataFile) { return WriteResult.builder().addDataFiles(dataFile).build(); } - @Test + @TestTemplate public void testCommitTxn() throws Exception { // Test with 3 continues checkpoints: // 1. snapshotState for checkpoint#1 @@ -225,7 +224,7 @@ public void testCommitTxn() throws Exception { } } - @Test + @TestTemplate public void testOrderedEventsBetweenCheckpoints() throws Exception { // It's possible that two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -278,7 +277,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testDisorderedEventsBetweenCheckpoints() throws Exception { // It's possible that the two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -331,7 +330,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -392,7 +391,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's // possible that we @@ -490,7 +489,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except } } - @Test + @TestTemplate public void testStartAnotherJobToWriteSameTable() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -557,7 +556,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleJobsWriteSameTable() throws Exception { long timestamp = 0; List tableRows = Lists.newArrayList(); @@ -595,7 +594,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -693,7 +692,7 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; @@ -722,7 +721,7 @@ public void testBoundedStream() throws Exception { } } - @Test + @TestTemplate public void testFlinkManifests() throws Exception { long timestamp = 0; final long checkpoint = 10; @@ -766,7 +765,7 @@ public void testFlinkManifests() throws Exception { } } - @Test + @TestTemplate public void testDeleteFiles() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -835,7 +834,7 @@ public void testDeleteFiles() throws Exception { } } - @Test + @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -888,7 +887,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } - @Test + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; int checkpointId = 0; diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 6d1891baf538..a612617835b0 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -22,7 +22,9 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.time.Duration; +import java.util.Arrays; import java.util.List; import java.util.concurrent.CountDownLatch; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -33,9 +35,11 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -50,13 +54,12 @@ import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingMonitorFunction extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingMonitorFunction extends TestBase { private static final Schema SCHEMA = new Schema( @@ -65,19 +68,15 @@ public class TestStreamingMonitorFunction extends TableTestBase { private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; private static final long WAIT_TIME_MILLIS = 10 * 1000L; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingMonitorFunction(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -99,7 +98,7 @@ private void runSourceFunctionInTask( task.start(); } - @Test + @TestTemplate public void testConsumeWithoutStartSnapshotId() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -122,7 +121,7 @@ public void testConsumeWithoutStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartSnapshotId() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -155,7 +154,7 @@ public void testConsumeFromStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartTag() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -187,7 +186,7 @@ public void testConsumeFromStartTag() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -243,7 +242,7 @@ private void awaitExpectedSplits(TestSourceContext sourceContext) { }); } - @Test + @TestTemplate public void testInvalidMaxPlanningSnapshotCount() { ScanContext scanContext1 = ScanContext.builder() @@ -265,7 +264,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .hasMessage("The max-planning-snapshot-count must be greater than zero"); } - @Test + @TestTemplate public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { generateRecordsAndCommitTxn(10); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index e258a197edf3..f96426a59a2d 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -20,6 +20,8 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -32,28 +34,28 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.TestTableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingReaderOperator extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingReaderOperator extends TestBase { private static final Schema SCHEMA = new Schema( @@ -61,19 +63,15 @@ public class TestStreamingReaderOperator extends TableTestBase { Types.NestedField.required(2, "data", Types.StringType.get())); private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingReaderOperator(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -81,7 +79,7 @@ public void setupTable() throws IOException { table = create(SCHEMA, PartitionSpec.unpartitioned()); } - @Test + @TestTemplate public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); @@ -109,7 +107,7 @@ public void testProcessAllRecords() throws Exception { } } - @Test + @TestTemplate public void testTriggerCheckpoint() throws Exception { // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading // records from @@ -150,7 +148,7 @@ public void testTriggerCheckpoint() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 4ecbd1c12921..f780c6135bee 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -45,12 +45,15 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.TestTables; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; @@ -65,31 +68,28 @@ import org.apache.iceberg.util.StructLikeSet; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestDeltaTaskWriter extends TableTestBase { - private static final int FORMAT_V2 = 2; - - private final FileFormat format; - - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; - } - - public TestDeltaTaskWriter(String fileFormat) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDeltaTaskWriter extends TestBase { + + @Parameter(index = 1) + private FileFormat format; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created by table create this.metadataDir = new File(tableDir, "metadata"); @@ -170,13 +170,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitioned() throws IOException { createAndInitTable(false); testCdcEvents(false); } - @Test + @TestTemplate public void testPartitioned() throws IOException { createAndInitTable(true); testCdcEvents(true); @@ -201,12 +201,12 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(false); } - @Test + @TestTemplate public void testPartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(true); } @@ -243,17 +243,17 @@ private void testAbort(boolean partitioned) throws IOException { } } - @Test + @TestTemplate public void testUnpartitionedAbort() throws IOException { testAbort(false); } - @Test + @TestTemplate public void testPartitionedAbort() throws IOException { testAbort(true); } - @Test + @TestTemplate public void testPartitionedTableWithDataAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId()); @@ -298,7 +298,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testPartitionedTableWithDataAndIdAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); @@ -323,7 +323,7 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); } - @Test + @TestTemplate public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { Schema tableSchema = new Schema( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java index da45241256f5..414ee40d1357 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkFileWriterFactory extends TestFileWriterFactory { - public TestFlinkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java index 3951c2e70f65..f79a3e634071 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPartitioningWriters extends TestPartitioningWriters { - public TestFlinkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java index 9e846efe6fc9..3050752d1c24 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestFlinkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java index 07716b9c3e60..03051b69cf87 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.FileWriterFactory; @@ -29,10 +28,6 @@ public class TestFlinkRollingFileWriters extends TestRollingFileWriters { - public TestFlinkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index a2fe092b7cf6..4459f224afe6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.NavigableMap; @@ -56,10 +57,13 @@ import org.apache.iceberg.GenericManifestFile; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestHelpers; @@ -73,44 +77,39 @@ import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestIcebergFilesCommitter extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergFilesCommitter extends TestBase { private static final Configuration CONF = new Configuration(); private File flinkManifestFolder; - private final FileFormat format; - private final String branch; - - @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion = {1}, branch = {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, "main"}, - new Object[] {"avro", 2, "test-branch"}, - new Object[] {"parquet", 1, "main"}, - new Object[] {"parquet", 2, "test-branch"}, - new Object[] {"orc", 1, "main"}, - new Object[] {"orc", 2, "test-branch"} - }; - } - - public TestIcebergFilesCommitter(String format, int formatVersion, String branch) { - super(formatVersion); - this.format = FileFormat.fromString(format); - this.branch = branch; + @Parameter(index = 1) + private FileFormat format; + + @Parameter(index = 2) + private String branch; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {2, FileFormat.AVRO, "test-branch"}, + new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {2, FileFormat.PARQUET, "test-branch"}, + new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {2, FileFormat.ORC, "test-branch"}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - flinkManifestFolder = temp.newFolder(); + flinkManifestFolder = Files.createTempDirectory(temp, "flink").toFile(); - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -125,7 +124,7 @@ public void setupTable() throws IOException { .commit(); } - @Test + @TestTemplate public void testCommitTxnWithoutDataFiles() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -156,7 +155,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { } } - @Test + @TestTemplate public void testMaxContinuousEmptyCommits() throws Exception { table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); @@ -182,7 +181,7 @@ private WriteResult of(DataFile dataFile) { return WriteResult.builder().addDataFiles(dataFile).build(); } - @Test + @TestTemplate public void testCommitTxn() throws Exception { // Test with 3 continues checkpoints: // 1. snapshotState for checkpoint#1 @@ -225,7 +224,7 @@ public void testCommitTxn() throws Exception { } } - @Test + @TestTemplate public void testOrderedEventsBetweenCheckpoints() throws Exception { // It's possible that two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -278,7 +277,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testDisorderedEventsBetweenCheckpoints() throws Exception { // It's possible that the two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -331,7 +330,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -392,7 +391,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's // possible that we @@ -490,7 +489,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except } } - @Test + @TestTemplate public void testStartAnotherJobToWriteSameTable() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -557,7 +556,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleJobsWriteSameTable() throws Exception { long timestamp = 0; List tableRows = Lists.newArrayList(); @@ -595,7 +594,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -693,7 +692,7 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; @@ -722,7 +721,7 @@ public void testBoundedStream() throws Exception { } } - @Test + @TestTemplate public void testFlinkManifests() throws Exception { long timestamp = 0; final long checkpoint = 10; @@ -766,7 +765,7 @@ public void testFlinkManifests() throws Exception { } } - @Test + @TestTemplate public void testDeleteFiles() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -835,7 +834,7 @@ public void testDeleteFiles() throws Exception { } } - @Test + @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -888,7 +887,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } - @Test + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; int checkpointId = 0; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 494c633088d9..451e3552b9ac 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -22,7 +22,9 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.time.Duration; +import java.util.Arrays; import java.util.List; import java.util.concurrent.CountDownLatch; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -33,9 +35,11 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -50,13 +54,12 @@ import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingMonitorFunction extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingMonitorFunction extends TestBase { private static final Schema SCHEMA = new Schema( @@ -65,19 +68,15 @@ public class TestStreamingMonitorFunction extends TableTestBase { private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; private static final long WAIT_TIME_MILLIS = 10 * 1000L; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingMonitorFunction(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -99,7 +98,7 @@ private void runSourceFunctionInTask( task.start(); } - @Test + @TestTemplate public void testConsumeWithoutStartSnapshotId() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -122,7 +121,7 @@ public void testConsumeWithoutStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartSnapshotId() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -155,7 +154,7 @@ public void testConsumeFromStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartTag() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -187,7 +186,7 @@ public void testConsumeFromStartTag() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -243,7 +242,7 @@ private void awaitExpectedSplits(TestSourceContext sourceContext) { }); } - @Test + @TestTemplate public void testInvalidMaxPlanningSnapshotCount() { ScanContext scanContext1 = ScanContext.builder() @@ -266,7 +265,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .hasMessage("The max-planning-snapshot-count must be greater than zero"); } - @Test + @TestTemplate public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { generateRecordsAndCommitTxn(10); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index e258a197edf3..f96426a59a2d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -20,6 +20,8 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -32,28 +34,28 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.TestTableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingReaderOperator extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingReaderOperator extends TestBase { private static final Schema SCHEMA = new Schema( @@ -61,19 +63,15 @@ public class TestStreamingReaderOperator extends TableTestBase { Types.NestedField.required(2, "data", Types.StringType.get())); private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingReaderOperator(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -81,7 +79,7 @@ public void setupTable() throws IOException { table = create(SCHEMA, PartitionSpec.unpartitioned()); } - @Test + @TestTemplate public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); @@ -109,7 +107,7 @@ public void testProcessAllRecords() throws Exception { } } - @Test + @TestTemplate public void testTriggerCheckpoint() throws Exception { // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading // records from @@ -150,7 +148,7 @@ public void testTriggerCheckpoint() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 4ecbd1c12921..f780c6135bee 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -45,12 +45,15 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.TestTables; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; @@ -65,31 +68,28 @@ import org.apache.iceberg.util.StructLikeSet; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestDeltaTaskWriter extends TableTestBase { - private static final int FORMAT_V2 = 2; - - private final FileFormat format; - - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; - } - - public TestDeltaTaskWriter(String fileFormat) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDeltaTaskWriter extends TestBase { + + @Parameter(index = 1) + private FileFormat format; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created by table create this.metadataDir = new File(tableDir, "metadata"); @@ -170,13 +170,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitioned() throws IOException { createAndInitTable(false); testCdcEvents(false); } - @Test + @TestTemplate public void testPartitioned() throws IOException { createAndInitTable(true); testCdcEvents(true); @@ -201,12 +201,12 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(false); } - @Test + @TestTemplate public void testPartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(true); } @@ -243,17 +243,17 @@ private void testAbort(boolean partitioned) throws IOException { } } - @Test + @TestTemplate public void testUnpartitionedAbort() throws IOException { testAbort(false); } - @Test + @TestTemplate public void testPartitionedAbort() throws IOException { testAbort(true); } - @Test + @TestTemplate public void testPartitionedTableWithDataAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId()); @@ -298,7 +298,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testPartitionedTableWithDataAndIdAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); @@ -323,7 +323,7 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); } - @Test + @TestTemplate public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { Schema tableSchema = new Schema( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java index da45241256f5..414ee40d1357 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkFileWriterFactory extends TestFileWriterFactory { - public TestFlinkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java index 3951c2e70f65..939ed2be7dbc 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.flink.sink; +import java.util.Arrays; import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,8 +35,12 @@ public class TestFlinkPartitioningWriters extends TestPartitioningWriters { - public TestFlinkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.PARQUET}, + new Object[] {2, FileFormat.ORC}); } @Override diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java index 9e846efe6fc9..3050752d1c24 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestFlinkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java index 07716b9c3e60..03051b69cf87 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.FileWriterFactory; @@ -29,10 +28,6 @@ public class TestFlinkRollingFileWriters extends TestRollingFileWriters { - public TestFlinkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index a2fe092b7cf6..4459f224afe6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.NavigableMap; @@ -56,10 +57,13 @@ import org.apache.iceberg.GenericManifestFile; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestHelpers; @@ -73,44 +77,39 @@ import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestIcebergFilesCommitter extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergFilesCommitter extends TestBase { private static final Configuration CONF = new Configuration(); private File flinkManifestFolder; - private final FileFormat format; - private final String branch; - - @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion = {1}, branch = {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, "main"}, - new Object[] {"avro", 2, "test-branch"}, - new Object[] {"parquet", 1, "main"}, - new Object[] {"parquet", 2, "test-branch"}, - new Object[] {"orc", 1, "main"}, - new Object[] {"orc", 2, "test-branch"} - }; - } - - public TestIcebergFilesCommitter(String format, int formatVersion, String branch) { - super(formatVersion); - this.format = FileFormat.fromString(format); - this.branch = branch; + @Parameter(index = 1) + private FileFormat format; + + @Parameter(index = 2) + private String branch; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {2, FileFormat.AVRO, "test-branch"}, + new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {2, FileFormat.PARQUET, "test-branch"}, + new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {2, FileFormat.ORC, "test-branch"}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - flinkManifestFolder = temp.newFolder(); + flinkManifestFolder = Files.createTempDirectory(temp, "flink").toFile(); - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -125,7 +124,7 @@ public void setupTable() throws IOException { .commit(); } - @Test + @TestTemplate public void testCommitTxnWithoutDataFiles() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -156,7 +155,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { } } - @Test + @TestTemplate public void testMaxContinuousEmptyCommits() throws Exception { table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); @@ -182,7 +181,7 @@ private WriteResult of(DataFile dataFile) { return WriteResult.builder().addDataFiles(dataFile).build(); } - @Test + @TestTemplate public void testCommitTxn() throws Exception { // Test with 3 continues checkpoints: // 1. snapshotState for checkpoint#1 @@ -225,7 +224,7 @@ public void testCommitTxn() throws Exception { } } - @Test + @TestTemplate public void testOrderedEventsBetweenCheckpoints() throws Exception { // It's possible that two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -278,7 +277,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testDisorderedEventsBetweenCheckpoints() throws Exception { // It's possible that the two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -331,7 +330,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -392,7 +391,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's // possible that we @@ -490,7 +489,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except } } - @Test + @TestTemplate public void testStartAnotherJobToWriteSameTable() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -557,7 +556,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleJobsWriteSameTable() throws Exception { long timestamp = 0; List tableRows = Lists.newArrayList(); @@ -595,7 +594,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -693,7 +692,7 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; @@ -722,7 +721,7 @@ public void testBoundedStream() throws Exception { } } - @Test + @TestTemplate public void testFlinkManifests() throws Exception { long timestamp = 0; final long checkpoint = 10; @@ -766,7 +765,7 @@ public void testFlinkManifests() throws Exception { } } - @Test + @TestTemplate public void testDeleteFiles() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -835,7 +834,7 @@ public void testDeleteFiles() throws Exception { } } - @Test + @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -888,7 +887,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } - @Test + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; int checkpointId = 0; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 494c633088d9..451e3552b9ac 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -22,7 +22,9 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.time.Duration; +import java.util.Arrays; import java.util.List; import java.util.concurrent.CountDownLatch; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -33,9 +35,11 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -50,13 +54,12 @@ import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingMonitorFunction extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingMonitorFunction extends TestBase { private static final Schema SCHEMA = new Schema( @@ -65,19 +68,15 @@ public class TestStreamingMonitorFunction extends TableTestBase { private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; private static final long WAIT_TIME_MILLIS = 10 * 1000L; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingMonitorFunction(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -99,7 +98,7 @@ private void runSourceFunctionInTask( task.start(); } - @Test + @TestTemplate public void testConsumeWithoutStartSnapshotId() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -122,7 +121,7 @@ public void testConsumeWithoutStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartSnapshotId() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -155,7 +154,7 @@ public void testConsumeFromStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartTag() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -187,7 +186,7 @@ public void testConsumeFromStartTag() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -243,7 +242,7 @@ private void awaitExpectedSplits(TestSourceContext sourceContext) { }); } - @Test + @TestTemplate public void testInvalidMaxPlanningSnapshotCount() { ScanContext scanContext1 = ScanContext.builder() @@ -266,7 +265,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .hasMessage("The max-planning-snapshot-count must be greater than zero"); } - @Test + @TestTemplate public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { generateRecordsAndCommitTxn(10); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index e258a197edf3..f96426a59a2d 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -20,6 +20,8 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -32,28 +34,28 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.TestTableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingReaderOperator extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingReaderOperator extends TestBase { private static final Schema SCHEMA = new Schema( @@ -61,19 +63,15 @@ public class TestStreamingReaderOperator extends TableTestBase { Types.NestedField.required(2, "data", Types.StringType.get())); private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingReaderOperator(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -81,7 +79,7 @@ public void setupTable() throws IOException { table = create(SCHEMA, PartitionSpec.unpartitioned()); } - @Test + @TestTemplate public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); @@ -109,7 +107,7 @@ public void testProcessAllRecords() throws Exception { } } - @Test + @TestTemplate public void testTriggerCheckpoint() throws Exception { // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading // records from @@ -150,7 +148,7 @@ public void testTriggerCheckpoint() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java index 4a3263e368c0..8f4b3042b1c5 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestFileWriterFactory; @@ -33,10 +32,6 @@ public class TestSparkFileWriterFactory extends TestFileWriterFactory { - public TestSparkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java index 276d8c632fc0..5e8107806a25 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPartitioningWriters; @@ -33,10 +32,6 @@ public class TestSparkPartitioningWriters extends TestPartitioningWriters { - public TestSparkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java index 245c392774f5..baac1efe0e40 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPositionDeltaWriters; @@ -33,10 +32,6 @@ public class TestSparkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestSparkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java index dcf9140a8885..5ebeafcb8cef 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestRollingFileWriters; @@ -30,10 +29,6 @@ public class TestSparkRollingFileWriters extends TestRollingFileWriters { - public TestSparkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java index 4a3263e368c0..8f4b3042b1c5 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestFileWriterFactory; @@ -33,10 +32,6 @@ public class TestSparkFileWriterFactory extends TestFileWriterFactory { - public TestSparkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java index 276d8c632fc0..5e8107806a25 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPartitioningWriters; @@ -33,10 +32,6 @@ public class TestSparkPartitioningWriters extends TestPartitioningWriters { - public TestSparkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java index 245c392774f5..baac1efe0e40 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPositionDeltaWriters; @@ -33,10 +32,6 @@ public class TestSparkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestSparkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java index dcf9140a8885..5ebeafcb8cef 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestRollingFileWriters; @@ -30,10 +29,6 @@ public class TestSparkRollingFileWriters extends TestRollingFileWriters { - public TestSparkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java index 4a3263e368c0..8f4b3042b1c5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestFileWriterFactory; @@ -33,10 +32,6 @@ public class TestSparkFileWriterFactory extends TestFileWriterFactory { - public TestSparkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java index 276d8c632fc0..5e8107806a25 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPartitioningWriters; @@ -33,10 +32,6 @@ public class TestSparkPartitioningWriters extends TestPartitioningWriters { - public TestSparkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java index 245c392774f5..baac1efe0e40 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPositionDeltaWriters; @@ -33,10 +32,6 @@ public class TestSparkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestSparkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java index dcf9140a8885..5ebeafcb8cef 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestRollingFileWriters; @@ -30,10 +29,6 @@ public class TestSparkRollingFileWriters extends TestRollingFileWriters { - public TestSparkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, From 089c9444cedf2ee189f722123075ae0748238e17 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 3 Apr 2024 09:59:13 +0200 Subject: [PATCH 09/11] Build: Ignore link-checking for Blogs / https://search.maven.org/ (#10081) --- site/docs/blogs.md | 71 +++++++++++++++++++++++++++++++++++ site/link-checker-config.json | 3 ++ 2 files changed, 74 insertions(+) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index e7cda2f997a6..4d327596129e 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -28,16 +28,19 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Data Lakehouse Versioning Comparison: (Nessie, Apache Iceberg, LakeFS)](https://www.dremio.com/blog/data-lakehouse-versioning-comparison-nessie-apache-iceberg-lakefs/) **Date**: March 5th, 2024, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [What is Lakehouse Management?: Git-for-Data, Automated Apache Iceberg Table Maintenance and more](https://www.dremio.com/blog/what-is-lakehouse-management-git-for-data-automated-apache-iceberg-table-maintenance-and-more/) **Date**: February 23rd, 2024, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [What is DataOps? Automating Data Management on the Apache Iceberg Lakehouse](https://www.dremio.com/blog/what-is-dataops-automating-data-management-on-the-apache-iceberg-lakehouse/) **Date**: February 23rd, 2024, **Company**: Dremio @@ -49,11 +52,13 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Ingesting Data Into Apache Iceberg Tables with Dremio: A Unified Path to Iceberg](https://www.dremio.com/blog/ingesting-data-into-apache-iceberg-tables-with-dremio-a-unified-path-to-iceberg/) **Date**: February 1st, 2024, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Open Source and the Data Lakehouse: Apache Arrow, Apache Iceberg, Nessie and Dremio](https://www.dremio.com/blog/open-source-and-the-data-lakehouse-apache-arrow-apache-iceberg-nessie-and-dremio/) **Date**: February 1st, 2024, **Company**: Dremio @@ -77,26 +82,31 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Ayush Saxena](https://www.linkedin.com/in/ayush151/) + ### [Getting Started with Flink SQL and Apache Iceberg](https://www.dremio.com/blog/getting-started-with-flink-sql-and-apache-iceberg/) **Date**: August 8th, 2023, **Company**: Dremio **Authors**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) & [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) + ### [Using Flink with Apache Iceberg and Nessie](https://www.dremio.com/blog/using-flink-with-apache-iceberg-and-nessie/) **Date**: July 28th, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [From Hive Tables to Iceberg Tables: Hassle-Free](https://blog.cloudera.com/from-hive-tables-to-iceberg-tables-hassle-free/) **Date**: July 14th, 2023, **Company**: Cloudera **Authors**: [Srinivas Rishindra Pothireddi](https://www.linkedin.com/in/srinivas-rishindra/) + ### [From Hive Tables to Iceberg Tables: Hassle-Free](https://blog.cloudera.com/from-hive-tables-to-iceberg-tables-hassle-free/) **Date**: July 14th, 2023, **Company**: Cloudera **Authors**: [Srinivas Rishindra Pothireddi](https://www.linkedin.com/in/srinivas-rishindra/) + ### [12 Times Faster Query Planning With Iceberg Manifest Caching in Impala](https://blog.cloudera.com/12-times-faster-query-planning-with-iceberg-manifest-caching-in-impala/) **Date**: July 13th, 2023, **Company**: Cloudera @@ -108,46 +118,55 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Rui Li](https://www.linkedin.com/in/rui-li-19282979/) + ### [How to Convert JSON Files Into an Apache Iceberg Table with Dremio](https://www.dremio.com/blog/how-to-convert-json-files-into-an-apache-iceberg-table-with-dremio/) **Date**: May 31st, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Deep Dive Into Configuring Your Apache Iceberg Catalog with Apache Spark](https://www.dremio.com/blog/deep-dive-into-configuring-your-apache-iceberg-catalog-with-apache-spark/) **Date**: May 31st, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Streamlining Data Quality in Apache Iceberg with write-audit-publish & branching](https://www.dremio.com/blog/streamlining-data-quality-in-apache-iceberg-with-write-audit-publish-branching/) **Date**: May 19th, 2023, **Company**: Dremio **Authors**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) & [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) + ### [Introducing the Apache Iceberg Catalog Migration Tool](https://www.dremio.com/blog/introducing-the-apache-iceberg-catalog-migration-tool/) **Date**: May 12th, 2023, **Company**: Dremio **Authors**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) & [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) + ### [3 Ways to Use Python with Apache Iceberg](https://www.dremio.com/blog/3-ways-to-use-python-with-apache-iceberg/) **Date**: April 12th, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [3 Ways to Convert a Delta Lake Table Into an Apache Iceberg Table](https://www.dremio.com/blog/3-ways-to-convert-a-delta-lake-table-into-an-apache-iceberg-table/) **Date**: April 3rd, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [How to Convert CSV Files into an Apache Iceberg table with Dremio](https://www.dremio.com/blog/how-to-convert-csv-files-into-an-apache-iceberg-table-with-dremio/) **Date**: April 3rd, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Open Data Lakehouse powered by Iceberg for all your Data Warehouse needs](https://blog.cloudera.com/open-data-lakehouse-powered-by-iceberg-for-all-your-data-warehouse-needs/) **Date**: April 3rd, 2023, **Company**: Cloudera **Authors**: [Zoltan Borok-Nagy](https://www.linkedin.com/in/zoltán-borók-nagy-7370a65b/), [Ayush Saxena](https://www.linkedin.com/in/ayush-saxena151/), [Tamas Mate](https://www.linkedin.com/in/tmater/), [Simhadri Govindappa](https://www.linkedin.com/in/simhadri-govindappa-1a7788148/) + ### [Exploring Branch & Tags in Apache Iceberg using Spark](https://www.dremio.com/blog/exploring-branch-tags-in-apache-iceberg-using-spark/) **Date**: March 29th, 2022, **Company**: Dremio @@ -159,11 +178,13 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Ron Ortloff](https://www.linkedin.com/in/ron-ortloff/), [Dennis Huo](https://www.linkedin.com/in/dennis-huo-2aaba92a/) + ### [Dealing with Data Incidents Using the Rollback Feature in Apache Iceberg](https://www.dremio.com/blog/dealing-with-data-incidents-using-the-rollback-feature-in-apache-iceberg/) **Date**: February 24th, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Partition and File Pruning for Dremio’s Apache Iceberg-backed Reflections](https://www.dremio.com/blog/partition-and-file-pruning-for-dremios-apache-iceberg-backed-reflections/) **Date**: February 8th, 2022, **Company**: Dremio @@ -193,181 +214,217 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Sumeet Tandure](https://www.linkedin.com/in/sumeettandure/) + ### [Multi-Cloud Open Lakehouse with Apache Iceberg in Cloudera Data Platform](https://blog.cloudera.com/implement-a-multi-cloud-open-lakehouse-with-apache-iceberg-in-cloudera-data-platform/) **Date**: December 15th, 2022, **Company**: Cloudera **Authors**: [Bill Zhang](https://www.linkedin.com/in/billzhang01/), [Shaun Ahmadian](https://www.linkedin.com/in/ssahmadian/), [Zoltan Borok-Nagy](https://www.linkedin.com/in/zoltán-borók-nagy-7370a65b/), [Vincent Kulandaisamy](https://www.linkedin.com/in/vincentkulandaisamy/) + ### [Connecting Tableau to Apache Iceberg Tables with Dremio](https://www.dremio.com/blog/connecting-tableau-to-apache-iceberg-tables-with-dremio/) **Date**: December 15th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Getting Started with Project Nessie, Apache Iceberg, and Apache Spark Using Docker](https://www.dremio.com/blog/getting-started-with-project-nessie-apache-iceberg-and-apache-spark-using-docker/) **Date**: December 15th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Apache Iceberg FAQ](https://www.dremio.com/blog/apache-iceberg-faq/) **Date**: December 14th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [A Notebook for getting started with Project Nessie, Apache Iceberg, and Apache Spark](https://www.dremio.com/blog/a-notebook-for-getting-started-with-project-nessie-apache-iceberg-and-apache-spark/) **Date**: December 5th, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Time Travel with Dremio and Apache Iceberg](https://www.dremio.com/blog/time-travel-with-dremio-and-apache-iceberg/) **Date**: November 29th, 2022, **Company**: Dremio **Author**: [Michael Flower](https://www.linkedin.com/in/michael-flower-b0a3474/) + ### [Compaction in Apache Iceberg: Fine-Tuning Your Iceberg Table's Data Files](https://www.dremio.com/subsurface/compaction-in-apache-iceberg-fine-tuning-your-iceberg-tables-data-files/) **Date**: November 9th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [The Life of a Read Query for Apache Iceberg Tables](https://www.dremio.com/subsurface/the-life-of-a-read-query-for-apache-iceberg-tables/) **Date**: October 31st, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Puffins and Icebergs: Additional Stats for Apache Iceberg Tables](https://www.dremio.com/subsurface/puffins-and-icebergs-additional-stats-for-apache-iceberg-tables/) **Date**: October 17th, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Apache Iceberg and the Right to be Forgotten](https://www.dremio.com/subsurface/apache-iceberg-and-the-right-to-be-forgotten/) **Date**: September 30th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Streaming Data into Apache Iceberg tables using AWS Kinesis and AWS Glue](https://www.dremio.com/subsurface/streaming-data-into-apache-iceberg-tables-using-aws-kinesis-and-aws-glue/) **Date**: September 26th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Iceberg Flink Sink: Stream Directly into your Data Warehouse Tables](https://tabular.io/blog/flink-sink/) **Date**: October 12, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Partitioning for Correctness (and Performance)](https://tabular.io/blog/partitioning/) **Date**: September 28, 2022, **Company**: Tabular **Author**: [Jason Reid](https://www.linkedin.com/in/jasonreid/) + ### [Ensuring High Performance at Any Scale with Apache Iceberg’s Object Store File Layout](https://www.dremio.com/subsurface/ensuring-high-performance-at-any-scale-with-apache-icebergs-object-store-file-layout/) **Date**: September 20, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Introduction to Apache Iceberg Using Spark](https://www.dremio.com/subsurface/introduction-to-apache-iceberg-using-spark/) **Date**: September 15, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [How Z-Ordering in Apache Iceberg Helps Improve Performance](https://www.dremio.com/subsurface/how-z-ordering-in-apache-iceberg-helps-improve-performance/) **Date**: September 13th, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Apache Iceberg 101 – Your Guide to Learning Apache Iceberg Concepts and Practices](https://www.dremio.com/subsurface/apache-iceberg-101-your-guide-to-learning-apache-iceberg-concepts-and-practices/) **Date**: September 12th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [A Hands-On Look at the Structure of an Apache Iceberg Table](https://www.dremio.com/subsurface/a-hands-on-look-at-the-structure-of-an-apache-iceberg-table/) **Date**: August 24, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Future-Proof Partitioning and Fewer Table Rewrites with Apache Iceberg](https://www.dremio.com/subsurface/future-proof-partitioning-and-fewer-table-rewrites-with-apache-iceberg/) **Date**: August 18, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [How to use Apache Iceberg in CDP's Open Lakehouse](https://blog.cloudera.com/how-to-use-apache-iceberg-in-cdps-open-lakehouse/) **Date**: August 8th, 2022, **Company**: Cloudera **Authors**: [Bill Zhang](https://www.linkedin.com/in/billzhang01/), [Peter Ableda](https://www.linkedin.com/in/peterableda), [Shaun Ahmadian](https://www.linkedin.com/in/ssahmadian/), [Manish Maheshwari](https://www.linkedin.com/in/mmaheshwari/) + ### [Near Real-Time Ingestion For Trino](https://www.starburst.io/blog/near-real-time-ingestion-for-trino/) **Date**: August 4th, 2022, **Company**: Starburst **Authors**: [Eric Hwang](https://www.linkedin.com/in/ericwhwang), [Monica Miller](https://www.linkedin.com/in/monica-d-miller), [Brian Zhan](https://www.linkedin.com/in/bzhan) + ### [How to implement Apache Iceberg in AWS Athena](https://big-data-demystified.ninja/2022/07/28/how-to-implement-apache-iceberg-in-aws-athena/) **Date**: July 28th, 2022 **Author**: [Shneior Dicastro] + ### [Supercharge your Data Lakehouse with Apache Iceberg in Cloudera Data Platform](https://blog.cloudera.com/supercharge-your-data-lakehouse-with-apache-iceberg-in-cloudera-data-platform/) **Date**: June 30th, 2022, **Company**: Cloudera **Authors**: [Bill Zhang](https://www.linkedin.com/in/billzhang01/), [Shaun Ahmadian](https://www.linkedin.com/in/ssahmadian/) + ### [Migrating a Hive Table to an Iceberg Table Hands-on Tutorial](https://www.dremio.com/subsurface/migrating-a-hive-table-to-an-iceberg-table-hands-on-tutorial/) **Date**: June 6th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Fewer Accidental Full Table Scans Brought to You by Apache Iceberg’s Hidden Partitioning](https://www.dremio.com/subsurface/fewer-accidental-full-table-scans-brought-to-you-by-apache-icebergs-hidden-partitioning/) **Date**: May 21st, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [An Introduction To The Iceberg Java API Part 2 - Table Scans](https://tabular.io/blog/java-api-part-2/) **Date**: May 11th, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Iceberg's Guiding Light: The Iceberg Open Table Format Specification](https://tabular.io/blog/iceberg-format-version/) **Date**: April 26th, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [How to Migrate a Hive Table to an Iceberg Table](https://www.dremio.com/subsurface/how-to-migrate-a-hive-table-to-an-iceberg-table/) **Date**: April 15th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Using Iceberg's S3FileIO Implementation To Store Your Data In MinIO](https://tabular.io/blog/minio/) **Date**: April 14th, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Maintaining Iceberg Tables – Compaction, Expiring Snapshots, and More](https://www.dremio.com/subsurface/maintaining-iceberg-tables-compaction-expiring-snapshots-and-more/) **Date**: April 7th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [An Introduction To The Iceberg Java API - Part 1](https://tabular.io/blog/java-api-part-1/) **Date**: April 1st, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Integrated Audits: Streamlined Data Observability With Apache Iceberg](https://tabular.io/blog/integrated-audits/) **Date**: March 2nd, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Introducing Apache Iceberg in Cloudera Data Platform](https://blog.cloudera.com/introducing-apache-iceberg-in-cloudera-data-platform/) **Date**: February 23rd, 2022, **Company**: Cloudera **Authors**: [Bill Zhang](https://www.linkedin.com/in/billzhang01/), [Peter Vary](https://www.linkedin.com/in/peter-vary/), [Marton Bod](https://www.linkedin.com/in/martonbod/), [Wing Yew Poon](https://github.com/wypoon) + ### [What's new in Iceberg 0.13](https://tabular.io/blog/whats-new-in-iceberg-0.13/) **Date**: February 22nd, 2022, **Company**: Tabular **Author**: [Ryan Blue](https://www.linkedin.com/in/rdblue/) + ### [Apache Iceberg Becomes Industry Open Standard with Ecosystem Adoption](https://www.dremio.com/apache-iceberg-becomes-industry-open-standard-with-ecosystem-adoption/) **Date**: February 3rd, 2022, **Company**: Dremio **Author**: [Mark Lyons](https://www.linkedin.com/in/markclyons/) + ### [Docker, Spark, and Iceberg: The Fastest Way to Try Iceberg!](https://tabular.io/blog/docker-spark-and-iceberg/) **Date**: February 2nd, 2022, **Company**: Tabular @@ -379,31 +436,37 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [James Malone](https://www.linkedin.com/in/jamesamalone/) + ### [Iceberg FileIO: Cloud Native Tables](https://tabular.io/blog/iceberg-fileio/) **Date**: December 16th, 2021, **Company**: Tabular **Author**: [Daniel Weeks](https://www.linkedin.com/in/daniel-weeks-a1946860/) + ### [Using Spark in EMR with Apache Iceberg](https://tabular.io/blog/emr-spark-and-iceberg/) **Date**: December 10th, 2021, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Metadata Indexing in Iceberg](https://tabular.io/blog/iceberg-metadata-indexing/) **Date**: October 10th, 2021, **Company**: Tabular **Author**: [Ryan Blue](https://www.linkedin.com/in/rdblue/) + ### [Using Debezium to Create a Data Lake with Apache Iceberg](https://debezium.io/blog/2021/10/20/using-debezium-create-data-lake-with-apache-iceberg/) **Date**: October 20th, 2021, **Company**: Memiiso Community **Author**: [Ismail Simsek](https://www.linkedin.com/in/ismailsimsek/) + ### [How to Analyze CDC Data in Iceberg Data Lake Using Flink](https://www.alibabacloud.com/blog/how-to-analyze-cdc-data-in-iceberg-data-lake-using-flink_597838) **Date**: June 15th, 2021, **Company**: Alibaba Cloud Community **Author**: [Li Jinsong](https://www.linkedin.com/in/%E5%8A%B2%E6%9D%BE-%E6%9D%8E-48b54b101/), [Hu Zheng](https://www.linkedin.com/in/zheng-hu-37017683/), [Yang Weihai](https://www.linkedin.com/in/weihai-yang-697a16224/), [Peidan Li](https://www.linkedin.com/in/peidian-li-18938820a/) + ### [Apache Iceberg: An Architectural Look Under the Covers](https://www.dremio.com/apache-iceberg-an-architectural-look-under-the-covers/) **Date**: July 6th, 2021, **Company**: Dremio @@ -415,26 +478,31 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Romin Parekh](https://www.linkedin.com/in/rominparekh/), [Miao Wang](https://www.linkedin.com/in/miao-wang-0406a74/), [Shone Sadler](https://www.linkedin.com/in/shonesadler/) + ### [Flink + Iceberg: How to Construct a Whole-scenario Real-time Data Warehouse](https://www.alibabacloud.com/blog/flink-%2B-iceberg-how-to-construct-a-whole-scenario-real-time-data-warehouse_597824) **Date**: Jun 8th, 2021, **Company**: Tencent **Author** [Shu (Simon Su) Su](https://www.linkedin.com/in/shu-su-62944994/) + ### [Trino on Ice III: Iceberg Concurrency Model, Snapshots, and the Iceberg Spec](https://blog.starburst.io/trino-on-ice-iii-iceberg-concurrency-model-snapshots-and-the-iceberg-spec) **Date**: May 25th, 2021, **Company**: Starburst **Author**: [Brian Olsen](https://www.linkedin.com/in/bitsondatadev) + ### [Trino on Ice II: In-Place Table Evolution and Cloud Compatibility with Iceberg](https://blog.starburst.io/trino-on-ice-ii-in-place-table-evolution-and-cloud-compatibility-with-iceberg) **Date**: May 11th, 2021, **Company**: Starburst **Author**: [Brian Olsen](https://www.linkedin.com/in/bitsondatadev) + ### [Trino On Ice I: A Gentle Introduction To Iceberg](https://blog.starburst.io/trino-on-ice-i-a-gentle-introduction-to-iceberg) **Date**: Apr 27th, 2021, **Company**: Starburst **Author**: [Brian Olsen](https://www.linkedin.com/in/bitsondatadev) + ### [Apache Iceberg: A Different Table Design for Big Data](https://thenewstack.io/apache-iceberg-a-different-table-design-for-big-data/) **Date**: Feb 1st, 2021, **Company**: thenewstack.io @@ -452,6 +520,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Gautam Kowshik](https://www.linkedin.com/in/gautamk/), [Xabriel J. Collazo Mojica](https://www.linkedin.com/in/xabriel/) + ### [FastIngest: Low-latency Gobblin with Apache Iceberg and ORC format](https://engineering.linkedin.com/blog/2021/fastingest-low-latency-gobblin) **Date**: Jan 6th, 2021, **Company**: Linkedin @@ -463,6 +532,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Andrei Ionescu](https://www.linkedin.com/in/andreiionescu), [Shone Sadler](https://www.linkedin.com/in/shonesadler/), [Anil Malkani](https://www.linkedin.com/in/anil-malkani-52861a/) + ### [Optimizing data warehouse storage](https://netflixtechblog.com/optimizing-data-warehouse-storage-7b94a48fdcbe) **Date**: Dec 21st, 2020, **Company**: Netflix @@ -474,6 +544,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Shone Sadler](https://www.linkedin.com/in/shonesadler/), [Romin Parekh](https://www.linkedin.com/in/rominparekh/), [Anil Malkani](https://www.linkedin.com/in/anil-malkani-52861a/) + ### [Bulldozer: Batch Data Moving from Data Warehouse to Online Key-Value Stores](https://netflixtechblog.com/bulldozer-batch-data-moving-from-data-warehouse-to-online-key-value-stores-41bac13863f8) **Date**: Oct 27th, 2020, **Company**: Netflix diff --git a/site/link-checker-config.json b/site/link-checker-config.json index 8eed0c163404..54ff94c0145c 100644 --- a/site/link-checker-config.json +++ b/site/link-checker-config.json @@ -8,6 +8,9 @@ }, { "pattern": "^../../javadoc" + }, + { + "pattern": "^https://search.maven.org/" } ], "replacementPatterns": [ From ced897ca699d4e91ac8da6ac26bec107ee64ee74 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 3 Apr 2024 19:28:29 +0900 Subject: [PATCH 10/11] Core, Data, Flink: Migrate TableTestBase related classes to JUnit5 (#10080) --- .../iceberg/util/TestTableScanUtil.java | 8 +- .../iceberg/io/TestPartitioningWriters.java | 95 +++++++++--------- .../iceberg/io/TestPositionDeltaWriters.java | 36 +++---- .../iceberg/io/TestRollingFileWriters.java | 49 +++++----- .../flink/sink/TestDeltaTaskWriter.java | 97 ++++++++----------- .../flink/sink/TestIcebergFilesCommitter.java | 55 +++++------ .../source/TestStreamingMonitorFunction.java | 16 ++- .../source/TestStreamingReaderOperator.java | 36 ++++--- .../flink/sink/TestDeltaTaskWriter.java | 97 ++++++++----------- .../flink/sink/TestIcebergFilesCommitter.java | 55 +++++------ .../source/TestStreamingMonitorFunction.java | 16 ++- .../source/TestStreamingReaderOperator.java | 36 ++++--- .../flink/sink/TestDeltaTaskWriter.java | 97 ++++++++----------- .../flink/sink/TestIcebergFilesCommitter.java | 55 +++++------ .../source/TestStreamingMonitorFunction.java | 16 ++- .../source/TestStreamingReaderOperator.java | 36 ++++--- 16 files changed, 382 insertions(+), 418 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java index d868fba90190..eb713a4d2e0b 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.util; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; import java.util.Collections; @@ -49,7 +50,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -163,13 +163,13 @@ public void testTaskGroupPlanningCorruptedOffset() { TableScanUtil.planTaskGroups(CloseableIterable.withNoopClose(baseFileScanTasks), 1, 1, 0)) { for (FileScanTask fileScanTask : task.tasks()) { DataFile taskDataFile = fileScanTask.file(); - Assertions.assertThat(taskDataFile.splitOffsets()).isNull(); + assertThat(taskDataFile.splitOffsets()).isNull(); taskCount++; } } // 10 tasks since the split offsets are ignored and there are 1 byte splits for a 10 byte file - Assertions.assertThat(taskCount).isEqualTo(10); + assertThat(taskCount).isEqualTo(10); } @Test @@ -280,7 +280,7 @@ public void testTaskGroupPlanningByPartition() { ImmutableList.of( taskWithPartition(SPEC1, PARTITION1, 128), taskWithPartition(SPEC2, PARTITION2, 128)); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> TableScanUtil.planTaskGroups(tasks2, 128, 10, 4, SPEC2.partitionType())) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot find field"); diff --git a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java index 88d3c16c2dca..8dc031314eda 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java @@ -19,6 +19,7 @@ package org.apache.iceberg.io; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -38,8 +39,6 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -72,7 +71,7 @@ protected FileFormat format() { @BeforeEach public void setupTable() throws Exception { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created during table creation + assertThat(tableDir.delete()).isTrue(); // created during table creation this.metadataDir = new File(tableDir, "metadata"); this.table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -86,10 +85,10 @@ public void testClusteredDataWriterNoRecords() throws IOException { new ClusteredDataWriter<>(writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); } @TestTemplate @@ -111,7 +110,7 @@ public void testClusteredDataWriterMultiplePartitions() throws IOException { writer.close(); DataWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 data files", 3, result.dataFiles().size()); + assertThat(result.dataFiles()).hasSize(3); RowDelta rowDelta = table.newRowDelta(); result.dataFiles().forEach(rowDelta::addRows); @@ -120,7 +119,7 @@ public void testClusteredDataWriterMultiplePartitions() throws IOException { List expectedRows = ImmutableList.of( toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "bbb"), toRow(4, "bbb"), toRow(5, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -139,8 +138,7 @@ public void testClusteredDataWriterOutOfOrderPartitions() throws IOException { writer.write(toRow(4, "bbb"), spec, partitionKey(spec, "bbb")); writer.write(toRow(5, "ccc"), spec, partitionKey(spec, "ccc")); - Assertions.assertThatThrownBy( - () -> writer.write(toRow(6, "aaa"), spec, partitionKey(spec, "aaa"))) + assertThatThrownBy(() -> writer.write(toRow(6, "aaa"), spec, partitionKey(spec, "aaa"))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Encountered records that belong to already closed files") .hasMessageEndingWith("partition 'data=aaa' in spec " + spec); @@ -159,14 +157,14 @@ public void testClusteredEqualityDeleteWriterNoRecords() throws IOException { writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -222,17 +220,16 @@ public void testClusteredEqualityDeleteWriterMultipleSpecs() throws IOException writer.close(); DeleteWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 delete files", 3, result.deleteFiles().size()); - Assert.assertEquals( - "Must not reference data files", 0, writer.result().referencedDataFiles().size()); - Assert.assertFalse("Must not reference data files", writer.result().referencesDataFiles()); + assertThat(result.deleteFiles()).hasSize(3); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); RowDelta rowDelta = table.newRowDelta(); result.deleteFiles().forEach(rowDelta::addDeletes); rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(11, "aaa"), toRow(12, "bbb"), toRow(13, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -265,13 +262,13 @@ public void testClusteredEqualityDeleteWriterOutOfOrderSpecsAndPartitions() thro writer.write(toRow(5, "ccc"), identitySpec, partitionKey(identitySpec, "ccc")); writer.write(toRow(6, "ddd"), identitySpec, partitionKey(identitySpec, "ddd")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> writer.write(toRow(7, "ccc"), identitySpec, partitionKey(identitySpec, "ccc"))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Encountered records that belong to already closed files") .hasMessageEndingWith("partition 'data=ccc' in spec " + identitySpec); - Assertions.assertThatThrownBy(() -> writer.write(toRow(7, "aaa"), unpartitionedSpec, null)) + assertThatThrownBy(() -> writer.write(toRow(7, "aaa"), unpartitionedSpec, null)) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Encountered records that belong to already closed files") .hasMessageEndingWith("spec []"); @@ -297,14 +294,14 @@ private void checkClusteredPositionDeleteWriterNoRecords(DeleteGranularity delet writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE, deleteGranularity); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -373,17 +370,16 @@ private void checkClusteredPositionDeleteWriterMultipleSpecs(DeleteGranularity d writer.close(); DeleteWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 delete files", 3, result.deleteFiles().size()); - Assert.assertEquals( - "Must reference 3 data files", 3, writer.result().referencedDataFiles().size()); - Assert.assertTrue("Must reference data files", writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).hasSize(3); + assertThat(writer.result().referencedDataFiles()).hasSize(3); + assertThat(writer.result().referencesDataFiles()).isTrue(); RowDelta rowDelta = table.newRowDelta(); result.deleteFiles().forEach(rowDelta::addDeletes); rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(11, "aaa"), toRow(12, "bbb"), toRow(13, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -433,7 +429,7 @@ private void checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions( identitySpec, partitionKey(identitySpec, "ddd")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { PositionDelete positionDelete = positionDelete("file-5.parquet", 1L, null); writer.write(positionDelete, identitySpec, partitionKey(identitySpec, "ccc")); @@ -442,7 +438,7 @@ private void checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions( .hasMessageContaining("Encountered records that belong to already closed files") .hasMessageEndingWith("partition 'data=ccc' in spec " + identitySpec); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { PositionDelete positionDelete = positionDelete("file-1.parquet", 3L, null); writer.write(positionDelete, unpartitionedSpec, null); @@ -516,10 +512,10 @@ public void testFanoutDataWriterNoRecords() throws IOException { new FanoutDataWriter<>(writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); } @TestTemplate @@ -541,7 +537,7 @@ public void testFanoutDataWriterMultiplePartitions() throws IOException { writer.close(); DataWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 data files", 3, result.dataFiles().size()); + assertThat(result.dataFiles()).hasSize(3); RowDelta rowDelta = table.newRowDelta(); result.dataFiles().forEach(rowDelta::addRows); @@ -550,7 +546,7 @@ public void testFanoutDataWriterMultiplePartitions() throws IOException { List expectedRows = ImmutableList.of( toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "bbb"), toRow(4, "bbb"), toRow(5, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -571,14 +567,14 @@ private void checkFanoutPositionOnlyDeleteWriterNoRecords(DeleteGranularity dele writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE, deleteGranularity); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -657,17 +653,16 @@ private void checkFanoutPositionOnlyDeleteWriterOutOfOrderRecords( writer.close(); DeleteWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 delete files", 3, result.deleteFiles().size()); - Assert.assertEquals( - "Must reference 3 data files", 3, writer.result().referencedDataFiles().size()); - Assert.assertTrue("Must reference data files", writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).hasSize(3); + assertThat(writer.result().referencedDataFiles()).hasSize(3); + assertThat(writer.result().referencesDataFiles()).isTrue(); RowDelta rowDelta = table.newRowDelta(); result.deleteFiles().forEach(rowDelta::addDeletes); rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(12, "bbb")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate diff --git a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java index 2de7bf7324c0..177982a59cb3 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -34,7 +36,6 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -67,7 +68,7 @@ protected FileFormat format() { @BeforeEach public void setupTable() throws Exception { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created during table creation + assertThat(tableDir.delete()).isTrue(); // created during table creation this.metadataDir = new File(tableDir, "metadata"); this.table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -94,9 +95,9 @@ public void testPositionDeltaWithOneDataWriter() throws IOException { DeleteFile[] deleteFiles = result.deleteFiles(); CharSequence[] referencedDataFiles = result.referencedDataFiles(); - Assert.assertEquals("Must be 1 data files", 1, dataFiles.length); - Assert.assertEquals("Must be no delete files", 0, deleteFiles.length); - Assert.assertEquals("Must not reference data files", 0, referencedDataFiles.length); + assertThat(dataFiles).hasSize(1); + assertThat(deleteFiles).isEmpty(); + assertThat(referencedDataFiles).isEmpty(); } @TestTemplate @@ -121,9 +122,9 @@ public void testPositionDeltaInsertOnly() throws IOException { DeleteFile[] deleteFiles = result.deleteFiles(); CharSequence[] referencedDataFiles = result.referencedDataFiles(); - Assert.assertEquals("Must be 1 data files", 1, dataFiles.length); - Assert.assertEquals("Must be no delete files", 0, deleteFiles.length); - Assert.assertEquals("Must not reference data files", 0, referencedDataFiles.length); + assertThat(dataFiles).hasSize(1); + assertThat(deleteFiles).isEmpty(); + assertThat(referencedDataFiles).isEmpty(); RowDelta rowDelta = table.newRowDelta(); for (DataFile dataFile : dataFiles) { @@ -132,7 +133,7 @@ public void testPositionDeltaInsertOnly() throws IOException { rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(1, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -177,10 +178,9 @@ public void testPositionDeltaDeleteOnly() throws IOException { DeleteFile[] deleteFiles = result.deleteFiles(); CharSequence[] referencedDataFiles = result.referencedDataFiles(); - Assert.assertEquals("Must be 0 data files", 0, dataFiles.length); - Assert.assertEquals("Must be 2 delete files", 2, deleteFiles.length); - Assert.assertEquals("Must reference 2 data files", 2, referencedDataFiles.length); - + assertThat(dataFiles).isEmpty(); + assertThat(deleteFiles).hasSize(2); + assertThat(referencedDataFiles).hasSize(2); RowDelta rowDelta = table.newRowDelta(); for (DeleteFile deleteFile : deleteFiles) { rowDelta.addDeletes(deleteFile); @@ -188,7 +188,7 @@ public void testPositionDeltaDeleteOnly() throws IOException { rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "bbb")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -234,9 +234,9 @@ public void testPositionDeltaMultipleSpecs() throws IOException { DeleteFile[] deleteFiles = result.deleteFiles(); CharSequence[] referencedDataFiles = result.referencedDataFiles(); - Assert.assertEquals("Must be 1 data files", 1, dataFiles.length); - Assert.assertEquals("Must be 2 delete files", 2, deleteFiles.length); - Assert.assertEquals("Must reference 2 data files", 2, referencedDataFiles.length); + assertThat(dataFiles).hasSize(1); + assertThat(deleteFiles).hasSize(2); + assertThat(referencedDataFiles).hasSize(2); RowDelta rowDelta = table.newRowDelta(); for (DataFile dataFile : dataFiles) { @@ -249,6 +249,6 @@ public void testPositionDeltaMultipleSpecs() throws IOException { List expectedRows = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "bbb"), toRow(10, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } } diff --git a/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java b/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java index f6d5d41b72b3..6407fd0cbf70 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -33,7 +35,6 @@ import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -74,7 +75,7 @@ protected FileFormat format() { @BeforeEach public void setupTable() throws Exception { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created during table creation + assertThat(tableDir.delete()).isTrue(); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -97,10 +98,10 @@ public void testRollingDataWriterNoRecords() throws IOException { writerFactory, fileFactory, table.io(), DEFAULT_FILE_SIZE, table.spec(), partition); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); } @TestTemplate @@ -122,7 +123,7 @@ public void testRollingDataWriterSplitData() throws IOException { // call close again to ensure it is idempotent writer.close(); - Assert.assertEquals(4, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).hasSize(4); } @TestTemplate @@ -136,14 +137,14 @@ public void testRollingEqualityDeleteWriterNoRecords() throws IOException { writerFactory, fileFactory, table.io(), DEFAULT_FILE_SIZE, table.spec(), partition); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -168,10 +169,9 @@ public void testRollingEqualityDeleteWriterSplitDeletes() throws IOException { // call close again to ensure it is idempotent writer.close(); - DeleteWriteResult result = writer.result(); - Assert.assertEquals(4, result.deleteFiles().size()); - Assert.assertEquals(0, result.referencedDataFiles().size()); - Assert.assertFalse(result.referencesDataFiles()); + assertThat(writer.result().deleteFiles()).hasSize(4); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -182,14 +182,14 @@ public void testRollingPositionDeleteWriterNoRecords() throws IOException { writerFactory, fileFactory, table.io(), DEFAULT_FILE_SIZE, table.spec(), partition); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -212,9 +212,8 @@ public void testRollingPositionDeleteWriterSplitDeletes() throws IOException { // call close again to ensure it is idempotent writer.close(); - DeleteWriteResult result = writer.result(); - Assert.assertEquals(4, result.deleteFiles().size()); - Assert.assertEquals(1, result.referencedDataFiles().size()); - Assert.assertTrue(result.referencesDataFiles()); + assertThat(writer.result().deleteFiles()).hasSize(4); + assertThat(writer.result().referencedDataFiles()).hasSize(1); + assertThat(writer.result().referencesDataFiles()).isTrue(); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index f780c6135bee..21f3ee2c655a 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -66,8 +67,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -90,7 +89,7 @@ protected static List parameters() { @BeforeEach public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created by table create + assertThat(tableDir.delete()).isTrue(); // created by table create this.metadataDir = new File(tableDir, "metadata"); } @@ -132,18 +131,17 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. WriteResult result = writer.complete(); - Assert.assertEquals(partitioned ? 7 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records.", - expectedRowSet( - createRecord(1, "eee"), - createRecord(2, "ddd"), - createRecord(4, "fff"), - createRecord(5, "ggg")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(1, "eee"), + createRecord(2, "ddd"), + createRecord(4, "fff"), + createRecord(5, "ggg"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -160,14 +158,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(4, "fff")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(partitioned ? 2 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); } @TestTemplate @@ -194,11 +191,11 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(0, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).isEmpty(); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); } @TestTemplate @@ -232,14 +229,11 @@ private void testAbort(boolean partitioned) throws IOException { .filter(p -> p.toFile().isFile()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - "Should have expected file count, but files are: " + files, - partitioned ? 4 : 2, - files.size()); + assertThat(files).hasSize(partitioned ? 4 : 2); writer.abort(); for (Path file : files) { - Assert.assertFalse(Files.exists(file)); + assertThat(file).doesNotExist(); } } @@ -268,14 +262,13 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createInsert(4, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(3, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(3); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -284,18 +277,17 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createDelete(7, "ccc")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(2, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(2); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet( - createRecord(2, "aaa"), - createRecord(5, "aaa"), - createRecord(3, "bbb"), - createRecord(6, "bbb")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(2, "aaa"), + createRecord(5, "aaa"), + createRecord(3, "bbb"), + createRecord(6, "bbb"))); } @TestTemplate @@ -312,15 +304,12 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { writer.write(createDelete(2, "aaa")); // 1 pos-delete. WriteResult result = writer.complete(); - Assert.assertEquals(1, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); - Assert.assertEquals( - Sets.newHashSet(FileContent.POSITION_DELETES), - Sets.newHashSet(result.deleteFiles()[0].content())); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); } @TestTemplate @@ -361,10 +350,10 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { WriteResult result = writer.complete(); // One data file - Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + assertThat(result.dataFiles()).hasSize(1); // One eq delete file + one pos delete file - Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); - Assertions.assertThat( + assertThat(result.deleteFiles()).hasSize(2); + assertThat( Arrays.stream(result.deleteFiles()) .map(ContentFile::content) .collect(Collectors.toSet())) @@ -376,7 +365,7 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { int cutPrecisionNano = start.getNano() / 1000000 * 1000000; expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); - Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); } private void commitTransaction(WriteResult result) { diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 4459f224afe6..06b6bc9a977b 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -75,8 +76,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -111,7 +110,7 @@ public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); @@ -217,9 +216,8 @@ public void testCommitTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); assertSnapshotSize(i); assertMaxCommittedCheckpointId(jobID, operatorId, i); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } } @@ -715,9 +713,8 @@ public void testBoundedStream() throws Exception { SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } @@ -745,16 +742,16 @@ public void testFlinkManifests() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -767,7 +764,9 @@ public void testFlinkManifests() throws Exception { @TestTemplate public void testDeleteFiles() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -792,16 +791,16 @@ public void testDeleteFiles() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -836,7 +835,9 @@ public void testDeleteFiles() throws Exception { @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -882,8 +883,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); assertFlinkManifests(0); - Assert.assertEquals( - "Should have committed 2 txn.", 2, ImmutableList.copyOf(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(2); } } @@ -1047,10 +1047,7 @@ private List assertFlinkManifests(int expectedCount) throws IOException { Files.list(flinkManifestFolder.toPath()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - String.format("Expected %s flink manifests, but the list is: %s", expectedCount, manifests), - expectedCount, - manifests.size()); + assertThat(manifests).hasSize(expectedCount); return manifests; } @@ -1084,12 +1081,12 @@ private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long actualId = IcebergFilesCommitter.getMaxCommittedCheckpointId( table, jobID.toString(), operatorID.toHexString(), branch); - Assert.assertEquals(expectedId, actualId); + assertThat(actualId).isEqualTo(expectedId); } private void assertSnapshotSize(int expectedSnapshotSize) { table.refresh(); - Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index a612617835b0..f7b13598bc2a 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -51,9 +52,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -78,7 +77,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -249,7 +248,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .monitorInterval(Duration.ofMillis(100)) .maxPlanningSnapshotCount(0) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext1)) + assertThatThrownBy(() -> createFunction(scanContext1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); @@ -259,7 +258,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(-10) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext2)) + assertThatThrownBy(() -> createFunction(scanContext2)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); } @@ -282,7 +281,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { FlinkInputSplit[] expectedSplits = FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); - Assert.assertEquals("should produce 9 splits", 9, expectedSplits.length); + assertThat(expectedSplits).hasSize(9); // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the // total splits number @@ -306,10 +305,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { function.monitorAndForwardSplits(); if (maxPlanningSnapshotCount < 10) { - Assert.assertEquals( - "Should produce same splits as max-planning-snapshot-count", - maxPlanningSnapshotCount, - sourceContext.splits.size()); + assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); } } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index f96426a59a2d..1606ee9f9648 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -49,7 +51,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -73,7 +74,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -84,7 +85,7 @@ public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 10, splits.size()); + assertThat(splits).hasSize(10); try (OneInputStreamOperatorTestHarness harness = createReader()) { harness.setup(); @@ -98,7 +99,7 @@ public void testProcessAllRecords() throws Exception { harness.processElement(splits.get(i), -1); // Run the mail-box once to read all records from the given split. - Assert.assertTrue("Should processed 1 split", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); // Assert the output has expected elements. expected.addAll(expectedRecords.get(i)); @@ -115,7 +116,7 @@ public void testTriggerCheckpoint() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(3); List splits = generateSplits(); - Assert.assertEquals("Should have 3 splits", 3, splits.size()); + assertThat(splits).hasSize(3); long timestamp = 0; try (OneInputStreamOperatorTestHarness harness = createReader()) { @@ -131,17 +132,18 @@ public void testTriggerCheckpoint() throws Exception { // Trigger snapshot state, it will start to work once all records from split0 are read. processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); - Assert.assertTrue("Should have processed the split0", processor.runMailboxStep()); - Assert.assertTrue( - "Should have processed the snapshot state action", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); + assertThat(processor.runMailboxStep()) + .as("Should have processed the snapshot state action") + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); // Read records from split1. - Assert.assertTrue("Should have processed the split1", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); // Read records from split2. - Assert.assertTrue("Should have processed the split2", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); TestHelpers.assertRecords( readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); @@ -153,7 +155,7 @@ public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 15, splits.size()); + assertThat(splits).hasSize(15); OperatorSubtaskState state; List expected = Lists.newArrayList(); @@ -170,7 +172,9 @@ public void testCheckpointRestore() throws Exception { SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); for (int i = 0; i < 5; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -190,7 +194,9 @@ public void testCheckpointRestore() throws Exception { for (int i = 5; i < 10; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed one split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -200,7 +206,9 @@ public void testCheckpointRestore() throws Exception { expected.addAll(expectedRecords.get(i)); harness.processElement(splits.get(i), 1); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index f780c6135bee..21f3ee2c655a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -66,8 +67,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -90,7 +89,7 @@ protected static List parameters() { @BeforeEach public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created by table create + assertThat(tableDir.delete()).isTrue(); // created by table create this.metadataDir = new File(tableDir, "metadata"); } @@ -132,18 +131,17 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. WriteResult result = writer.complete(); - Assert.assertEquals(partitioned ? 7 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records.", - expectedRowSet( - createRecord(1, "eee"), - createRecord(2, "ddd"), - createRecord(4, "fff"), - createRecord(5, "ggg")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(1, "eee"), + createRecord(2, "ddd"), + createRecord(4, "fff"), + createRecord(5, "ggg"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -160,14 +158,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(4, "fff")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(partitioned ? 2 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); } @TestTemplate @@ -194,11 +191,11 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(0, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).isEmpty(); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); } @TestTemplate @@ -232,14 +229,11 @@ private void testAbort(boolean partitioned) throws IOException { .filter(p -> p.toFile().isFile()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - "Should have expected file count, but files are: " + files, - partitioned ? 4 : 2, - files.size()); + assertThat(files).hasSize(partitioned ? 4 : 2); writer.abort(); for (Path file : files) { - Assert.assertFalse(Files.exists(file)); + assertThat(file).doesNotExist(); } } @@ -268,14 +262,13 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createInsert(4, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(3, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(3); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -284,18 +277,17 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createDelete(7, "ccc")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(2, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(2); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet( - createRecord(2, "aaa"), - createRecord(5, "aaa"), - createRecord(3, "bbb"), - createRecord(6, "bbb")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(2, "aaa"), + createRecord(5, "aaa"), + createRecord(3, "bbb"), + createRecord(6, "bbb"))); } @TestTemplate @@ -312,15 +304,12 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { writer.write(createDelete(2, "aaa")); // 1 pos-delete. WriteResult result = writer.complete(); - Assert.assertEquals(1, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); - Assert.assertEquals( - Sets.newHashSet(FileContent.POSITION_DELETES), - Sets.newHashSet(result.deleteFiles()[0].content())); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); } @TestTemplate @@ -361,10 +350,10 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { WriteResult result = writer.complete(); // One data file - Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + assertThat(result.dataFiles()).hasSize(1); // One eq delete file + one pos delete file - Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); - Assertions.assertThat( + assertThat(result.deleteFiles()).hasSize(2); + assertThat( Arrays.stream(result.deleteFiles()) .map(ContentFile::content) .collect(Collectors.toSet())) @@ -376,7 +365,7 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { int cutPrecisionNano = start.getNano() / 1000000 * 1000000; expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); - Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); } private void commitTransaction(WriteResult result) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 4459f224afe6..06b6bc9a977b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -75,8 +76,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -111,7 +110,7 @@ public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); @@ -217,9 +216,8 @@ public void testCommitTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); assertSnapshotSize(i); assertMaxCommittedCheckpointId(jobID, operatorId, i); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } } @@ -715,9 +713,8 @@ public void testBoundedStream() throws Exception { SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } @@ -745,16 +742,16 @@ public void testFlinkManifests() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -767,7 +764,9 @@ public void testFlinkManifests() throws Exception { @TestTemplate public void testDeleteFiles() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -792,16 +791,16 @@ public void testDeleteFiles() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -836,7 +835,9 @@ public void testDeleteFiles() throws Exception { @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -882,8 +883,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); assertFlinkManifests(0); - Assert.assertEquals( - "Should have committed 2 txn.", 2, ImmutableList.copyOf(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(2); } } @@ -1047,10 +1047,7 @@ private List assertFlinkManifests(int expectedCount) throws IOException { Files.list(flinkManifestFolder.toPath()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - String.format("Expected %s flink manifests, but the list is: %s", expectedCount, manifests), - expectedCount, - manifests.size()); + assertThat(manifests).hasSize(expectedCount); return manifests; } @@ -1084,12 +1081,12 @@ private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long actualId = IcebergFilesCommitter.getMaxCommittedCheckpointId( table, jobID.toString(), operatorID.toHexString(), branch); - Assert.assertEquals(expectedId, actualId); + assertThat(actualId).isEqualTo(expectedId); } private void assertSnapshotSize(int expectedSnapshotSize) { table.refresh(); - Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 451e3552b9ac..9c4f476b02b4 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -51,9 +52,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -78,7 +77,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -250,7 +249,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(0) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext1)) + assertThatThrownBy(() -> createFunction(scanContext1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); @@ -260,7 +259,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(-10) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext2)) + assertThatThrownBy(() -> createFunction(scanContext2)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); } @@ -283,7 +282,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { FlinkInputSplit[] expectedSplits = FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); - Assert.assertEquals("should produce 9 splits", 9, expectedSplits.length); + assertThat(expectedSplits).hasSize(9); // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the // total splits number @@ -307,10 +306,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { function.monitorAndForwardSplits(); if (maxPlanningSnapshotCount < 10) { - Assert.assertEquals( - "Should produce same splits as max-planning-snapshot-count", - maxPlanningSnapshotCount, - sourceContext.splits.size()); + assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); } } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index f96426a59a2d..1606ee9f9648 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -49,7 +51,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -73,7 +74,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -84,7 +85,7 @@ public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 10, splits.size()); + assertThat(splits).hasSize(10); try (OneInputStreamOperatorTestHarness harness = createReader()) { harness.setup(); @@ -98,7 +99,7 @@ public void testProcessAllRecords() throws Exception { harness.processElement(splits.get(i), -1); // Run the mail-box once to read all records from the given split. - Assert.assertTrue("Should processed 1 split", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); // Assert the output has expected elements. expected.addAll(expectedRecords.get(i)); @@ -115,7 +116,7 @@ public void testTriggerCheckpoint() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(3); List splits = generateSplits(); - Assert.assertEquals("Should have 3 splits", 3, splits.size()); + assertThat(splits).hasSize(3); long timestamp = 0; try (OneInputStreamOperatorTestHarness harness = createReader()) { @@ -131,17 +132,18 @@ public void testTriggerCheckpoint() throws Exception { // Trigger snapshot state, it will start to work once all records from split0 are read. processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); - Assert.assertTrue("Should have processed the split0", processor.runMailboxStep()); - Assert.assertTrue( - "Should have processed the snapshot state action", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); + assertThat(processor.runMailboxStep()) + .as("Should have processed the snapshot state action") + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); // Read records from split1. - Assert.assertTrue("Should have processed the split1", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); // Read records from split2. - Assert.assertTrue("Should have processed the split2", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); TestHelpers.assertRecords( readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); @@ -153,7 +155,7 @@ public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 15, splits.size()); + assertThat(splits).hasSize(15); OperatorSubtaskState state; List expected = Lists.newArrayList(); @@ -170,7 +172,9 @@ public void testCheckpointRestore() throws Exception { SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); for (int i = 0; i < 5; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -190,7 +194,9 @@ public void testCheckpointRestore() throws Exception { for (int i = 5; i < 10; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed one split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -200,7 +206,9 @@ public void testCheckpointRestore() throws Exception { expected.addAll(expectedRecords.get(i)); harness.processElement(splits.get(i), 1); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index f780c6135bee..21f3ee2c655a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -66,8 +67,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -90,7 +89,7 @@ protected static List parameters() { @BeforeEach public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created by table create + assertThat(tableDir.delete()).isTrue(); // created by table create this.metadataDir = new File(tableDir, "metadata"); } @@ -132,18 +131,17 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. WriteResult result = writer.complete(); - Assert.assertEquals(partitioned ? 7 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records.", - expectedRowSet( - createRecord(1, "eee"), - createRecord(2, "ddd"), - createRecord(4, "fff"), - createRecord(5, "ggg")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(1, "eee"), + createRecord(2, "ddd"), + createRecord(4, "fff"), + createRecord(5, "ggg"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -160,14 +158,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(4, "fff")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(partitioned ? 2 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); } @TestTemplate @@ -194,11 +191,11 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(0, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).isEmpty(); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); } @TestTemplate @@ -232,14 +229,11 @@ private void testAbort(boolean partitioned) throws IOException { .filter(p -> p.toFile().isFile()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - "Should have expected file count, but files are: " + files, - partitioned ? 4 : 2, - files.size()); + assertThat(files).hasSize(partitioned ? 4 : 2); writer.abort(); for (Path file : files) { - Assert.assertFalse(Files.exists(file)); + assertThat(file).doesNotExist(); } } @@ -268,14 +262,13 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createInsert(4, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(3, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(3); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -284,18 +277,17 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createDelete(7, "ccc")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(2, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(2); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet( - createRecord(2, "aaa"), - createRecord(5, "aaa"), - createRecord(3, "bbb"), - createRecord(6, "bbb")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(2, "aaa"), + createRecord(5, "aaa"), + createRecord(3, "bbb"), + createRecord(6, "bbb"))); } @TestTemplate @@ -312,15 +304,12 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { writer.write(createDelete(2, "aaa")); // 1 pos-delete. WriteResult result = writer.complete(); - Assert.assertEquals(1, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); - Assert.assertEquals( - Sets.newHashSet(FileContent.POSITION_DELETES), - Sets.newHashSet(result.deleteFiles()[0].content())); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); } @TestTemplate @@ -361,10 +350,10 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { WriteResult result = writer.complete(); // One data file - Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + assertThat(result.dataFiles()).hasSize(1); // One eq delete file + one pos delete file - Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); - Assertions.assertThat( + assertThat(result.deleteFiles()).hasSize(2); + assertThat( Arrays.stream(result.deleteFiles()) .map(ContentFile::content) .collect(Collectors.toSet())) @@ -376,7 +365,7 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { int cutPrecisionNano = start.getNano() / 1000000 * 1000000; expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); - Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); } private void commitTransaction(WriteResult result) { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 4459f224afe6..06b6bc9a977b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -75,8 +76,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -111,7 +110,7 @@ public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); @@ -217,9 +216,8 @@ public void testCommitTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); assertSnapshotSize(i); assertMaxCommittedCheckpointId(jobID, operatorId, i); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } } @@ -715,9 +713,8 @@ public void testBoundedStream() throws Exception { SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } @@ -745,16 +742,16 @@ public void testFlinkManifests() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -767,7 +764,9 @@ public void testFlinkManifests() throws Exception { @TestTemplate public void testDeleteFiles() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -792,16 +791,16 @@ public void testDeleteFiles() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -836,7 +835,9 @@ public void testDeleteFiles() throws Exception { @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -882,8 +883,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); assertFlinkManifests(0); - Assert.assertEquals( - "Should have committed 2 txn.", 2, ImmutableList.copyOf(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(2); } } @@ -1047,10 +1047,7 @@ private List assertFlinkManifests(int expectedCount) throws IOException { Files.list(flinkManifestFolder.toPath()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - String.format("Expected %s flink manifests, but the list is: %s", expectedCount, manifests), - expectedCount, - manifests.size()); + assertThat(manifests).hasSize(expectedCount); return manifests; } @@ -1084,12 +1081,12 @@ private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long actualId = IcebergFilesCommitter.getMaxCommittedCheckpointId( table, jobID.toString(), operatorID.toHexString(), branch); - Assert.assertEquals(expectedId, actualId); + assertThat(actualId).isEqualTo(expectedId); } private void assertSnapshotSize(int expectedSnapshotSize) { table.refresh(); - Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 451e3552b9ac..9c4f476b02b4 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -51,9 +52,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -78,7 +77,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -250,7 +249,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(0) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext1)) + assertThatThrownBy(() -> createFunction(scanContext1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); @@ -260,7 +259,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(-10) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext2)) + assertThatThrownBy(() -> createFunction(scanContext2)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); } @@ -283,7 +282,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { FlinkInputSplit[] expectedSplits = FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); - Assert.assertEquals("should produce 9 splits", 9, expectedSplits.length); + assertThat(expectedSplits).hasSize(9); // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the // total splits number @@ -307,10 +306,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { function.monitorAndForwardSplits(); if (maxPlanningSnapshotCount < 10) { - Assert.assertEquals( - "Should produce same splits as max-planning-snapshot-count", - maxPlanningSnapshotCount, - sourceContext.splits.size()); + assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); } } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index f96426a59a2d..1606ee9f9648 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -49,7 +51,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -73,7 +74,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -84,7 +85,7 @@ public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 10, splits.size()); + assertThat(splits).hasSize(10); try (OneInputStreamOperatorTestHarness harness = createReader()) { harness.setup(); @@ -98,7 +99,7 @@ public void testProcessAllRecords() throws Exception { harness.processElement(splits.get(i), -1); // Run the mail-box once to read all records from the given split. - Assert.assertTrue("Should processed 1 split", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); // Assert the output has expected elements. expected.addAll(expectedRecords.get(i)); @@ -115,7 +116,7 @@ public void testTriggerCheckpoint() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(3); List splits = generateSplits(); - Assert.assertEquals("Should have 3 splits", 3, splits.size()); + assertThat(splits).hasSize(3); long timestamp = 0; try (OneInputStreamOperatorTestHarness harness = createReader()) { @@ -131,17 +132,18 @@ public void testTriggerCheckpoint() throws Exception { // Trigger snapshot state, it will start to work once all records from split0 are read. processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); - Assert.assertTrue("Should have processed the split0", processor.runMailboxStep()); - Assert.assertTrue( - "Should have processed the snapshot state action", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); + assertThat(processor.runMailboxStep()) + .as("Should have processed the snapshot state action") + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); // Read records from split1. - Assert.assertTrue("Should have processed the split1", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); // Read records from split2. - Assert.assertTrue("Should have processed the split2", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); TestHelpers.assertRecords( readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); @@ -153,7 +155,7 @@ public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 15, splits.size()); + assertThat(splits).hasSize(15); OperatorSubtaskState state; List expected = Lists.newArrayList(); @@ -170,7 +172,9 @@ public void testCheckpointRestore() throws Exception { SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); for (int i = 0; i < 5; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -190,7 +194,9 @@ public void testCheckpointRestore() throws Exception { for (int i = 5; i < 10; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed one split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -200,7 +206,9 @@ public void testCheckpointRestore() throws Exception { expected.addAll(expectedRecords.get(i)); harness.processElement(splits.get(i), 1); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } } From c65023b1e740280044da034150eacf745147e26a Mon Sep 17 00:00:00 2001 From: lurnagao-dahua <91278331+lurnagao-dahua@users.noreply.github.com> Date: Wed, 3 Apr 2024 21:27:39 +0800 Subject: [PATCH 11/11] Hive: Avoid NPE on Throwables without error msg (#10069) --- .../iceberg/hive/HiveTableOperations.java | 11 ++++++----- .../apache/iceberg/hive/TestHiveCommits.java | 18 ++++++++++++++++++ 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index bae074d55d24..75d59de75d4d 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -263,11 +263,12 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { throw e; } catch (Throwable e) { - if (e.getMessage() - .contains( - "The table has been modified. The parameter value for key '" - + HiveTableOperations.METADATA_LOCATION_PROP - + "' is")) { + if (e.getMessage() != null + && e.getMessage() + .contains( + "The table has been modified. The parameter value for key '" + + HiveTableOperations.METADATA_LOCATION_PROP + + "' is")) { throw new CommitFailedException( e, "The table %s.%s has been modified concurrently", database, tableName); } diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java index aaa659042118..acf4f8dc5cd3 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java @@ -397,6 +397,24 @@ public void testLockExceptionUnknownSuccessCommit() throws TException, Interrupt .isTrue(); } + @Test + public void testCommitExceptionWithoutMessage() throws TException, InterruptedException { + Table table = catalog.loadTable(TABLE_IDENTIFIER); + HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations(); + + TableMetadata metadataV1 = ops.current(); + table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); + + ops.refresh(); + + HiveTableOperations spyOps = spy(ops); + doThrow(new RuntimeException()).when(spyOps).persistTable(any(), anyBoolean(), any()); + + assertThatThrownBy(() -> spyOps.commit(ops.current(), metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("null\nCannot determine whether the commit was successful or not"); + } + private void commitAndThrowException( HiveTableOperations realOperations, HiveTableOperations spyOperations) throws TException, InterruptedException {