This is an automated email from the ASF dual-hosted git repository.
etudenhoefner pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git
The following commit(s) were added to refs/heads/main by this push:
new 4eef2fe826 Core, Data: Migrate tests to JUnit5 (#10039)
4eef2fe826 is described below
commit 4eef2fe8263f11e8e448a11c4e07acf2cbecda7f
Author: Tom Tanaka <[email protected]>
AuthorDate: Thu Mar 28 23:21:31 2024 +0900
Core, Data: Migrate tests to JUnit5 (#10039)
---
.../org/apache/iceberg/FilterFilesTestBase.java | 11 +-
.../test/java/org/apache/iceberg/TestMetrics.java | 170 ++++---
.../java/org/apache/iceberg/TestWapWorkflow.java | 554 +++++++--------------
.../apache/iceberg/actions/TestCommitService.java | 39 +-
.../iceberg/actions/TestSizeBasedRewriter.java | 26 +-
.../org/apache/iceberg/avro/AvroTestHelpers.java | 24 +-
.../avro/TestNameMappingWithAvroSchema.java | 13 +-
.../apache/iceberg/encryption/TestGcmStreams.java | 104 ++--
.../encryption/TestStandardKeyMetadataParser.java | 13 +-
.../apache/iceberg/io/TestOutputFileFactory.java | 48 +-
.../apache/iceberg/mapping/TestMappingUpdates.java | 172 ++++---
.../apache/iceberg/mapping/TestNameMapping.java | 81 +--
.../org/apache/iceberg/orc/TestOrcMetrics.java | 28 +-
.../apache/iceberg/parquet/TestParquetMetrics.java | 17 +-
14 files changed, 554 insertions(+), 746 deletions(-)
diff --git a/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java
b/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java
index ad92d0f662..bb4bb282a3 100644
--- a/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java
@@ -19,7 +19,7 @@
package org.apache.iceberg;
import static org.apache.iceberg.types.Types.NestedField.required;
-import static org.junit.Assert.assertEquals;
+import static org.assertj.core.api.Assertions.assertThat;
import java.io.File;
import java.io.IOException;
@@ -28,7 +28,6 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Map;
import org.apache.iceberg.expressions.Expressions;
-import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.types.Conversions;
import org.apache.iceberg.types.Types;
@@ -119,10 +118,10 @@ public abstract class FilterFilesTestBase<
table.refresh();
ScanT emptyScan = newScan(table).filter(Expressions.equal("id", 5));
- assertEquals(0, Iterables.size(emptyScan.planFiles()));
+ assertThat(emptyScan.planFiles()).isEmpty();
ScanT nonEmptyScan = newScan(table).filter(Expressions.equal("id", 1));
- assertEquals(1, Iterables.size(nonEmptyScan.planFiles()));
+ assertThat(nonEmptyScan.planFiles()).hasSize(1);
}
private void testCaseInsensitiveFilterFiles(Table table) {
@@ -153,9 +152,9 @@ public abstract class FilterFilesTestBase<
table.refresh();
ScanT emptyScan =
newScan(table).caseSensitive(false).filter(Expressions.equal("ID", 5));
- assertEquals(0, Iterables.size(emptyScan.planFiles()));
+ assertThat(emptyScan.planFiles()).hasSize(0);
ScanT nonEmptyScan =
newScan(table).caseSensitive(false).filter(Expressions.equal("ID", 1));
- assertEquals(1, Iterables.size(nonEmptyScan.planFiles()));
+ assertThat(nonEmptyScan.planFiles()).hasSize(1);
}
}
diff --git a/core/src/test/java/org/apache/iceberg/TestMetrics.java
b/core/src/test/java/org/apache/iceberg/TestMetrics.java
index 32bc6299ce..424e0e0a7b 100644
--- a/core/src/test/java/org/apache/iceberg/TestMetrics.java
+++ b/core/src/test/java/org/apache/iceberg/TestMetrics.java
@@ -21,6 +21,8 @@ package org.apache.iceberg;
import static org.apache.iceberg.types.Conversions.fromByteBuffer;
import static org.apache.iceberg.types.Types.NestedField.optional;
import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
import java.io.File;
import java.io.IOException;
@@ -28,9 +30,11 @@ import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.CharBuffer;
import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
import java.util.List;
import java.util.Map;
-import java.util.Objects;
import org.apache.iceberg.data.GenericRecord;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.io.InputFile;
@@ -56,21 +60,19 @@ import org.apache.iceberg.types.Types.StructType;
import org.apache.iceberg.types.Types.TimeType;
import org.apache.iceberg.types.Types.TimestampType;
import org.apache.iceberg.util.DateTimeUtil;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.io.TempDir;
/** Tests for Metrics. */
public abstract class TestMetrics {
- protected TestMetrics(int formatVersion) {
- this.formatVersion = formatVersion;
+ @Parameters(name = "formatVersion = {0}")
+ public static List<Object> parameters() {
+ return Arrays.asList(1, 2);
}
- @Rule public TemporaryFolder temp = new TemporaryFolder();
+ @TempDir public Path temp;
private static final StructType LEAF_STRUCT_TYPE =
StructType.of(
@@ -113,10 +115,10 @@ public abstract class TestMetrics {
private static final Record NAN_ONLY_RECORD =
createRecordWithFloatAndDouble(Float.NaN, Double.NaN);
- private final int formatVersion;
+ @Parameter private int formatVersion;
private final byte[] fixed = "abcd".getBytes(StandardCharsets.UTF_8);
- @After
+ @AfterEach
public void after() {
TestTables.clearTables();
}
@@ -146,7 +148,7 @@ public abstract class TestMetrics {
protected abstract OutputFile createOutputFile() throws IOException;
- @Test
+ @TestTemplate
public void testMetricsForRepeatedValues() throws IOException {
Record record = GenericRecord.create(SIMPLE_SCHEMA);
record.setField("booleanCol", true);
@@ -164,7 +166,7 @@ public abstract class TestMetrics {
record.setField("timestampColBelowEpoch",
DateTimeUtil.timestampFromMicros(0L));
Metrics metrics = getMetrics(SIMPLE_SCHEMA, record, record);
- Assert.assertEquals(2L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(2L);
assertCounts(1, 2L, 0L, metrics);
assertCounts(2, 2L, 0L, metrics);
assertCounts(3, 2L, 2L, metrics);
@@ -180,7 +182,7 @@ public abstract class TestMetrics {
assertCounts(13, 2L, 0L, metrics);
}
- @Test
+ @TestTemplate
public void testMetricsForTopLevelFields() throws IOException {
Record firstRecord = GenericRecord.create(SIMPLE_SCHEMA);
firstRecord.setField("booleanCol", true);
@@ -212,7 +214,7 @@ public abstract class TestMetrics {
secondRecord.setField("timestampColBelowEpoch",
DateTimeUtil.timestampFromMicros(-7_000L));
Metrics metrics = getMetrics(SIMPLE_SCHEMA, firstRecord, secondRecord);
- Assert.assertEquals(2L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(2L);
assertCounts(1, 2L, 0L, metrics);
assertBounds(1, BooleanType.get(), false, true, metrics);
assertCounts(2, 2L, 0L, metrics);
@@ -255,7 +257,7 @@ public abstract class TestMetrics {
}
}
- @Test
+ @TestTemplate
public void testMetricsForDecimals() throws IOException {
Schema schema =
new Schema(
@@ -269,7 +271,7 @@ public abstract class TestMetrics {
record.setField("decimalAsFixed", new BigDecimal("5.80"));
Metrics metrics = getMetrics(schema, record);
- Assert.assertEquals(1L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(1);
assertCounts(1, 1L, 0L, metrics);
assertBounds(1, DecimalType.of(4, 2), new BigDecimal("2.55"), new
BigDecimal("2.55"), metrics);
assertCounts(2, 1L, 0L, metrics);
@@ -278,10 +280,10 @@ public abstract class TestMetrics {
assertBounds(3, DecimalType.of(22, 2), new BigDecimal("5.80"), new
BigDecimal("5.80"), metrics);
}
- @Test
+ @TestTemplate
public void testMetricsForNestedStructFields() throws IOException {
Metrics metrics = getMetrics(NESTED_SCHEMA, buildNestedTestRecord());
- Assert.assertEquals(1L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(1L);
assertCounts(1, 1L, 0L, metrics);
assertBounds(1, IntegerType.get(), Integer.MAX_VALUE, Integer.MAX_VALUE,
metrics);
assertCounts(3, 1L, 0L, metrics);
@@ -299,7 +301,7 @@ public abstract class TestMetrics {
assertBounds(7, DoubleType.get(), null, null, metrics);
}
- @Test
+ @TestTemplate
public void testMetricsModeForNestedStructFields() throws IOException {
Map<String, String> properties =
ImmutableMap.of(
@@ -310,9 +312,9 @@ public abstract class TestMetrics {
MetricsConfig config = MetricsConfig.fromProperties(properties);
Metrics metrics = getMetrics(NESTED_SCHEMA, config,
buildNestedTestRecord());
- Assert.assertEquals(1L, (long) metrics.recordCount());
- Assert.assertEquals(1, metrics.lowerBounds().size());
- Assert.assertEquals(1, metrics.upperBounds().size());
+ assertThat(metrics.recordCount()).isEqualTo(1L);
+ assertThat(metrics.lowerBounds()).hasSize(1);
+ assertThat(metrics.upperBounds()).hasSize(1);
assertBounds(3, LongType.get(), 100L, 100L, metrics);
}
@@ -331,7 +333,7 @@ public abstract class TestMetrics {
return record;
}
- @Test
+ @TestTemplate
public void testMetricsForListAndMapElements() throws IOException {
StructType structType =
StructType.of(
@@ -352,7 +354,7 @@ public abstract class TestMetrics {
record.set(1, map);
Metrics metrics = getMetrics(schema, record);
- Assert.assertEquals(1L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(1L);
if (fileFormat() != FileFormat.ORC) {
assertCounts(1, 1L, 0L, metrics);
assertCounts(2, 1L, 0L, metrics);
@@ -371,7 +373,7 @@ public abstract class TestMetrics {
assertBounds(7, structType, null, null, metrics);
}
- @Test
+ @TestTemplate
public void testMetricsForNullColumns() throws IOException {
Schema schema = new Schema(optional(1, "intCol", IntegerType.get()));
Record firstRecord = GenericRecord.create(schema);
@@ -380,15 +382,15 @@ public abstract class TestMetrics {
secondRecord.setField("intCol", null);
Metrics metrics = getMetrics(schema, firstRecord, secondRecord);
- Assert.assertEquals(2L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(2L);
assertCounts(1, 2L, 2L, metrics);
assertBounds(1, IntegerType.get(), null, null, metrics);
}
- @Test
+ @TestTemplate
public void testMetricsForNaNColumns() throws IOException {
Metrics metrics = getMetrics(FLOAT_DOUBLE_ONLY_SCHEMA, NAN_ONLY_RECORD,
NAN_ONLY_RECORD);
- Assert.assertEquals(2L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(2L);
assertCounts(1, 2L, 0L, 2L, metrics);
assertCounts(2, 2L, 0L, 2L, metrics);
@@ -396,7 +398,7 @@ public abstract class TestMetrics {
assertBounds(2, DoubleType.get(), null, null, metrics);
}
- @Test
+ @TestTemplate
public void testColumnBoundsWithNaNValueAtFront() throws IOException {
Metrics metrics =
getMetrics(
@@ -404,7 +406,7 @@ public abstract class TestMetrics {
NAN_ONLY_RECORD,
FLOAT_DOUBLE_RECORD_1,
FLOAT_DOUBLE_RECORD_2);
- Assert.assertEquals(3L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(3L);
assertCounts(1, 3L, 0L, 1L, metrics);
assertCounts(2, 3L, 0L, 1L, metrics);
@@ -412,7 +414,7 @@ public abstract class TestMetrics {
assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics);
}
- @Test
+ @TestTemplate
public void testColumnBoundsWithNaNValueInMiddle() throws IOException {
Metrics metrics =
getMetrics(
@@ -420,7 +422,7 @@ public abstract class TestMetrics {
FLOAT_DOUBLE_RECORD_1,
NAN_ONLY_RECORD,
FLOAT_DOUBLE_RECORD_2);
- Assert.assertEquals(3L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(3L);
assertCounts(1, 3L, 0L, 1L, metrics);
assertCounts(2, 3L, 0L, 1L, metrics);
@@ -428,7 +430,7 @@ public abstract class TestMetrics {
assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics);
}
- @Test
+ @TestTemplate
public void testColumnBoundsWithNaNValueAtEnd() throws IOException {
Metrics metrics =
getMetrics(
@@ -436,7 +438,7 @@ public abstract class TestMetrics {
FLOAT_DOUBLE_RECORD_1,
FLOAT_DOUBLE_RECORD_2,
NAN_ONLY_RECORD);
- Assert.assertEquals(3L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(3L);
assertCounts(1, 3L, 0L, 1L, metrics);
assertCounts(2, 3L, 0L, 1L, metrics);
@@ -444,10 +446,11 @@ public abstract class TestMetrics {
assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics);
}
- @Test
+ @TestTemplate
public void testMetricsForTopLevelWithMultipleRowGroup() throws Exception {
- Assume.assumeTrue(
- "Skip test for formats that do not support small row groups",
supportsSmallRowGroups());
+ assumeThat(supportsSmallRowGroups())
+ .as("Skip test for formats that do not support small row groups")
+ .isTrue();
int recordCount = 201;
List<Record> records = Lists.newArrayListWithExpectedSize(recordCount);
@@ -479,11 +482,11 @@ public abstract class TestMetrics {
SIMPLE_SCHEMA, outputFile, records.toArray(new Record[0]));
InputFile recordsFile = outputFile.toInputFile();
- Assert.assertNotNull(recordsFile);
+ assertThat(recordsFile).isNotNull();
// rowgroup size should be > 1
- Assert.assertEquals(3, splitCount(recordsFile));
+ assertThat(splitCount(recordsFile)).isEqualTo(3);
- Assert.assertEquals(201L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(201L);
assertCounts(1, 201L, 0L, metrics);
assertBounds(1, Types.BooleanType.get(), false, true, metrics);
assertBounds(2, Types.IntegerType.get(), 1, 201, metrics);
@@ -498,10 +501,11 @@ public abstract class TestMetrics {
6, Types.DecimalType.of(10, 2), new BigDecimal("2.00"), new
BigDecimal("201.00"), metrics);
}
- @Test
+ @TestTemplate
public void testMetricsForNestedStructFieldsWithMultipleRowGroup() throws
IOException {
- Assume.assumeTrue(
- "Skip test for formats that do not support small row groups",
supportsSmallRowGroups());
+ assumeThat(supportsSmallRowGroups())
+ .as("Skip test for formats that do not support small row groups")
+ .isTrue();
int recordCount = 201;
List<Record> records = Lists.newArrayListWithExpectedSize(recordCount);
@@ -527,11 +531,11 @@ public abstract class TestMetrics {
NESTED_SCHEMA, outputFile, records.toArray(new Record[0]));
InputFile recordsFile = outputFile.toInputFile();
- Assert.assertNotNull(recordsFile);
+ assertThat(recordsFile).isNotNull();
// rowgroup size should be > 1
- Assert.assertEquals(3, splitCount(recordsFile));
+ assertThat(splitCount(recordsFile)).isEqualTo(3);
- Assert.assertEquals(201L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(201L);
assertCounts(1, 201L, 0L, metrics);
assertBounds(1, IntegerType.get(), 1, 201, metrics);
assertCounts(3, 201L, 0L, metrics);
@@ -549,15 +553,15 @@ public abstract class TestMetrics {
assertBounds(7, DoubleType.get(), null, null, metrics);
}
- @Test
+ @TestTemplate
public void testNoneMetricsMode() throws IOException {
Metrics metrics =
getMetrics(
NESTED_SCHEMA,
MetricsConfig.fromProperties(ImmutableMap.of("write.metadata.metrics.default",
"none")),
buildNestedTestRecord());
- Assert.assertEquals(1L, (long) metrics.recordCount());
-
Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull));
+ assertThat(metrics.recordCount()).isEqualTo(1L);
+ assertThat(metrics.columnSizes()).doesNotContainValue(null);
assertCounts(1, null, null, metrics);
assertBounds(1, Types.IntegerType.get(), null, null, metrics);
assertCounts(3, null, null, metrics);
@@ -570,7 +574,7 @@ public abstract class TestMetrics {
assertBounds(7, Types.DoubleType.get(), null, null, metrics);
}
- @Test
+ @TestTemplate
public void testCountsMetricsMode() throws IOException {
Metrics metrics =
getMetrics(
@@ -578,8 +582,8 @@ public abstract class TestMetrics {
MetricsConfig.fromProperties(
ImmutableMap.of("write.metadata.metrics.default", "counts")),
buildNestedTestRecord());
- Assert.assertEquals(1L, (long) metrics.recordCount());
-
Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull));
+ assertThat(metrics.recordCount()).isEqualTo(1L);
+ assertThat(metrics.columnSizes()).doesNotContainValue(null);
assertCounts(1, 1L, 0L, metrics);
assertBounds(1, Types.IntegerType.get(), null, null, metrics);
assertCounts(3, 1L, 0L, metrics);
@@ -592,15 +596,15 @@ public abstract class TestMetrics {
assertBounds(7, Types.DoubleType.get(), null, null, metrics);
}
- @Test
+ @TestTemplate
public void testFullMetricsMode() throws IOException {
Metrics metrics =
getMetrics(
NESTED_SCHEMA,
MetricsConfig.fromProperties(ImmutableMap.of("write.metadata.metrics.default",
"full")),
buildNestedTestRecord());
- Assert.assertEquals(1L, (long) metrics.recordCount());
-
Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull));
+ assertThat(metrics.recordCount()).isEqualTo(1L);
+ assertThat(metrics.columnSizes()).doesNotContainValue(null);
assertCounts(1, 1L, 0L, metrics);
assertBounds(1, Types.IntegerType.get(), Integer.MAX_VALUE,
Integer.MAX_VALUE, metrics);
assertCounts(3, 1L, 0L, metrics);
@@ -618,7 +622,7 @@ public abstract class TestMetrics {
assertBounds(7, Types.DoubleType.get(), null, null, metrics);
}
- @Test
+ @TestTemplate
public void testTruncateStringMetricsMode() throws IOException {
String colName = "str_to_truncate";
Schema singleStringColSchema = new Schema(required(1, colName,
Types.StringType.get()));
@@ -636,13 +640,13 @@ public abstract class TestMetrics {
CharBuffer expectedMinBound = CharBuffer.wrap("Lorem ipsu");
CharBuffer expectedMaxBound = CharBuffer.wrap("Lorem ipsv");
- Assert.assertEquals(1L, (long) metrics.recordCount());
-
Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull));
+ assertThat(metrics.recordCount()).isEqualTo(1L);
+ assertThat(metrics.columnSizes()).doesNotContainValue(null);
assertCounts(1, 1L, 0L, metrics);
assertBounds(1, Types.StringType.get(), expectedMinBound,
expectedMaxBound, metrics);
}
- @Test
+ @TestTemplate
public void testTruncateBinaryMetricsMode() throws IOException {
String colName = "bin_to_truncate";
Schema singleBinaryColSchema = new Schema(required(1, colName,
Types.BinaryType.get()));
@@ -660,16 +664,16 @@ public abstract class TestMetrics {
ByteBuffer expectedMinBounds = ByteBuffer.wrap(new byte[] {0x1, 0x2, 0x3,
0x4, 0x5});
ByteBuffer expectedMaxBounds = ByteBuffer.wrap(new byte[] {0x1, 0x2, 0x3,
0x4, 0x6});
- Assert.assertEquals(1L, (long) metrics.recordCount());
-
Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull));
+ assertThat(metrics.recordCount()).isEqualTo(1L);
+ assertThat(metrics.columnSizes()).doesNotContainValue(null);
assertCounts(1, 1L, 0L, metrics);
assertBounds(1, Types.BinaryType.get(), expectedMinBounds,
expectedMaxBounds, metrics);
}
- @Test
+ @TestTemplate
public void testSortedColumnMetrics() throws IOException {
- File tableDir = temp.newFolder();
- tableDir.delete(); // created by table create
+ File tableDir = Files.createTempDirectory(temp, "junit").toFile();
+ assertThat(tableDir.delete()).isTrue(); // created by table create
SortOrder sortOrder =
SortOrder.builderFor(SIMPLE_SCHEMA)
@@ -719,7 +723,7 @@ public abstract class TestMetrics {
Metrics metrics =
getMetrics(SIMPLE_SCHEMA, MetricsConfig.forTable(table), firstRecord,
secondRecord);
- Assert.assertEquals(2L, (long) metrics.recordCount());
+ assertThat(metrics.recordCount()).isEqualTo(2L);
assertBounds(1, BooleanType.get(), false, true, metrics);
assertBounds(2, IntegerType.get(), Integer.MIN_VALUE, Integer.MAX_VALUE,
metrics);
assertBounds(3, LongType.get(), Long.MIN_VALUE, Long.MAX_VALUE, metrics);
@@ -729,10 +733,10 @@ public abstract class TestMetrics {
assertBounds(8, DateType.get(), 1500, 3000, metrics);
}
- @Test
+ @TestTemplate
public void testMetricsForSortedNestedStructFields() throws IOException {
- File tableDir = temp.newFolder();
- tableDir.delete(); // created by table create
+ File tableDir = Files.createTempDirectory(temp, "junit").toFile();
+ assertThat(tableDir.delete()).isTrue(); // created by table create
SortOrder sortOrder =
SortOrder.builderFor(NESTED_SCHEMA)
@@ -766,12 +770,9 @@ public abstract class TestMetrics {
protected void assertCounts(
int fieldId, Long valueCount, Long nullValueCount, Long nanValueCount,
Metrics metrics) {
- Map<Integer, Long> valueCounts = metrics.valueCounts();
- Map<Integer, Long> nullValueCounts = metrics.nullValueCounts();
- Map<Integer, Long> nanValueCounts = metrics.nanValueCounts();
- Assert.assertEquals(valueCount, valueCounts.get(fieldId));
- Assert.assertEquals(nullValueCount, nullValueCounts.get(fieldId));
- Assert.assertEquals(nanValueCount, nanValueCounts.get(fieldId));
+ assertThat(metrics.valueCounts().get(fieldId)).isEqualTo(valueCount);
+
assertThat(metrics.nullValueCounts().get(fieldId)).isEqualTo(nullValueCount);
+ assertThat(metrics.nanValueCounts().get(fieldId)).isEqualTo(nanValueCount);
}
protected <T> void assertBounds(
@@ -779,11 +780,16 @@ public abstract class TestMetrics {
Map<Integer, ByteBuffer> lowerBounds = metrics.lowerBounds();
Map<Integer, ByteBuffer> upperBounds = metrics.upperBounds();
- Assert.assertEquals(
- lowerBound,
- lowerBounds.containsKey(fieldId) ? fromByteBuffer(type,
lowerBounds.get(fieldId)) : null);
- Assert.assertEquals(
- upperBound,
- upperBounds.containsKey(fieldId) ? fromByteBuffer(type,
upperBounds.get(fieldId)) : null);
+ if (lowerBounds.containsKey(fieldId)) {
+ assertThat((Object) fromByteBuffer(type,
lowerBounds.get(fieldId))).isEqualTo(lowerBound);
+ } else {
+ assertThat(lowerBound).isNull();
+ }
+
+ if (upperBounds.containsKey(fieldId)) {
+ assertThat((Object) fromByteBuffer(type,
upperBounds.get(fieldId))).isEqualTo(upperBound);
+ } else {
+ assertThat(upperBound).isNull();
+ }
}
}
diff --git a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
index c65d469e32..58b83f43f3 100644
--- a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
+++ b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java
@@ -18,35 +18,32 @@
*/
package org.apache.iceberg;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.util.Arrays;
+import java.util.List;
import org.apache.iceberg.exceptions.CherrypickAncestorCommitException;
import org.apache.iceberg.exceptions.DuplicateWAPCommitException;
import org.apache.iceberg.exceptions.ValidationException;
-import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Streams;
-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 TestWapWorkflow extends TableTestBase {
- @Parameterized.Parameters(name = "formatVersion = {0}")
- public static Object[] parameters() {
- return new Object[] {1, 2};
- }
-
- public TestWapWorkflow(int formatVersion) {
- super(formatVersion);
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+@ExtendWith(ParameterizedTestExtension.class)
+public class TestWapWorkflow extends TestBase {
+ @Parameters(name = "formatVersion = {0}")
+ protected static List<Object> parameters() {
+ return Arrays.asList(1, 2);
}
- @Before
+ @BeforeEach
public void setupTableProperties() {
table.updateProperties().set(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED,
"true").commit();
}
- @Test
+ @TestTemplate
public void testCherryPickOverwrite() {
table.newAppend().appendFile(FILE_A).commit();
@@ -68,7 +65,7 @@ public class TestWapWorkflow extends TableTestBase {
validateTableFiles(table, FILE_B);
}
- @Test
+ @TestTemplate
public void testCherryPickOverwriteFailsIfCurrentHasChanged() {
table.newAppend().appendFile(FILE_A).commit();
@@ -87,8 +84,7 @@ public class TestWapWorkflow extends TableTestBase {
.get();
// try to cherry-pick, which should fail because the overwrite's parent is
no longer current
- Assertions.assertThatThrownBy(
- () ->
table.manageSnapshots().cherrypick(overwrite.snapshotId()).commit())
+ assertThatThrownBy(() ->
table.manageSnapshots().cherrypick(overwrite.snapshotId()).commit())
.isInstanceOf(ValidationException.class)
.hasMessage(
"Cannot cherry-pick snapshot 2: not append, dynamic overwrite, or
fast-forward");
@@ -97,7 +93,7 @@ public class TestWapWorkflow extends TableTestBase {
validateTableFiles(table, FILE_A, FILE_C);
}
- @Test
+ @TestTemplate
public void testCurrentSnapshotOperation() {
table.newAppend().appendFile(FILE_A).commit();
@@ -109,38 +105,27 @@ public class TestWapWorkflow extends TableTestBase {
Snapshot wapSnapshot = base.snapshots().get(1);
- Assert.assertEquals("Metadata should have both snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Snapshot should have wap id in summary", "123456789",
wapSnapshot.summary().get("wap.id"));
- Assert.assertEquals(
- "Current snapshot should be first commit's snapshot",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 1,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(wapSnapshot.summary()).containsEntry("wap.id", "123456789");
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(1);
// do setCurrentSnapshot
table.manageSnapshots().setCurrentSnapshot(wapSnapshot.snapshotId()).commit();
base = readMetadata();
- Assert.assertEquals(
- "Current snapshot should be what we rolled back to",
- wapSnapshot.snapshotId(),
- base.currentSnapshot().snapshotId());
- Assert.assertEquals("Metadata should have both snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Should contain manifests for both files",
- 2,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 2,
base.snapshotLog().size());
+
assertThat(base.currentSnapshot().snapshotId()).isEqualTo(wapSnapshot.snapshotId());
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(2);
}
- @Test
+ @TestTemplate
public void testSetCurrentSnapshotNoWAP() {
table.newAppend().appendFile(FILE_A).commit();
@@ -154,24 +139,16 @@ public class TestWapWorkflow extends TableTestBase {
table.manageSnapshots().setCurrentSnapshot(firstSnapshotId).commit();
base = readMetadata();
- Assert.assertEquals(
- "Current snapshot should be what we rolled back to",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals("Metadata should have both snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Should contain manifests for both files",
- 1,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 3,
base.snapshotLog().size());
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(1);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(3);
}
- @Test
+ @TestTemplate
public void testRollbackOnInvalidNonAncestor() {
table.newAppend().appendFile(FILE_A).commit();
@@ -183,42 +160,31 @@ public class TestWapWorkflow extends TableTestBase {
Snapshot wapSnapshot = base.snapshots().get(1);
- Assert.assertEquals("Metadata should have both snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Snapshot should have wap id in summary", "123456789",
wapSnapshot.summary().get("wap.id"));
- Assert.assertEquals(
- "Current snapshot should be first commit's snapshot",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 1,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(wapSnapshot.summary()).containsEntry("wap.id", "123456789");
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(1);
// do rollback
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
// rollback to snapshot that is not an ancestor
() ->
table.manageSnapshots().rollbackTo(wapSnapshot.snapshotId()).commit())
.isInstanceOf(ValidationException.class)
.hasMessage("Cannot roll back to snapshot, not an ancestor of the
current state: 2");
base = readMetadata();
- Assert.assertEquals(
- "Current snapshot should be what we rolled back to",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals("Metadata should have both snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Should contain manifests for one snapshot",
- 1,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 1,
base.snapshotLog().size());
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(1);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(1);
}
- @Test
+ @TestTemplate
public void testRollbackAndCherrypick() {
// first snapshot
table.newAppend().appendFile(FILE_A).commit();
@@ -239,26 +205,22 @@ public class TestWapWorkflow extends TableTestBase {
// rollback to first snapshot
table.manageSnapshots().rollbackTo(firstSnapshotId).commit();
base = readMetadata();
- Assert.assertEquals(
- "Should be at first snapshot", firstSnapshotId,
base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Should have all three snapshots in the system", 3,
base.snapshots().size());
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshots()).hasSize(3);
// fast forward to third snapshot
table.manageSnapshots().cherrypick(thirdSnapshot.snapshotId()).commit();
base = readMetadata();
- Assert.assertEquals(
- "Current state should be at third snapshot", 4,
base.currentSnapshot().snapshotId());
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(4);
// fast forward to 2nd snapshot
table.manageSnapshots().cherrypick(secondSnapshot.snapshotId()).commit();
base = readMetadata();
- Assert.assertEquals(
- "Current state should be at second snapshot", 5,
base.currentSnapshot().snapshotId());
- Assert.assertEquals("Count all snapshots", 5, base.snapshots().size());
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(5);
+ assertThat(base.snapshots()).hasSize(5);
}
- @Test
+ @TestTemplate
public void testRollbackToTime() {
// first snapshot
@@ -279,13 +241,11 @@ public class TestWapWorkflow extends TableTestBase {
table.manageSnapshots().rollbackToTime(secondSnapshot.timestampMillis()).commit();
base = readMetadata();
- Assert.assertEquals(
- "Should be at first snapshot", firstSnapshotId,
base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Should have all three snapshots in the system", 3,
base.snapshots().size());
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshots()).hasSize(3);
}
- @Test
+ @TestTemplate
public void testWithCherryPicking() {
table.newAppend().appendFile(FILE_A).commit();
@@ -299,15 +259,12 @@ public class TestWapWorkflow extends TableTestBase {
// pick the snapshot that's staged but not committed
Snapshot wapSnapshot = base.snapshots().get(1);
- Assert.assertEquals("Should have both snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Should have first wap id in summary", "123456789",
wapSnapshot.summary().get("wap.id"));
- Assert.assertEquals(
- "Current snapshot should be first commit's snapshot",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 1,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(wapSnapshot.summary()).containsEntry("wap.id", "123456789");
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(1);
// cherry-pick snapshot
table.manageSnapshots().cherrypick(wapSnapshot.snapshotId()).commit();
@@ -315,24 +272,16 @@ public class TestWapWorkflow extends TableTestBase {
// check if the effective current snapshot is set to the new snapshot
created
// as a result of the cherry-pick operation
- Assert.assertEquals(
- "Current snapshot should be fast-forwarded to wap snapshot",
- wapSnapshot.snapshotId(),
- base.currentSnapshot().snapshotId());
- Assert.assertEquals("Should have two snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Should contain manifests for both files",
- 2,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 2,
base.snapshotLog().size());
+
assertThat(base.currentSnapshot().snapshotId()).isEqualTo(wapSnapshot.snapshotId());
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(2);
}
- @Test
+ @TestTemplate
public void testWithTwoPhaseCherryPicking() {
table.newAppend().appendFile(FILE_A).commit();
@@ -352,25 +301,15 @@ public class TestWapWorkflow extends TableTestBase {
Snapshot wap1Snapshot = base.snapshots().get(1);
Snapshot wap2Snapshot = base.snapshots().get(2);
- Assert.assertEquals("Should have three snapshots", 3,
base.snapshots().size());
- Assert.assertEquals(
- "Should have first wap id in summary", "123456789",
wap1Snapshot.summary().get("wap.id"));
- Assert.assertEquals(
- "Should have second wap id in summary", "987654321",
wap2Snapshot.summary().get("wap.id"));
- Assert.assertEquals(
- "Current snapshot should be first commit's snapshot",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Parent snapshot id should be same for first WAP snapshot",
- firstSnapshotId,
- wap1Snapshot.parentId().longValue());
- Assert.assertEquals(
- "Parent snapshot id should be same for second WAP snapshot",
- firstSnapshotId,
- wap2Snapshot.parentId().longValue());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 1,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(3);
+ assertThat(wap1Snapshot.summary()).containsEntry("wap.id", "123456789");
+ assertThat(wap2Snapshot.summary()).containsEntry("wap.id", "987654321");
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(wap1Snapshot.parentId()).isEqualTo(firstSnapshotId);
+ assertThat(wap2Snapshot.parentId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(1);
// load current snapshot
parentSnapshot = base.currentSnapshot();
@@ -380,24 +319,15 @@ public class TestWapWorkflow extends TableTestBase {
// check if the effective current snapshot is set to the new snapshot
created
// as a result of the cherry-pick operation
- Assert.assertEquals(
- "Current snapshot should be set to one after wap snapshot",
- parentSnapshot.snapshotId() + 1,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Should contain manifests for both files",
- 2,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Parent snapshot id should change to latest snapshot before commit",
- parentSnapshot.snapshotId(),
- base.currentSnapshot().parentId().longValue());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 2,
base.snapshotLog().size());
+
assertThat(base.currentSnapshot().snapshotId()).isEqualTo(parentSnapshot.snapshotId()
+ 1);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+ assertThat(base.currentSnapshot().parentId())
+ .as("Parent snapshot id should change to latest snapshot before
commit")
+ .isEqualTo(parentSnapshot.snapshotId());
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(2);
// load current snapshot
parentSnapshot = base.currentSnapshot();
@@ -407,27 +337,19 @@ public class TestWapWorkflow extends TableTestBase {
// check if the effective current snapshot is set to the new snapshot
created
// as a result of the cherry-pick operation
- Assert.assertEquals(
- "Current snapshot should be set to one after wap snapshot",
- parentSnapshot.snapshotId() + 1 /* one fast-forwarded snapshot */ + 1,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Should contain manifests for both files",
- 3,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Parent snapshot id should change to latest snapshot before commit",
- parentSnapshot.snapshotId(),
- base.currentSnapshot().parentId().longValue());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 3,
base.snapshotLog().size());
+ assertThat(base.currentSnapshot().snapshotId())
+ .isEqualTo(parentSnapshot.snapshotId() + 1 /* one fast-forwarded
snapshot */ + 1);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(3);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+ assertThat(base.currentSnapshot().parentId())
+ .as("Parent snapshot id should change to latest snapshot before
commit")
+ .isEqualTo(parentSnapshot.snapshotId());
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(3);
}
- @Test
+ @TestTemplate
public void testWithCommitsBetweenCherryPicking() {
table.newAppend().appendFile(FILE_A).commit();
TableMetadata base = readMetadata();
@@ -446,25 +368,15 @@ public class TestWapWorkflow extends TableTestBase {
Snapshot wap1Snapshot = base.snapshots().get(1);
Snapshot wap2Snapshot = base.snapshots().get(2);
- Assert.assertEquals("Should have three snapshots", 3,
base.snapshots().size());
- Assert.assertEquals(
- "Should have first wap id in summary", "123456789",
wap1Snapshot.summary().get("wap.id"));
- Assert.assertEquals(
- "Should have second wap id in summary", "987654321",
wap2Snapshot.summary().get("wap.id"));
- Assert.assertEquals(
- "Current snapshot should be first commit's snapshot",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Parent snapshot id should be same for first WAP snapshot",
- firstSnapshotId,
- wap1Snapshot.parentId().longValue());
- Assert.assertEquals(
- "Parent snapshot id should be same for second WAP snapshot",
- firstSnapshotId,
- wap2Snapshot.parentId().longValue());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 1,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(3);
+ assertThat(wap1Snapshot.summary()).containsEntry("wap.id", "123456789");
+ assertThat(wap2Snapshot.summary()).containsEntry("wap.id", "987654321");
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(wap1Snapshot.parentId()).isEqualTo(firstSnapshotId);
+ assertThat(wap2Snapshot.parentId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(1);
// load current snapshot
parentSnapshot = base.currentSnapshot();
@@ -473,17 +385,12 @@ public class TestWapWorkflow extends TableTestBase {
table.newAppend().appendFile(FILE_D).commit();
base = readMetadata();
- Assert.assertEquals("Should have four snapshots", 4,
base.snapshots().size());
- Assert.assertEquals(
- "Current snapshot should carry over the parent snapshot",
- parentSnapshot.snapshotId(),
- base.currentSnapshot().parentId().longValue());
- Assert.assertEquals(
- "Should contain manifests for two files",
- 2,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 2,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(4);
+
assertThat(base.currentSnapshot().parentId()).isEqualTo(parentSnapshot.snapshotId());
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(2);
// load current snapshot
parentSnapshot = base.currentSnapshot();
@@ -493,25 +400,14 @@ public class TestWapWorkflow extends TableTestBase {
// check if the effective current snapshot is set to the new snapshot
created
// as a result of the cherry-pick operation
- Assert.assertEquals("Should have five snapshots", 5,
base.snapshots().size());
- Assert.assertEquals(
- "Current snapshot should be set to one after wap snapshot",
- parentSnapshot.snapshotId() + 1,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Should contain manifests for three files",
- 3,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Parent snapshot id should point to same snapshot",
- parentSnapshot.snapshotId(),
- base.currentSnapshot().parentId().longValue());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 3,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(5);
+
assertThat(base.currentSnapshot().snapshotId()).isEqualTo(parentSnapshot.snapshotId()
+ 1);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(3);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+
assertThat(base.currentSnapshot().parentId()).isEqualTo(parentSnapshot.snapshotId());
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(3);
// load current snapshot
parentSnapshot = base.currentSnapshot();
@@ -521,28 +417,17 @@ public class TestWapWorkflow extends TableTestBase {
// check if the effective current snapshot is set to the new snapshot
created
// as a result of the cherry-pick operation
- Assert.assertEquals("Should have all the snapshots", 6,
base.snapshots().size());
- Assert.assertEquals(
- "Current snapshot should be set to one after wap snapshot",
- parentSnapshot.snapshotId() + 1,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Should contain manifests for four files",
- 4,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Parent snapshot id should point to same snapshot",
- parentSnapshot.snapshotId(),
- base.currentSnapshot().parentId().longValue());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 4,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(6);
+
assertThat(base.currentSnapshot().snapshotId()).isEqualTo(parentSnapshot.snapshotId()
+ 1);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(4);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+
assertThat(base.currentSnapshot().parentId()).isEqualTo(parentSnapshot.snapshotId());
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(4);
}
- @Test
+ @TestTemplate
public void testWithCherryPickingWithCommitRetry() {
table.newAppend().appendFile(FILE_A).commit();
@@ -559,19 +444,13 @@ public class TestWapWorkflow extends TableTestBase {
// pick the snapshot that's staged but not committed
Snapshot wap1Snapshot = base.snapshots().get(1);
- Assert.assertEquals("Should have two snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Should have first wap id in summary", "123456789",
wap1Snapshot.summary().get("wap.id"));
- Assert.assertEquals(
- "Current snapshot should be first commit's snapshot",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Parent snapshot id should be same for first WAP snapshot",
- firstSnapshotId,
- wap1Snapshot.parentId().longValue());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 1,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(wap1Snapshot.summary()).containsEntry("wap.id", "123456789");
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(wap1Snapshot.parentId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(1);
// load current snapshot
base = readMetadata();
@@ -583,27 +462,16 @@ public class TestWapWorkflow extends TableTestBase {
// check if the effective current snapshot is set to the new snapshot
created
// as a result of the cherry-pick operation
- Assert.assertEquals(
- "Current snapshot should be set to one after wap snapshot",
- parentSnapshot.snapshotId() + 1,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Should contain manifests for both files",
- 2,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should not contain redundant append due to retry",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Parent snapshot id should change to latest snapshot before commit",
- parentSnapshot.snapshotId(),
- base.currentSnapshot().parentId().longValue());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 2,
base.snapshotLog().size());
+
assertThat(base.currentSnapshot().snapshotId()).isEqualTo(parentSnapshot.snapshotId()
+ 1);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+
assertThat(base.currentSnapshot().parentId()).isEqualTo(parentSnapshot.snapshotId());
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(2);
}
- @Test
+ @TestTemplate
public void testCherrypickingAncestor() {
table.newAppend().appendFile(FILE_A).commit();
@@ -617,15 +485,12 @@ public class TestWapWorkflow extends TableTestBase {
// pick the snapshot that's staged but not committed
Snapshot wapSnapshot = base.snapshots().get(1);
- Assert.assertEquals("Should have both snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Should have first wap id in summary", "123456789",
wapSnapshot.summary().get("wap.id"));
- Assert.assertEquals(
- "Current snapshot should be first commit's snapshot",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 1,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(wapSnapshot.summary()).containsEntry("wap.id", "123456789");
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(1);
// cherry-pick snapshot
table.manageSnapshots().cherrypick(wapSnapshot.snapshotId()).commit();
@@ -634,30 +499,22 @@ public class TestWapWorkflow extends TableTestBase {
// check if the effective current snapshot is set to the new snapshot
created
// as a result of the cherry-pick operation
- Assert.assertEquals(
- "Current snapshot should be fast-forwarded to wap snapshot",
- wapSnapshot.snapshotId(),
- base.currentSnapshot().snapshotId());
- Assert.assertEquals("Should have two snapshots", 2,
base.snapshots().size());
- Assert.assertEquals(
- "Should contain manifests for both files",
- 2,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 2,
base.snapshotLog().size());
-
- Assertions.assertThatThrownBy(
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(wapPublishedId);
+ assertThat(base.snapshots()).hasSize(2);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(2);
+
+ assertThatThrownBy(
// duplicate cherry-pick snapshot
() -> table.manageSnapshots().cherrypick(firstSnapshotId).commit())
.isInstanceOf(CherrypickAncestorCommitException.class)
.hasMessage("Cannot cherrypick snapshot 1: already an ancestor");
}
- @Test
+ @TestTemplate
public void testDuplicateCherrypick() {
table.newAppend().appendFile(FILE_A).commit();
TableMetadata base = readMetadata();
@@ -673,39 +530,26 @@ public class TestWapWorkflow extends TableTestBase {
Snapshot wapSnapshot1 = base.snapshots().get(1);
Snapshot wapSnapshot2 = base.snapshots().get(2);
- Assert.assertEquals("Should have both snapshots", 3,
base.snapshots().size());
- Assert.assertEquals(
- "Should have wap id in first wap snapshot summary",
- "123456789",
- wapSnapshot1.summary().get("wap.id"));
- Assert.assertEquals(
- "Should have wap id in second wap snapshot summary",
- "123456789",
- wapSnapshot2.summary().get("wap.id"));
- Assert.assertEquals(
- "Current snapshot should be first commit's snapshot",
- firstSnapshotId,
- base.currentSnapshot().snapshotId());
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 1,
base.snapshotLog().size());
+ assertThat(base.snapshots()).hasSize(3);
+ assertThat(wapSnapshot1.summary()).containsEntry("wap.id", "123456789");
+ assertThat(wapSnapshot2.summary()).containsEntry("wap.id", "123456789");
+ assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(1);
// cherry-pick snapshot
table.manageSnapshots().cherrypick(wapSnapshot1.snapshotId()).commit();
base = readMetadata();
- Assert.assertEquals("Should have three snapshots", 3,
base.snapshots().size());
- Assert.assertEquals(
- "Should contain manifests for both files",
- 2,
- base.currentSnapshot().allManifests(table.io()).size());
- Assert.assertEquals(
- "Should contain append from last commit",
- 1,
- Iterables.size(base.currentSnapshot().addedDataFiles(table.io())));
- Assert.assertEquals(
- "Snapshot log should indicate number of snapshots committed", 2,
base.snapshotLog().size());
-
- Assertions.assertThatThrownBy(
+ assertThat(base.snapshots()).hasSize(3);
+ assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2);
+ assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1);
+ assertThat(base.snapshotLog())
+ .as("Snapshot log should indicate number of snapshots committed")
+ .hasSize(2);
+
+ assertThatThrownBy(
// duplicate cherry-pick snapshot
() ->
table.manageSnapshots().cherrypick(wapSnapshot2.snapshotId()).commit())
.isInstanceOf(DuplicateWAPCommitException.class)
@@ -713,7 +557,7 @@ public class TestWapWorkflow extends TableTestBase {
"Duplicate request to cherry pick wap id that was published
already: 123456789");
}
- @Test
+ @TestTemplate
public void testNonWapCherrypick() {
table.newAppend().appendFile(FILE_A).commit();
TableMetadata base = readMetadata();
@@ -727,41 +571,29 @@ public class TestWapWorkflow extends TableTestBase {
base = readMetadata();
long thirdSnapshotId = base.currentSnapshot().snapshotId();
- Assert.assertEquals(
- "Should be pointing to third snapshot",
- thirdSnapshotId,
- table.currentSnapshot().snapshotId());
+
assertThat(table.currentSnapshot().snapshotId()).isEqualTo(thirdSnapshotId);
// NOOP commit
table.manageSnapshots().commit();
- Assert.assertEquals(
- "Should still be pointing to third snapshot",
- thirdSnapshotId,
- table.currentSnapshot().snapshotId());
+
assertThat(table.currentSnapshot().snapshotId()).isEqualTo(thirdSnapshotId);
// Rollback to second snapshot
table.manageSnapshots().rollbackTo(secondSnapshotId).commit();
- Assert.assertEquals(
- "Should be pointing to second snapshot",
- secondSnapshotId,
- table.currentSnapshot().snapshotId());
+
assertThat(table.currentSnapshot().snapshotId()).isEqualTo(secondSnapshotId);
// Cherrypick down to third
table.manageSnapshots().cherrypick(thirdSnapshotId).commit();
- Assert.assertEquals(
- "Should be re-using wap snapshot after cherrypick",
- 3,
- table.currentSnapshot().snapshotId());
+ assertThat(table.currentSnapshot().snapshotId()).isEqualTo(3);
// try double cherrypicking of the third snapshot
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
// double cherrypicking of second snapshot
() -> table.manageSnapshots().cherrypick(thirdSnapshotId).commit())
.isInstanceOf(CherrypickAncestorCommitException.class)
.hasMessage("Cannot cherrypick snapshot 3: already an ancestor");
// try cherrypicking an ancestor
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
// double cherrypicking of second snapshot
() -> table.manageSnapshots().cherrypick(firstSnapshotId).commit())
.isInstanceOf(CherrypickAncestorCommitException.class)
diff --git
a/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java
b/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java
index 1aae648333..2d2e78a815 100644
--- a/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java
+++ b/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java
@@ -19,31 +19,36 @@
package org.apache.iceberg.actions;
import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;
+import java.util.Arrays;
+import java.util.List;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.stream.IntStream;
+import org.apache.iceberg.ParameterizedTestExtension;
+import org.apache.iceberg.Parameters;
import org.apache.iceberg.Table;
-import org.apache.iceberg.TableTestBase;
-import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
-import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.TestBase;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.util.Tasks;
-import org.assertj.core.api.Assertions;
import org.awaitility.Awaitility;
-import org.junit.Test;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
-public class TestCommitService extends TableTestBase {
+@ExtendWith(ParameterizedTestExtension.class)
+public class TestCommitService extends TestBase {
- public TestCommitService() {
- super(1);
+ @Parameters(name = "formatVersion = {0}")
+ protected static List<Object> parameters() {
+ return Arrays.asList(1);
}
- @Test
+ @TestTemplate
public void testCommittedResultsCorrectly() {
CustomCommitService commitService = new CustomCommitService(table, 5,
10000);
commitService.start();
@@ -55,10 +60,10 @@ public class TestCommitService extends TableTestBase {
Set<Integer> expected = Sets.newHashSet(IntStream.range(0,
100).iterator());
Set<Integer> actual = Sets.newHashSet(commitService.results());
- Assertions.assertThat(actual).isEqualTo(expected);
+ assertThat(actual).isEqualTo(expected);
}
- @Test
+ @TestTemplate
public void testAbortFileGroupsAfterTimeout() {
CustomCommitService commitService = new CustomCommitService(table, 5, 200);
commitService.start();
@@ -90,7 +95,7 @@ public class TestCommitService extends TableTestBase {
// simulate timeout on the main thread, which then tries to abort file
groups [5-7].
// This tests the race conditions, as the committerService is also trying
to commit groups
// [5-7].
- Assertions.assertThatThrownBy(commitService::close)
+ assertThatThrownBy(commitService::close)
.isInstanceOf(IllegalArgumentException.class)
.hasMessageContaining("Timeout occurred when waiting for commits");
@@ -102,14 +107,12 @@ public class TestCommitService extends TableTestBase {
.untilAsserted(() ->
assertThat(commitService.completedRewritesAllCommitted()).isTrue());
if (commitService.aborted.isEmpty()) {
// All file groups are committed
- Assertions.assertThat(commitService.results())
- .isEqualTo(ImmutableList.of(0, 1, 2, 3, 4, 5, 6, 7));
+ assertThat(commitService.results()).containsExactly(0, 1, 2, 3, 4, 5, 6,
7);
} else {
// File groups [5-7] are aborted
- Assertions.assertThat(commitService.results())
- .doesNotContainAnyElementsOf(commitService.aborted);
-
Assertions.assertThat(commitService.results()).isEqualTo(ImmutableList.of(0, 1,
2, 3, 4));
-
Assertions.assertThat(commitService.aborted).isEqualTo(ImmutableSet.of(5, 6,
7));
+
assertThat(commitService.results()).doesNotContainAnyElementsOf(commitService.aborted);
+ assertThat(commitService.results()).containsExactly(0, 1, 2, 3, 4);
+ assertThat(commitService.aborted).containsExactly(5, 6, 7);
}
}
diff --git
a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java
b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java
index c33bbc6f6d..a9a23d587a 100644
--- a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java
+++ b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java
@@ -20,33 +20,31 @@ package org.apache.iceberg.actions;
import static org.assertj.core.api.Assertions.assertThat;
+import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.MockFileScanTask;
+import org.apache.iceberg.ParameterizedTestExtension;
+import org.apache.iceberg.Parameters;
import org.apache.iceberg.Table;
-import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.TestBase;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
-public class TestSizeBasedRewriter extends TableTestBase {
+@ExtendWith(ParameterizedTestExtension.class)
+public class TestSizeBasedRewriter extends TestBase {
- @Parameterized.Parameters(name = "formatVersion = {0}")
- public static Object[] parameters() {
- return new Object[] {1, 2};
+ @Parameters(name = "formatVersion = {0}")
+ protected static List<Object> parameters() {
+ return Arrays.asList(1, 2);
}
- public TestSizeBasedRewriter(int formatVersion) {
- super(formatVersion);
- }
-
- @Test
+ @TestTemplate
public void testSplitSizeLowerBound() {
SizeBasedDataFileRewriterImpl rewriter = new
SizeBasedDataFileRewriterImpl(table);
diff --git a/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java
b/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java
index af35e27f5b..03108376eb 100644
--- a/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java
+++ b/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java
@@ -19,6 +19,7 @@
package org.apache.iceberg.avro;
import static org.apache.iceberg.avro.AvroSchemaUtil.toOption;
+import static org.assertj.core.api.Assertions.assertThat;
import java.util.Arrays;
import java.util.List;
@@ -28,7 +29,6 @@ import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData.Record;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
-import org.assertj.core.api.Assertions;
class AvroTestHelpers {
@@ -81,7 +81,7 @@ class AvroTestHelpers {
static void assertEquals(Types.ListType list, List<?> expected, List<?>
actual) {
Type elementType = list.elementType();
- Assertions.assertThat(actual).as("List size should
match").hasSameSizeAs(expected);
+ assertThat(actual).as("List size should match").hasSameSizeAs(expected);
for (int i = 0; i < expected.size(); i += 1) {
Object expectedValue = expected.get(i);
@@ -94,7 +94,7 @@ class AvroTestHelpers {
static void assertEquals(Types.MapType map, Map<?, ?> expected, Map<?, ?>
actual) {
Type valueType = map.valueType();
- Assertions.assertThat(actual).as("Map keys should
match").hasSameSizeAs(expected);
+ assertThat(actual).as("Map keys should match").hasSameSizeAs(expected);
for (Object expectedKey : expected.keySet()) {
Object expectedValue = expected.get(expectedKey);
@@ -123,25 +123,21 @@ class AvroTestHelpers {
case FIXED:
case BINARY:
case DECIMAL:
- Assertions.assertThat(actual)
- .as("Primitive value should be equal to expected")
- .isEqualTo(expected);
+ assertThat(actual).as("Primitive value should be equal to
expected").isEqualTo(expected);
break;
case STRUCT:
- Assertions.assertThat(expected)
- .as("Expected should be a Record")
- .isInstanceOf(Record.class);
- Assertions.assertThat(actual).as("Actual should be a
Record").isInstanceOf(Record.class);
+ assertThat(expected).as("Expected should be a
Record").isInstanceOf(Record.class);
+ assertThat(actual).as("Actual should be a
Record").isInstanceOf(Record.class);
assertEquals(type.asStructType(), (Record) expected, (Record) actual);
break;
case LIST:
- Assertions.assertThat(expected).as("Expected should be a
List").isInstanceOf(List.class);
- Assertions.assertThat(actual).as("Actual should be a
List").isInstanceOf(List.class);
+ assertThat(expected).as("Expected should be a
List").isInstanceOf(List.class);
+ assertThat(actual).as("Actual should be a
List").isInstanceOf(List.class);
assertEquals(type.asListType(), (List) expected, (List) actual);
break;
case MAP:
- Assertions.assertThat(expected).as("Expected should be a
Map").isInstanceOf(Map.class);
- Assertions.assertThat(actual).as("Actual should be a
Map").isInstanceOf(Map.class);
+ assertThat(expected).as("Expected should be a
Map").isInstanceOf(Map.class);
+ assertThat(actual).as("Actual should be a
Map").isInstanceOf(Map.class);
assertEquals(type.asMapType(), (Map<?, ?>) expected, (Map<?, ?>)
actual);
break;
default:
diff --git
a/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java
b/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java
index 9c8ea8453d..686456b3b1 100644
---
a/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java
+++
b/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java
@@ -18,12 +18,13 @@
*/
package org.apache.iceberg.avro;
+import static org.assertj.core.api.Assertions.assertThat;
+
import org.apache.avro.Schema;
import org.apache.iceberg.mapping.MappedField;
import org.apache.iceberg.mapping.MappedFields;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
public class TestNameMappingWithAvroSchema {
@Test
@@ -132,9 +133,9 @@ public class TestNameMappingWithAvroSchema {
MappedField.of(13, "string"),
MappedField.of(14, "int"))))),
MappedField.of(21, "timezone"),
MappedField.of(22, "bitmap"))));
- Assert.assertEquals(
- expected,
- AvroWithPartnerByStructureVisitor.visit(
- icebergSchema.asStruct(), schema, nameMappingWithAvroSchema));
+ assertThat(
+ AvroWithPartnerByStructureVisitor.visit(
+ icebergSchema.asStruct(), schema, nameMappingWithAvroSchema))
+ .isEqualTo(expected);
}
}
diff --git
a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java
b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java
index a954cf760b..af910f9fdd 100644
--- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java
+++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java
@@ -18,10 +18,14 @@
*/
package org.apache.iceberg.encryption;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
+import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import java.util.Arrays;
import java.util.Random;
@@ -29,15 +33,12 @@ import javax.crypto.AEADBadTagException;
import org.apache.iceberg.Files;
import org.apache.iceberg.io.PositionOutputStream;
import org.apache.iceberg.io.SeekableInputStream;
-import org.assertj.core.api.Assertions;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
public class TestGcmStreams {
- @Rule public TemporaryFolder temp = new TemporaryFolder();
+ @TempDir private Path temp;
@Test
public void testEmptyFile() throws IOException {
@@ -48,7 +49,7 @@ public class TestGcmStreams {
random.nextBytes(aadPrefix);
byte[] readBytes = new byte[1];
- File testFile = temp.newFile();
+ File testFile = File.createTempFile("test", null, temp.toFile());
AesGcmOutputFile encryptedFile =
new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix);
@@ -56,20 +57,20 @@ public class TestGcmStreams {
encryptedStream.close();
AesGcmInputFile decryptedFile = new
AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
- Assert.assertEquals("File size", 0, decryptedFile.getLength());
+ assertThat(decryptedFile.getLength()).isEqualTo(0);
try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
- Assert.assertEquals("Read empty stream", -1,
decryptedStream.read(readBytes));
+ assertThat(decryptedStream.read(readBytes)).as("Read empty
stream").isEqualTo(-1);
}
// check that the AAD is still verified, even for an empty file
byte[] badAAD = Arrays.copyOf(aadPrefix, aadPrefix.length);
badAAD[1] -= 1; // modify the AAD slightly
AesGcmInputFile badAADFile = new
AesGcmInputFile(Files.localInput(testFile), key, badAAD);
- Assert.assertEquals("File size", 0, badAADFile.getLength());
+ assertThat(badAADFile.getLength()).isEqualTo(0);
try (SeekableInputStream decryptedStream = badAADFile.newStream()) {
- Assertions.assertThatThrownBy(() -> decryptedStream.read(readBytes))
+ assertThatThrownBy(() -> decryptedStream.read(readBytes))
.isInstanceOf(RuntimeException.class)
.hasCauseInstanceOf(AEADBadTagException.class)
.hasMessageContaining("GCM tag check failed");
@@ -86,7 +87,7 @@ public class TestGcmStreams {
byte[] content = new byte[Ciphers.PLAIN_BLOCK_SIZE / 2]; // half a block
random.nextBytes(content);
- File testFile = temp.newFile();
+ File testFile = File.createTempFile("test", null, temp.toFile());
AesGcmOutputFile encryptedFile =
new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix);
@@ -96,27 +97,24 @@ public class TestGcmStreams {
// verify the data can be read correctly with the right AAD
AesGcmInputFile decryptedFile = new
AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
- Assert.assertEquals("File size", content.length,
decryptedFile.getLength());
+ assertThat(decryptedFile.getLength()).isEqualTo(content.length);
try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE];
int bytesRead = decryptedStream.read(readContent);
- Assert.assertEquals("Bytes read should match bytes written",
content.length, bytesRead);
- Assert.assertEquals(
- "Content should match",
- ByteBuffer.wrap(content),
- ByteBuffer.wrap(readContent, 0, bytesRead));
+ assertThat(bytesRead).as("Bytes read should match bytes
written").isEqualTo(content.length);
+ assertThat(ByteBuffer.wrap(readContent, 0,
bytesRead)).isEqualTo(ByteBuffer.wrap(content));
}
// test with the wrong AAD
byte[] badAAD = Arrays.copyOf(aadPrefix, aadPrefix.length);
badAAD[1] -= 1; // modify the AAD slightly
AesGcmInputFile badAADFile = new
AesGcmInputFile(Files.localInput(testFile), key, badAAD);
- Assert.assertEquals("File size", content.length, badAADFile.getLength());
+ assertThat(badAADFile.getLength()).isEqualTo(content.length);
try (SeekableInputStream decryptedStream = badAADFile.newStream()) {
byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE];
- Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent))
+ assertThatThrownBy(() -> decryptedStream.read(readContent))
.isInstanceOf(RuntimeException.class)
.hasCauseInstanceOf(AEADBadTagException.class)
.hasMessageContaining("GCM tag check failed");
@@ -132,7 +130,7 @@ public class TestGcmStreams {
// read with the correct AAD and verify the tag check fails
try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE];
- Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent))
+ assertThatThrownBy(() -> decryptedStream.read(readContent))
.isInstanceOf(RuntimeException.class)
.hasCauseInstanceOf(AEADBadTagException.class)
.hasMessageContaining("GCM tag check failed");
@@ -149,7 +147,7 @@ public class TestGcmStreams {
byte[] content = new byte[Ciphers.PLAIN_BLOCK_SIZE / 2]; // half a block
random.nextBytes(content);
- File testFile = temp.newFile();
+ File testFile = File.createTempFile("test", null, temp.toFile());
AesGcmOutputFile encryptedFile =
new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix);
@@ -159,16 +157,13 @@ public class TestGcmStreams {
// verify the data can be read correctly with the right AAD
AesGcmInputFile decryptedFile = new
AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
- Assert.assertEquals("File size", content.length,
decryptedFile.getLength());
+ assertThat(decryptedFile.getLength()).isEqualTo(content.length);
try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE];
int bytesRead = decryptedStream.read(readContent);
- Assert.assertEquals("Bytes read should match bytes written",
content.length, bytesRead);
- Assert.assertEquals(
- "Content should match",
- ByteBuffer.wrap(content),
- ByteBuffer.wrap(readContent, 0, bytesRead));
+ assertThat(bytesRead).as("Bytes read should match bytes
written").isEqualTo(content.length);
+ assertThat(ByteBuffer.wrap(readContent, 0,
bytesRead)).isEqualTo(ByteBuffer.wrap(content));
}
// replace the first block's nonce
@@ -181,7 +176,7 @@ public class TestGcmStreams {
// read with the correct AAD and verify the read fails
try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE];
- Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent))
+ assertThatThrownBy(() -> decryptedStream.read(readContent))
.isInstanceOf(RuntimeException.class)
.hasCauseInstanceOf(AEADBadTagException.class)
.hasMessageContaining("GCM tag check failed");
@@ -198,7 +193,7 @@ public class TestGcmStreams {
byte[] content = new byte[Ciphers.PLAIN_BLOCK_SIZE / 2]; // half a block
random.nextBytes(content);
- File testFile = temp.newFile();
+ File testFile = File.createTempFile("test", null, temp.toFile());
AesGcmOutputFile encryptedFile =
new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix);
@@ -208,16 +203,13 @@ public class TestGcmStreams {
// verify the data can be read correctly with the right AAD
AesGcmInputFile decryptedFile = new
AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
- Assert.assertEquals("File size", content.length,
decryptedFile.getLength());
+ assertThat(decryptedFile.getLength()).isEqualTo(content.length);
try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE];
int bytesRead = decryptedStream.read(readContent);
- Assert.assertEquals("Bytes read should match bytes written",
content.length, bytesRead);
- Assert.assertEquals(
- "Content should match",
- ByteBuffer.wrap(content),
- ByteBuffer.wrap(readContent, 0, bytesRead));
+ assertThat(bytesRead).as("Bytes read should match bytes
written").isEqualTo(content.length);
+ assertThat(ByteBuffer.wrap(readContent, 0,
bytesRead)).isEqualTo(ByteBuffer.wrap(content));
}
// replace part of the first block's content
@@ -230,7 +222,7 @@ public class TestGcmStreams {
// read with the correct AAD and verify the read fails
try (SeekableInputStream decryptedStream = decryptedFile.newStream()) {
byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE];
- Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent))
+ assertThatThrownBy(() -> decryptedStream.read(readContent))
.isInstanceOf(RuntimeException.class)
.hasCauseInstanceOf(AEADBadTagException.class)
.hasMessageContaining("GCM tag check failed");
@@ -260,7 +252,7 @@ public class TestGcmStreams {
byte[] key = new byte[keyLength];
random.nextBytes(key);
random.nextBytes(aadPrefix);
- File testFile = temp.newFile();
+ File testFile = File.createTempFile("test", null, temp.toFile());
AesGcmOutputFile encryptedFile =
new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix);
@@ -277,17 +269,19 @@ public class TestGcmStreams {
}
encryptedStream.write(testFileContents, offset, chunkLen);
offset += chunkLen;
- Assert.assertEquals("Position", offset, encryptedStream.getPos());
+ assertThat(encryptedStream.getPos()).isEqualTo(offset);
left -= chunkLen;
}
encryptedStream.close();
- Assert.assertEquals("Final position in closed stream", offset,
encryptedStream.getPos());
+ assertThat(encryptedStream.getPos())
+ .as("Final position in closed stream")
+ .isEqualTo(offset);
AesGcmInputFile decryptedFile =
new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
SeekableInputStream decryptedStream = decryptedFile.newStream();
- Assert.assertEquals("File size", testFileSize,
decryptedFile.getLength());
+ assertThat(decryptedFile.getLength()).isEqualTo(testFileSize);
byte[] chunk = new byte[testFileSize];
@@ -303,26 +297,26 @@ public class TestGcmStreams {
decryptedStream.seek(pos);
int len = decryptedStream.read(chunk, 0, chunkLen);
- Assert.assertEquals("Read length", len, chunkLen);
+ assertThat(chunkLen).isEqualTo(len);
long pos2 = decryptedStream.getPos();
- Assert.assertEquals("Position", pos + len, pos2);
+ assertThat(pos2).isEqualTo(pos + len);
ByteBuffer bb1 = ByteBuffer.wrap(chunk, 0, chunkLen);
ByteBuffer bb2 = ByteBuffer.wrap(testFileContents, pos, chunkLen);
- Assert.assertEquals("Read contents", bb1, bb2);
+ assertThat(bb2).isEqualTo(bb1);
// Test skip
long toSkip = random.nextInt(testFileSize);
long skipped = decryptedStream.skip(toSkip);
if (pos2 + toSkip < testFileSize) {
- Assert.assertEquals("Skipped", toSkip, skipped);
+ assertThat(skipped).isEqualTo(toSkip);
} else {
- Assert.assertEquals("Skipped", (testFileSize - pos2), skipped);
+ assertThat(skipped).isEqualTo(testFileSize - pos2);
}
int pos3 = (int) decryptedStream.getPos();
- Assert.assertEquals("Position", pos2 + skipped, pos3);
+ assertThat(pos3).isEqualTo(pos2 + skipped);
chunkLen = random.nextInt(testFileSize);
left = testFileSize - pos3;
@@ -334,7 +328,7 @@ public class TestGcmStreams {
decryptedStream.read(chunk, 0, chunkLen);
bb1 = ByteBuffer.wrap(chunk, 0, chunkLen);
bb2 = ByteBuffer.wrap(testFileContents, pos3, chunkLen);
- Assert.assertEquals("Read contents", bb1, bb2);
+ assertThat(bb2).isEqualTo(bb1);
}
decryptedStream.close();
@@ -357,7 +351,7 @@ public class TestGcmStreams {
byte[] aadPrefix = new byte[16];
random.nextBytes(aadPrefix);
- File testFile = temp.newFile();
+ File testFile = File.createTempFile("test", null, temp.toFile());
AesGcmOutputFile encryptedFile =
new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix);
PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite();
@@ -374,17 +368,17 @@ public class TestGcmStreams {
encryptedStream.write(testFileContents, offset, chunkLen);
offset += chunkLen;
- Assert.assertEquals("Position", offset, encryptedStream.getPos());
+ assertThat(encryptedStream.getPos()).isEqualTo(offset);
left -= chunkLen;
}
encryptedStream.close();
- Assert.assertEquals("Final position in closed stream", offset,
encryptedStream.getPos());
+ assertThat(encryptedStream.getPos()).as("Final position in closed
stream").isEqualTo(offset);
AesGcmInputFile decryptedFile =
new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix);
SeekableInputStream decryptedStream = decryptedFile.newStream();
- Assert.assertEquals("File size", testFileSize,
decryptedFile.getLength());
+ assertThat(decryptedFile.getLength()).isEqualTo(testFileSize);
offset = 0;
chunkLen = Ciphers.PLAIN_BLOCK_SIZE;
@@ -399,12 +393,12 @@ public class TestGcmStreams {
decryptedStream.seek(offset);
int len = decryptedStream.read(chunk, 0, chunkLen);
- Assert.assertEquals("Read length", len, chunkLen);
- Assert.assertEquals("Position", offset + len,
decryptedStream.getPos());
+ assertThat(chunkLen).isEqualTo(len);
+ assertThat(decryptedStream.getPos()).isEqualTo(offset + len);
ByteBuffer bb1 = ByteBuffer.wrap(chunk, 0, chunkLen);
ByteBuffer bb2 = ByteBuffer.wrap(testFileContents, offset, chunkLen);
- Assert.assertEquals("Read contents", bb1, bb2);
+ assertThat(bb2).isEqualTo(bb1);
offset += len;
left = testFileSize - offset;
diff --git
a/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java
b/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java
index 889506cb93..d73f52da46 100644
---
a/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java
+++
b/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java
@@ -18,11 +18,12 @@
*/
package org.apache.iceberg.encryption;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
-import org.assertj.core.api.Assertions;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
public class TestStandardKeyMetadataParser {
@@ -35,14 +36,14 @@ public class TestStandardKeyMetadataParser {
ByteBuffer serialized = metadata.buffer();
StandardKeyMetadata parsedMetadata = StandardKeyMetadata.parse(serialized);
- Assert.assertEquals(parsedMetadata.encryptionKey(), encryptionKey);
- Assert.assertEquals(parsedMetadata.aadPrefix(), aadPrefix);
+ assertThat(encryptionKey).isEqualTo(parsedMetadata.encryptionKey());
+ assertThat(aadPrefix).isEqualTo(parsedMetadata.aadPrefix());
}
@Test
public void testUnsupportedVersion() {
ByteBuffer badBuffer = ByteBuffer.wrap(new byte[] {0x02});
- Assertions.assertThatThrownBy(() -> StandardKeyMetadata.parse(badBuffer))
+ assertThatThrownBy(() -> StandardKeyMetadata.parse(badBuffer))
.isInstanceOf(UnsupportedOperationException.class)
.hasMessage("Cannot resolve schema for version: 2");
}
diff --git
a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java
b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java
index f7c81ae879..ceffeb3749 100644
--- a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java
+++ b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java
@@ -18,36 +18,35 @@
*/
package org.apache.iceberg.io;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.Arrays;
+import java.util.List;
import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.ParameterizedTestExtension;
+import org.apache.iceberg.Parameters;
import org.apache.iceberg.PartitionKey;
import org.apache.iceberg.PartitionSpec;
-import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.TestBase;
import org.apache.iceberg.data.GenericRecord;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.encryption.EncryptedOutputFile;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
-import org.assertj.core.api.Assertions;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
-public class TestOutputFileFactory extends TableTestBase {
+@ExtendWith(ParameterizedTestExtension.class)
+public class TestOutputFileFactory extends TestBase {
- @Parameterized.Parameters(name = "formatVersion = {0}")
- public static Object[] parameters() {
- return new Object[] {1, 2};
+ @Parameters(name = "formatVersion = {0}")
+ protected static List<Object> parameters() {
+ return Arrays.asList(1, 2);
}
private static final int PARTITION_ID = 1;
private static final int TASK_ID = 100;
- public TestOutputFileFactory(int formatVersion) {
- super(formatVersion);
- }
-
- @Test
+ @TestTemplate
public void testOutputFileFactoryWithCustomFormat() {
table.updateProperties().defaultFormat(FileFormat.ORC).commit();
@@ -55,11 +54,10 @@ public class TestOutputFileFactory extends TableTestBase {
OutputFileFactory.builderFor(table, PARTITION_ID,
TASK_ID).format(FileFormat.AVRO).build();
String location =
fileFactory.newOutputFile().encryptingOutputFile().location();
- Assert.assertEquals(
- "File format should be correct", FileFormat.AVRO,
FileFormat.fromFileName(location));
+ assertThat(FileFormat.fromFileName(location)).isEqualTo(FileFormat.AVRO);
}
- @Test
+ @TestTemplate
public void testOutputFileFactoryWithMultipleSpecs() {
OutputFileFactory fileFactory =
OutputFileFactory.builderFor(table, PARTITION_ID,
TASK_ID).operationId("append").build();
@@ -67,18 +65,17 @@ public class TestOutputFileFactory extends TableTestBase {
EncryptedOutputFile unpartitionedFile =
fileFactory.newOutputFile(PartitionSpec.unpartitioned(), null);
String unpartitionedFileLocation =
unpartitionedFile.encryptingOutputFile().location();
-
Assert.assertTrue(unpartitionedFileLocation.endsWith("data/00001-100-append-00001.parquet"));
+
assertThat(unpartitionedFileLocation).endsWith("data/00001-100-append-00001.parquet");
Record record =
GenericRecord.create(table.schema()).copy(ImmutableMap.of("data", "aaa"));
PartitionKey partitionKey = new PartitionKey(table.spec(), table.schema());
partitionKey.partition(record);
EncryptedOutputFile partitionedFile =
fileFactory.newOutputFile(table.spec(), partitionKey);
String partitionedFileLocation =
partitionedFile.encryptingOutputFile().location();
- Assert.assertTrue(
-
partitionedFileLocation.endsWith("data_bucket=7/00001-100-append-00002.parquet"));
+
assertThat(partitionedFileLocation).endsWith("data_bucket=7/00001-100-append-00002.parquet");
}
- @Test
+ @TestTemplate
public void testWithCustomSuffix() {
OutputFileFactory fileFactory =
OutputFileFactory.builderFor(table, PARTITION_ID, TASK_ID)
@@ -89,15 +86,14 @@ public class TestOutputFileFactory extends TableTestBase {
EncryptedOutputFile unpartitionedFile =
fileFactory.newOutputFile(PartitionSpec.unpartitioned(), null);
String unpartitionedFileLocation =
unpartitionedFile.encryptingOutputFile().location();
- Assertions.assertThat(unpartitionedFileLocation)
- .endsWith("data/00001-100-append-00001-suffix.parquet");
+
assertThat(unpartitionedFileLocation).endsWith("data/00001-100-append-00001-suffix.parquet");
Record record =
GenericRecord.create(table.schema()).copy(ImmutableMap.of("data", "aaa"));
PartitionKey partitionKey = new PartitionKey(table.spec(), table.schema());
partitionKey.partition(record);
EncryptedOutputFile partitionedFile =
fileFactory.newOutputFile(table.spec(), partitionKey);
String partitionedFileLocation =
partitionedFile.encryptingOutputFile().location();
- Assertions.assertThat(partitionedFileLocation)
+ assertThat(partitionedFileLocation)
.endsWith("data_bucket=7/00001-100-append-00002-suffix.parquet");
}
}
diff --git
a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java
b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java
index 6b59095225..b9ae9dc273 100644
--- a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java
+++ b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java
@@ -19,28 +19,27 @@
package org.apache.iceberg.mapping;
import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.ParameterizedTestExtension;
+import org.apache.iceberg.Parameters;
import org.apache.iceberg.TableProperties;
-import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.TestBase;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.types.Types;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class TestMappingUpdates extends TableTestBase {
- @Parameterized.Parameters(name = "formatVersion = {0}")
- public static Object[] parameters() {
- return new Object[] {1, 2};
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+@ExtendWith(ParameterizedTestExtension.class)
+public class TestMappingUpdates extends TestBase {
+ @Parameters(name = "formatVersion = {0}")
+ protected static List<Object> parameters() {
+ return Arrays.asList(1, 2);
}
- public TestMappingUpdates(int formatVersion) {
- super(formatVersion);
- }
-
- @Test
+ @TestTemplate
public void testAddColumnMappingUpdate() {
NameMapping mapping = MappingUtil.create(table.schema());
table
@@ -48,22 +47,21 @@ public class TestMappingUpdates extends TableTestBase {
.set(TableProperties.DEFAULT_NAME_MAPPING,
NameMappingParser.toJson(mapping))
.commit();
- Assert.assertEquals(
- MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data")),
- mapping.asMappedFields());
+ assertThat(mapping.asMappedFields())
+ .isEqualTo(MappedFields.of(MappedField.of(1, "id"), MappedField.of(2,
"data")));
table.updateSchema().addColumn("ts",
Types.TimestampType.withZone()).commit();
NameMapping updated =
NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING));
- Assert.assertEquals(
- MappedFields.of(
- MappedField.of(1, "id"), MappedField.of(2, "data"),
MappedField.of(3, "ts")),
- updated.asMappedFields());
+ assertThat(updated.asMappedFields())
+ .isEqualTo(
+ MappedFields.of(
+ MappedField.of(1, "id"), MappedField.of(2, "data"),
MappedField.of(3, "ts")));
}
- @Test
+ @TestTemplate
public void testAddNestedColumnMappingUpdate() {
NameMapping mapping = MappingUtil.create(table.schema());
table
@@ -71,9 +69,8 @@ public class TestMappingUpdates extends TableTestBase {
.set(TableProperties.DEFAULT_NAME_MAPPING,
NameMappingParser.toJson(mapping))
.commit();
- Assert.assertEquals(
- MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data")),
- mapping.asMappedFields());
+ assertThat(mapping.asMappedFields())
+ .isEqualTo(MappedFields.of(MappedField.of(1, "id"), MappedField.of(2,
"data")));
table
.updateSchema()
@@ -86,32 +83,32 @@ public class TestMappingUpdates extends TableTestBase {
NameMapping updated =
NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING));
- Assert.assertEquals(
- MappedFields.of(
- MappedField.of(1, "id"),
- MappedField.of(2, "data"),
- MappedField.of(
- 3, "point", MappedFields.of(MappedField.of(4, "x"),
MappedField.of(5, "y")))),
- updated.asMappedFields());
+ assertThat(updated.asMappedFields())
+ .isEqualTo(
+ MappedFields.of(
+ MappedField.of(1, "id"),
+ MappedField.of(2, "data"),
+ MappedField.of(
+ 3, "point", MappedFields.of(MappedField.of(4, "x"),
MappedField.of(5, "y")))));
table.updateSchema().addColumn("point", "z",
Types.DoubleType.get()).commit();
NameMapping pointUpdated =
NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING));
- Assert.assertEquals(
- MappedFields.of(
- MappedField.of(1, "id"),
- MappedField.of(2, "data"),
- MappedField.of(
- 3,
- "point",
- MappedFields.of(
- MappedField.of(4, "x"), MappedField.of(5, "y"),
MappedField.of(6, "z")))),
- pointUpdated.asMappedFields());
+ assertThat(pointUpdated.asMappedFields())
+ .isEqualTo(
+ MappedFields.of(
+ MappedField.of(1, "id"),
+ MappedField.of(2, "data"),
+ MappedField.of(
+ 3,
+ "point",
+ MappedFields.of(
+ MappedField.of(4, "x"), MappedField.of(5, "y"),
MappedField.of(6, "z")))));
}
- @Test
+ @TestTemplate
public void testRenameMappingUpdate() {
NameMapping mapping = MappingUtil.create(table.schema());
table
@@ -119,22 +116,21 @@ public class TestMappingUpdates extends TableTestBase {
.set(TableProperties.DEFAULT_NAME_MAPPING,
NameMappingParser.toJson(mapping))
.commit();
- Assert.assertEquals(
- MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data")),
- mapping.asMappedFields());
+ assertThat(mapping.asMappedFields())
+ .isEqualTo(MappedFields.of(MappedField.of(1, "id"), MappedField.of(2,
"data")));
table.updateSchema().renameColumn("id", "object_id").commit();
NameMapping updated =
NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING));
- Assert.assertEquals(
- MappedFields.of(
- MappedField.of(1, ImmutableList.of("id", "object_id")),
MappedField.of(2, "data")),
- updated.asMappedFields());
+ assertThat(updated.asMappedFields())
+ .isEqualTo(
+ MappedFields.of(
+ MappedField.of(1, ImmutableList.of("id", "object_id")),
MappedField.of(2, "data")));
}
- @Test
+ @TestTemplate
public void testRenameNestedFieldMappingUpdate() {
NameMapping mapping = MappingUtil.create(table.schema());
table
@@ -153,33 +149,33 @@ public class TestMappingUpdates extends TableTestBase {
NameMapping updated =
NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING));
- Assert.assertEquals(
- MappedFields.of(
- MappedField.of(1, "id"),
- MappedField.of(2, "data"),
- MappedField.of(
- 3, "point", MappedFields.of(MappedField.of(4, "x"),
MappedField.of(5, "y")))),
- updated.asMappedFields());
+ assertThat(updated.asMappedFields())
+ .isEqualTo(
+ MappedFields.of(
+ MappedField.of(1, "id"),
+ MappedField.of(2, "data"),
+ MappedField.of(
+ 3, "point", MappedFields.of(MappedField.of(4, "x"),
MappedField.of(5, "y")))));
table.updateSchema().renameColumn("point.x", "X").renameColumn("point.y",
"Y").commit();
NameMapping pointUpdated =
NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING));
- Assert.assertEquals(
- MappedFields.of(
- MappedField.of(1, "id"),
- MappedField.of(2, "data"),
- MappedField.of(
- 3,
- "point",
- MappedFields.of(
- MappedField.of(4, ImmutableList.of("x", "X")),
- MappedField.of(5, ImmutableList.of("y", "Y"))))),
- pointUpdated.asMappedFields());
+ assertThat(pointUpdated.asMappedFields())
+ .isEqualTo(
+ MappedFields.of(
+ MappedField.of(1, "id"),
+ MappedField.of(2, "data"),
+ MappedField.of(
+ 3,
+ "point",
+ MappedFields.of(
+ MappedField.of(4, ImmutableList.of("x", "X")),
+ MappedField.of(5, ImmutableList.of("y", "Y"))))));
}
- @Test
+ @TestTemplate
public void testRenameComplexFieldMappingUpdate() {
NameMapping mapping = MappingUtil.create(table.schema());
table
@@ -198,27 +194,27 @@ public class TestMappingUpdates extends TableTestBase {
NameMapping updated =
NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING));
- Assert.assertEquals(
- MappedFields.of(
- MappedField.of(1, "id"),
- MappedField.of(2, "data"),
- MappedField.of(
- 3, "point", MappedFields.of(MappedField.of(4, "x"),
MappedField.of(5, "y")))),
- updated.asMappedFields());
+ assertThat(updated.asMappedFields())
+ .isEqualTo(
+ MappedFields.of(
+ MappedField.of(1, "id"),
+ MappedField.of(2, "data"),
+ MappedField.of(
+ 3, "point", MappedFields.of(MappedField.of(4, "x"),
MappedField.of(5, "y")))));
table.updateSchema().renameColumn("point", "p2").commit();
NameMapping pointUpdated =
NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING));
- Assert.assertEquals(
- MappedFields.of(
- MappedField.of(1, "id"),
- MappedField.of(2, "data"),
- MappedField.of(
- 3,
- ImmutableList.of("point", "p2"),
- MappedFields.of(MappedField.of(4, "x"), MappedField.of(5,
"y")))),
- pointUpdated.asMappedFields());
+ assertThat(pointUpdated.asMappedFields())
+ .isEqualTo(
+ MappedFields.of(
+ MappedField.of(1, "id"),
+ MappedField.of(2, "data"),
+ MappedField.of(
+ 3,
+ ImmutableList.of("point", "p2"),
+ MappedFields.of(MappedField.of(4, "x"), MappedField.of(5,
"y")))));
}
}
diff --git a/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java
b/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java
index bde99b4113..d30a93d50d 100644
--- a/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java
+++ b/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java
@@ -19,12 +19,12 @@
package org.apache.iceberg.mapping;
import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
import org.apache.iceberg.Schema;
import org.apache.iceberg.types.Types;
-import org.assertj.core.api.Assertions;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
public class TestNameMapping {
@Test
@@ -36,7 +36,7 @@ public class TestNameMapping {
MappedFields expected = MappedFields.of(MappedField.of(1, "id"),
MappedField.of(2, "data"));
NameMapping mapping = MappingUtil.create(schema);
- Assert.assertEquals(expected, mapping.asMappedFields());
+ assertThat(mapping.asMappedFields()).isEqualTo(expected);
}
@Test
@@ -62,7 +62,7 @@ public class TestNameMapping {
MappedFields.of(MappedField.of(4, "latitude"),
MappedField.of(5, "longitude"))));
NameMapping mapping = MappingUtil.create(schema);
- Assert.assertEquals(expected, mapping.asMappedFields());
+ assertThat(mapping.asMappedFields()).isEqualTo(expected);
}
@Test
@@ -84,7 +84,7 @@ public class TestNameMapping {
3, "map", MappedFields.of(MappedField.of(4, "key"),
MappedField.of(5, "value"))));
NameMapping mapping = MappingUtil.create(schema);
- Assert.assertEquals(expected, mapping.asMappedFields());
+ assertThat(mapping.asMappedFields()).isEqualTo(expected);
}
@Test
@@ -117,7 +117,7 @@ public class TestNameMapping {
MappedField.of(5, "value"))));
NameMapping mapping = MappingUtil.create(schema);
- Assert.assertEquals(expected, mapping.asMappedFields());
+ assertThat(mapping.asMappedFields()).isEqualTo(expected);
}
@Test
@@ -152,7 +152,7 @@ public class TestNameMapping {
MappedFields.of(MappedField.of(6, "x"),
MappedField.of(7, "y"))))));
NameMapping mapping = MappingUtil.create(schema);
- Assert.assertEquals(expected, mapping.asMappedFields());
+ assertThat(mapping.asMappedFields()).isEqualTo(expected);
}
@Test
@@ -170,13 +170,13 @@ public class TestNameMapping {
MappedField.of(3, "list", MappedFields.of(MappedField.of(4,
"element"))));
NameMapping mapping = MappingUtil.create(schema);
- Assert.assertEquals(expected, mapping.asMappedFields());
+ assertThat(mapping.asMappedFields()).isEqualTo(expected);
}
@Test
public void testFailsDuplicateId() {
// the schema can be created because ID indexing is lazy
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
() ->
new Schema(
required(1, "id", Types.LongType.get()),
@@ -187,7 +187,7 @@ public class TestNameMapping {
@Test
public void testFailsDuplicateName() {
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
() -> new NameMapping(MappedFields.of(MappedField.of(1, "x"),
MappedField.of(2, "x"))))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Multiple entries with same key: x=2 and x=1");
@@ -227,17 +227,17 @@ public class TestNameMapping {
NameMapping mapping = MappingUtil.create(schema);
- Assert.assertNull("Should not return a field mapping for a missing ID",
mapping.find(100));
- Assert.assertEquals(MappedField.of(2, "data"), mapping.find(2));
- Assert.assertEquals(MappedField.of(6, "x"), mapping.find(6));
- Assert.assertEquals(MappedField.of(9, "element"), mapping.find(9));
- Assert.assertEquals(MappedField.of(11, "latitude"), mapping.find(11));
- Assert.assertEquals(
- MappedField.of(
- 10,
- "location",
- MappedFields.of(MappedField.of(11, "latitude"), MappedField.of(12,
"longitude"))),
- mapping.find(10));
+ assertThat(mapping.find(100)).as("Should not return a field mapping for a
missing ID").isNull();
+ assertThat(mapping.find(2)).isEqualTo(MappedField.of(2, "data"));
+ assertThat(mapping.find(6)).isEqualTo(MappedField.of(6, "x"));
+ assertThat(mapping.find(9)).isEqualTo(MappedField.of(9, "element"));
+ assertThat(mapping.find(11)).isEqualTo(MappedField.of(11, "latitude"));
+ assertThat(mapping.find(10))
+ .isEqualTo(
+ MappedField.of(
+ 10,
+ "location",
+ MappedFields.of(MappedField.of(11, "latitude"),
MappedField.of(12, "longitude"))));
}
@Test
@@ -266,20 +266,27 @@ public class TestNameMapping {
NameMapping mapping = MappingUtil.create(schema);
- Assert.assertNull(
- "Should not return a field mapping for a nested name",
mapping.find("element"));
- Assert.assertNull("Should not return a field mapping for a nested name",
mapping.find("x"));
- Assert.assertNull("Should not return a field mapping for a nested name",
mapping.find("key"));
- Assert.assertNull("Should not return a field mapping for a nested name",
mapping.find("value"));
- Assert.assertEquals(MappedField.of(2, "data"), mapping.find("data"));
- Assert.assertEquals(MappedField.of(6, "x"), mapping.find("map", "value",
"x"));
- Assert.assertEquals(MappedField.of(9, "element"), mapping.find("list",
"element"));
- Assert.assertEquals(MappedField.of(11, "latitude"),
mapping.find("location", "latitude"));
- Assert.assertEquals(
- MappedField.of(
- 10,
- "location",
- MappedFields.of(MappedField.of(11, "latitude"), MappedField.of(12,
"longitude"))),
- mapping.find("location"));
+ assertThat(mapping.find("element"))
+ .as("Should not return a field mapping for a nested name")
+ .isNull();
+ assertThat(mapping.find("x"))
+ .as("Should not return a field mapping for a nested name")
+ .isNull();
+ assertThat(mapping.find("key"))
+ .as("Should not return a field mapping for a nested name")
+ .isNull();
+ assertThat(mapping.find("value"))
+ .as("Should not return a field mapping for a nested name")
+ .isNull();
+ assertThat(mapping.find("data")).isEqualTo(MappedField.of(2, "data"));
+ assertThat(mapping.find("map", "value", "x")).isEqualTo(MappedField.of(6,
"x"));
+ assertThat(mapping.find("list", "element")).isEqualTo(MappedField.of(9,
"element"));
+ assertThat(mapping.find("location",
"latitude")).isEqualTo(MappedField.of(11, "latitude"));
+ assertThat(mapping.find("location"))
+ .isEqualTo(
+ MappedField.of(
+ 10,
+ "location",
+ MappedFields.of(MappedField.of(11, "latitude"),
MappedField.of(12, "longitude"))));
}
}
diff --git a/data/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java
b/data/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java
index 724970bc09..d85a73dbd2 100644
--- a/data/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java
+++ b/data/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java
@@ -18,6 +18,8 @@
*/
package org.apache.iceberg.orc;
+import static org.assertj.core.api.Assertions.assertThat;
+
import java.io.File;
import java.io.IOException;
import java.util.Map;
@@ -26,6 +28,7 @@ import org.apache.iceberg.FileFormat;
import org.apache.iceberg.Files;
import org.apache.iceberg.Metrics;
import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.ParameterizedTestExtension;
import org.apache.iceberg.Schema;
import org.apache.iceberg.TestMetrics;
import org.apache.iceberg.data.Record;
@@ -37,29 +40,18 @@ 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.Lists;
import org.apache.iceberg.types.Type;
-import org.junit.Assert;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.extension.ExtendWith;
/** Test Metrics for ORC. */
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public class TestOrcMetrics extends TestMetrics {
static final ImmutableSet<Object> BINARY_TYPES =
ImmutableSet.of(Type.TypeID.BINARY, Type.TypeID.FIXED, Type.TypeID.UUID);
- @Parameterized.Parameters(name = "formatVersion = {0}")
- public static Object[] parameters() {
- return new Object[] {1, 2};
- }
-
- public TestOrcMetrics(int formatVersion) {
- super(formatVersion);
- }
-
@Override
protected OutputFile createOutputFile() throws IOException {
- File tmpFolder = temp.newFolder("orc");
+ File tmpFolder = java.nio.file.Files.createTempDirectory(temp,
"orc").toFile();
String filename = UUID.randomUUID().toString();
return Files.localOutput(new File(tmpFolder,
FileFormat.ORC.addExtension(filename)));
}
@@ -119,12 +111,8 @@ public class TestOrcMetrics extends TestMetrics {
protected <T> void assertBounds(
int fieldId, Type type, T lowerBound, T upperBound, Metrics metrics) {
if (isBinaryType(type)) {
- Assert.assertFalse(
- "ORC binary field should not have lower bounds.",
- metrics.lowerBounds().containsKey(fieldId));
- Assert.assertFalse(
- "ORC binary field should not have upper bounds.",
- metrics.upperBounds().containsKey(fieldId));
+ assertThat(metrics.lowerBounds()).doesNotContainKey(fieldId);
+ assertThat(metrics.upperBounds()).doesNotContainKey(fieldId);
return;
}
super.assertBounds(fieldId, type, lowerBound, upperBound, metrics);
diff --git
a/data/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java
b/data/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java
index f363e5d979..545cff9311 100644
--- a/data/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java
+++ b/data/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java
@@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat;
import org.apache.iceberg.Files;
import org.apache.iceberg.Metrics;
import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.ParameterizedTestExtension;
import org.apache.iceberg.Schema;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.TestMetrics;
@@ -37,24 +38,14 @@ import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.parquet.hadoop.ParquetFileReader;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.extension.ExtendWith;
/** Test Metrics for Parquet. */
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public class TestParquetMetrics extends TestMetrics {
private static final Map<String, String> SMALL_ROW_GROUP_CONFIG =
ImmutableMap.of(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "1600");
- @Parameterized.Parameters(name = "formatVersion = {0}")
- public static Object[] parameters() {
- return new Object[] {1, 2};
- }
-
- public TestParquetMetrics(int formatVersion) {
- super(formatVersion);
- }
-
@Override
public FileFormat fileFormat() {
return FileFormat.PARQUET;
@@ -62,7 +53,7 @@ public class TestParquetMetrics extends TestMetrics {
@Override
protected OutputFile createOutputFile() throws IOException {
- File tmpFolder = temp.newFolder("parquet");
+ File tmpFolder = java.nio.file.Files.createTempDirectory(temp,
"parquet").toFile();
String filename = UUID.randomUUID().toString();
return Files.localOutput(new File(tmpFolder,
FileFormat.PARQUET.addExtension(filename)));
}