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 ea30d363e0 Core, Spark: Migrate tests that depend on ScanTestBase to
JUnit5 (#9416)
ea30d363e0 is described below
commit ea30d363e077763524c2f943ff1b1c5fe65eb295
Author: Chinmay Bhat <[email protected]>
AuthorDate: Tue Jan 16 14:19:43 2024 +0530
Core, Spark: Migrate tests that depend on ScanTestBase to JUnit5 (#9416)
---
.../org/apache/iceberg/DataTableScanTestBase.java | 96 +++--
.../apache/iceberg/DeleteFileIndexTestBase.java | 401 +++++++++++----------
.../org/apache/iceberg/FilterFilesTestBase.java | 36 +-
.../iceberg/ScanPlanningAndReportingTestBase.java | 26 +-
.../test/java/org/apache/iceberg/ScanTestBase.java | 104 +++---
.../src/test/java/org/apache/iceberg/TestBase.java | 4 +-
.../iceberg/TestBaseIncrementalAppendScan.java | 113 +++---
.../iceberg/TestBaseIncrementalChangelogScan.java | 124 ++++---
.../org/apache/iceberg/TestLocalDataTableScan.java | 4 -
.../org/apache/iceberg/TestLocalFilterFiles.java | 17 +-
.../io/TestGenericSortedPosDeleteWriter.java | 2 +-
.../iceberg/SparkDistributedDataScanTestBase.java | 45 +--
.../TestSparkDistributedDataScanDeletes.java | 45 ++-
.../TestSparkDistributedDataScanFilterFiles.java | 26 +-
...tSparkDistributedDataScanJavaSerialization.java | 13 +-
...tSparkDistributedDataScanKryoSerialization.java | 13 +-
.../TestSparkDistributedDataScanReporting.java | 41 +--
.../iceberg/SparkDistributedDataScanTestBase.java | 45 +--
.../TestSparkDistributedDataScanDeletes.java | 45 ++-
.../TestSparkDistributedDataScanFilterFiles.java | 26 +-
...tSparkDistributedDataScanJavaSerialization.java | 13 +-
...tSparkDistributedDataScanKryoSerialization.java | 13 +-
.../TestSparkDistributedDataScanReporting.java | 41 +--
23 files changed, 626 insertions(+), 667 deletions(-)
diff --git a/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java
b/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java
index 7133a5a761..04bbcf6622 100644
--- a/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java
@@ -18,6 +18,10 @@
*/
package org.apache.iceberg;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
import java.io.IOException;
import java.util.List;
import java.util.UUID;
@@ -25,28 +29,23 @@ import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
-import org.assertj.core.api.Assertions;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Test;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+@ExtendWith(ParameterizedTestExtension.class)
public abstract class DataTableScanTestBase<
ScanT extends Scan<ScanT, T, G>, T extends ScanTask, G extends
ScanTaskGroup<T>>
extends ScanTestBase<ScanT, T, G> {
- public DataTableScanTestBase(int formatVersion) {
- super(formatVersion);
- }
-
protected abstract ScanT useRef(ScanT scan, String ref);
protected abstract ScanT useSnapshot(ScanT scan, long snapshotId);
protected abstract ScanT asOfTime(ScanT scan, long timestampMillis);
- @Test
+ @TestTemplate
public void testTaskRowCounts() {
- Assume.assumeTrue(formatVersion == 2);
+ assumeThat(formatVersion).isEqualTo(2);
DataFile dataFile1 = newDataFile("data_bucket=0");
table.newFastAppend().appendFile(dataFile1).commit();
@@ -63,15 +62,15 @@ public abstract class DataTableScanTestBase<
ScanT scan = newScan().option(TableProperties.SPLIT_SIZE, "50");
List<T> fileScanTasks = Lists.newArrayList(scan.planFiles());
- Assert.assertEquals("Must have 2 FileScanTasks", 2, fileScanTasks.size());
+ assertThat(fileScanTasks).as("Must have 2 FileScanTasks").hasSize(2);
for (T task : fileScanTasks) {
- Assert.assertEquals("Rows count must match", 10,
task.estimatedRowsCount());
+ assertThat(task.estimatedRowsCount()).as("Rows count must
match").isEqualTo(10);
}
List<G> combinedScanTasks = Lists.newArrayList(scan.planTasks());
- Assert.assertEquals("Must have 4 CombinedScanTask", 4,
combinedScanTasks.size());
+ assertThat(combinedScanTasks).as("Must have 4
CombinedScanTask").hasSize(4);
for (G task : combinedScanTasks) {
- Assert.assertEquals("Rows count must match", 5,
task.estimatedRowsCount());
+ assertThat(task.estimatedRowsCount()).as("Rows count must
match").isEqualTo(5);
}
}
@@ -96,7 +95,7 @@ public abstract class DataTableScanTestBase<
.build();
}
- @Test
+ @TestTemplate
public void testScanFromBranchTip() throws IOException {
table.newFastAppend().appendFile(FILE_A).commit();
// Add B and C to new branch
@@ -112,7 +111,7 @@ public abstract class DataTableScanTestBase<
validateExpectedFileScanTasks(mainScan, ImmutableList.of(FILE_A.path(),
FILE_D.path()));
}
- @Test
+ @TestTemplate
public void testScanFromTag() throws IOException {
table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit();
table.manageSnapshots().createTag("tagB",
table.currentSnapshot().snapshotId()).commit();
@@ -124,31 +123,30 @@ public abstract class DataTableScanTestBase<
mainScan, ImmutableList.of(FILE_A.path(), FILE_B.path(),
FILE_C.path()));
}
- @Test
+ @TestTemplate
public void testScanFromRefWhenSnapshotSetFails() {
table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit();
table.manageSnapshots().createTag("tagB",
table.currentSnapshot().snapshotId()).commit();
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
() -> useRef(useSnapshot(newScan(),
table.currentSnapshot().snapshotId()), "tagB"))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot override ref, already set snapshot id=1");
}
- @Test
+ @TestTemplate
public void testSettingSnapshotWhenRefSetFails() {
table.newFastAppend().appendFile(FILE_A).commit();
Snapshot snapshotA = table.currentSnapshot();
table.newFastAppend().appendFile(FILE_B).commit();
table.manageSnapshots().createTag("tagB",
table.currentSnapshot().snapshotId()).commit();
- Assertions.assertThatThrownBy(
- () -> useSnapshot(useRef(newScan(), "tagB"),
snapshotA.snapshotId()))
+ assertThatThrownBy(() -> useSnapshot(useRef(newScan(), "tagB"),
snapshotA.snapshotId()))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot override snapshot, already set snapshot id=2");
}
- @Test
+ @TestTemplate
public void testBranchTimeTravelFails() {
table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit();
table
@@ -156,27 +154,26 @@ public abstract class DataTableScanTestBase<
.createBranch("testBranch", table.currentSnapshot().snapshotId())
.commit();
- Assertions.assertThatThrownBy(
- () -> asOfTime(useRef(newScan(), "testBranch"),
System.currentTimeMillis()))
+ assertThatThrownBy(() -> asOfTime(useRef(newScan(), "testBranch"),
System.currentTimeMillis()))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot override snapshot, already set snapshot id=1");
}
- @Test
+ @TestTemplate
public void testSettingMultipleRefsFails() {
table.newFastAppend().appendFile(FILE_A).commit();
table.manageSnapshots().createTag("tagA",
table.currentSnapshot().snapshotId()).commit();
table.newFastAppend().appendFile(FILE_B).commit();
table.manageSnapshots().createTag("tagB",
table.currentSnapshot().snapshotId()).commit();
- Assertions.assertThatThrownBy(() -> useRef(useRef(newScan(), "tagB"),
"tagA"))
+ assertThatThrownBy(() -> useRef(useRef(newScan(), "tagB"), "tagA"))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot override ref, already set snapshot id=2");
}
- @Test
+ @TestTemplate
public void testSettingInvalidRefFails() {
- Assertions.assertThatThrownBy(() -> useRef(newScan(), "nonexisting"))
+ assertThatThrownBy(() -> useRef(newScan(), "nonexisting"))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot find ref nonexisting");
}
@@ -184,18 +181,18 @@ public abstract class DataTableScanTestBase<
private void validateExpectedFileScanTasks(ScanT scan, List<CharSequence>
expectedFileScanPaths)
throws IOException {
try (CloseableIterable<T> scanTasks = scan.planFiles()) {
- Assert.assertEquals(expectedFileScanPaths.size(),
Iterables.size(scanTasks));
+ assertThat(scanTasks).hasSameSizeAs(expectedFileScanPaths);
List<CharSequence> actualFiles = Lists.newArrayList();
for (T task : scanTasks) {
actualFiles.add(((FileScanTask) task).file().path());
}
- Assert.assertTrue(actualFiles.containsAll(expectedFileScanPaths));
+ assertThat(actualFiles).containsAll(expectedFileScanPaths);
}
}
- @Test
+ @TestTemplate
public void testSequenceNumbersThroughPlanFiles() {
- Assume.assumeTrue(formatVersion == 2);
+ assumeThat(formatVersion).isEqualTo(2);
DataFile dataFile1 = newDataFile("data_bucket=0");
table.newFastAppend().appendFile(dataFile1).commit();
@@ -212,7 +209,7 @@ public abstract class DataTableScanTestBase<
ScanT scan = newScan();
List<T> fileScanTasks = Lists.newArrayList(scan.planFiles());
- Assert.assertEquals("Must have 2 FileScanTasks", 2, fileScanTasks.size());
+ assertThat(fileScanTasks).as("Must have 2 FileScanTasks").hasSize(2);
for (T task : fileScanTasks) {
FileScanTask fileScanTask = (FileScanTask) task;
DataFile file = fileScanTask.file();
@@ -228,26 +225,25 @@ public abstract class DataTableScanTestBase<
expectedDeleteSequenceNumber = 4L;
}
- Assert.assertEquals(
- "Data sequence number mismatch",
- expectedDataSequenceNumber,
- file.dataSequenceNumber().longValue());
- Assert.assertEquals(
- "File sequence number mismatch",
- expectedDataSequenceNumber,
- file.fileSequenceNumber().longValue());
+ assertThat(file.dataSequenceNumber().longValue())
+ .as("Data sequence number mismatch")
+ .isEqualTo(expectedDataSequenceNumber);
+
+ assertThat(file.fileSequenceNumber().longValue())
+ .as("File sequence number mismatch")
+ .isEqualTo(expectedDataSequenceNumber);
List<DeleteFile> deleteFiles = fileScanTask.deletes();
- Assert.assertEquals("Must have 1 delete file", 1,
Iterables.size(deleteFiles));
+ assertThat(deleteFiles).as("Must have 1 delete file").hasSize(1);
+
DeleteFile deleteFile = Iterables.getOnlyElement(deleteFiles);
- Assert.assertEquals(
- "Data sequence number mismatch",
- expectedDeleteSequenceNumber,
- deleteFile.dataSequenceNumber().longValue());
- Assert.assertEquals(
- "File sequence number mismatch",
- expectedDeleteSequenceNumber,
- deleteFile.fileSequenceNumber().longValue());
+ assertThat(deleteFile.dataSequenceNumber().longValue())
+ .as("Data sequence number mismatch")
+ .isEqualTo(expectedDeleteSequenceNumber);
+
+ assertThat(deleteFile.fileSequenceNumber().longValue())
+ .as("File sequence number mismatch")
+ .isEqualTo(expectedDeleteSequenceNumber);
}
}
}
diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java
b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java
index 6354c3ee18..229650566c 100644
--- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java
@@ -25,6 +25,7 @@ import static
org.assertj.core.api.Assertions.assertThatThrownBy;
import java.io.File;
import java.io.IOException;
+import java.nio.file.Files;
import java.util.Arrays;
import java.util.List;
import java.util.UUID;
@@ -35,15 +36,17 @@ import
org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.util.CharSequenceSet;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+@ExtendWith(ParameterizedTestExtension.class)
public abstract class DeleteFileIndexTestBase<
ScanT extends Scan<ScanT, T, G>, T extends ScanTask, G extends
ScanTaskGroup<T>>
- extends TableTestBase {
+ extends TestBase {
- public DeleteFileIndexTestBase() {
- super(2 /* table format version */);
+ @Parameters(name = "formatVersion = {0}")
+ public static List<Object> parameters() {
+ return Arrays.asList(2);
}
static final DeleteFile FILE_A_POS_1 =
@@ -119,7 +122,7 @@ public abstract class DeleteFileIndexTestBase<
protected abstract ScanT newScan(Table table);
- @Test
+ @TestTemplate
public void testMinSequenceNumberFilteringForFiles() {
PartitionSpec partSpec = PartitionSpec.unpartitioned();
@@ -136,10 +139,10 @@ public abstract class DeleteFileIndexTestBase<
DataFile file = unpartitionedFile(partSpec);
- Assert.assertEquals("Only one delete file should apply", 1,
index.forDataFile(0, file).length);
+ assertThat(index.forDataFile(0, file)).as("Only one delete file should
apply").hasSize(1);
}
- @Test
+ @TestTemplate
public void testUnpartitionedDeletes() {
PartitionSpec partSpec = PartitionSpec.unpartitioned();
@@ -156,41 +159,47 @@ public abstract class DeleteFileIndexTestBase<
.build();
DataFile unpartitionedFile = unpartitionedFile(partSpec);
- Assert.assertArrayEquals(
- "All deletes should apply to seq 0", deleteFiles, index.forDataFile(0,
unpartitionedFile));
- Assert.assertArrayEquals(
- "All deletes should apply to seq 3", deleteFiles, index.forDataFile(3,
unpartitionedFile));
- Assert.assertArrayEquals(
- "Last 3 deletes should apply to seq 4",
- Arrays.copyOfRange(deleteFiles, 1, 4),
- index.forDataFile(4, unpartitionedFile));
- Assert.assertArrayEquals(
- "Last 3 deletes should apply to seq 5",
- Arrays.copyOfRange(deleteFiles, 1, 4),
- index.forDataFile(5, unpartitionedFile));
- Assert.assertArrayEquals(
- "Last delete should apply to seq 6",
- Arrays.copyOfRange(deleteFiles, 3, 4),
- index.forDataFile(6, unpartitionedFile));
- Assert.assertArrayEquals(
- "No deletes should apply to seq 7",
- new DataFile[0],
- index.forDataFile(7, unpartitionedFile));
- Assert.assertArrayEquals(
- "No deletes should apply to seq 10",
- new DataFile[0],
- index.forDataFile(10, unpartitionedFile));
+ assertThat(index.forDataFile(0, unpartitionedFile))
+ .as("All deletes should apply to seq 0")
+ .isEqualTo(deleteFiles);
+
+ assertThat(index.forDataFile(3, unpartitionedFile))
+ .as("All deletes should apply to seq 3")
+ .isEqualTo(deleteFiles);
+
+ assertThat(index.forDataFile(4, unpartitionedFile))
+ .as("All deletes should apply to seq 4")
+ .isEqualTo(Arrays.copyOfRange(deleteFiles, 1, 4));
+
+ assertThat(index.forDataFile(4, unpartitionedFile))
+ .as("Last 3 deletes should apply to seq 4")
+ .isEqualTo(Arrays.copyOfRange(deleteFiles, 1, 4));
+
+ assertThat(index.forDataFile(5, unpartitionedFile))
+ .as("Last 3 deletes should apply to seq 5")
+ .isEqualTo(Arrays.copyOfRange(deleteFiles, 1, 4));
+
+ assertThat(index.forDataFile(6, unpartitionedFile))
+ .as("Last delete should apply to seq 6")
+ .isEqualTo(Arrays.copyOfRange(deleteFiles, 3, 4));
+
+ assertThat(index.forDataFile(7, unpartitionedFile))
+ .as("No deletes should apply to seq 7")
+ .isEqualTo(new DataFile[0]);
+
+ assertThat(index.forDataFile(10, unpartitionedFile))
+ .as("No deletes should apply to seq 10")
+ .isEqualTo(new DataFile[0]);
// copy file A with a different spec ID
DataFile partitionedFileA = FILE_A.copy();
((BaseFile<?>) partitionedFileA).setSpecId(1);
- Assert.assertArrayEquals(
- "All global equality deletes should apply to a partitioned file",
- Arrays.copyOfRange(deleteFiles, 0, 2),
- index.forDataFile(0, partitionedFileA));
+ assertThat(index.forDataFile(0, partitionedFileA))
+ .as("All global equality deletes should apply to a partitioned file")
+ .isEqualTo(Arrays.copyOfRange(deleteFiles, 0, 2));
}
- @Test
+ @TestTemplate
public void testPartitionedDeleteIndex() {
DeleteFile[] deleteFiles = {
withDataSequenceNumber(4, partitionedEqDeletes(SPEC,
FILE_A.partition())),
@@ -204,49 +213,53 @@ public abstract class DeleteFileIndexTestBase<
.specsById(ImmutableMap.of(SPEC.specId(), SPEC, 1,
PartitionSpec.unpartitioned()))
.build();
- Assert.assertArrayEquals(
- "All deletes should apply to seq 0", deleteFiles, index.forDataFile(0,
FILE_A));
- Assert.assertArrayEquals(
- "All deletes should apply to seq 3", deleteFiles, index.forDataFile(3,
FILE_A));
- Assert.assertArrayEquals(
- "Last 3 deletes should apply to seq 4",
- Arrays.copyOfRange(deleteFiles, 1, 4),
- index.forDataFile(4, FILE_A));
- Assert.assertArrayEquals(
- "Last 3 deletes should apply to seq 5",
- Arrays.copyOfRange(deleteFiles, 1, 4),
- index.forDataFile(5, FILE_A));
- Assert.assertArrayEquals(
- "Last delete should apply to seq 6",
- Arrays.copyOfRange(deleteFiles, 3, 4),
- index.forDataFile(6, FILE_A));
- Assert.assertArrayEquals(
- "No deletes should apply to seq 7", new DataFile[0],
index.forDataFile(7, FILE_A));
- Assert.assertArrayEquals(
- "No deletes should apply to seq 10", new DataFile[0],
index.forDataFile(10, FILE_A));
-
- Assert.assertEquals(
- "No deletes should apply to FILE_B, partition not in index",
- 0,
- index.forDataFile(0, FILE_B).length);
-
- Assert.assertEquals(
- "No deletes should apply to FILE_C, no indexed delete files",
- 0,
- index.forDataFile(0, FILE_C).length);
+ assertThat(index.forDataFile(0, FILE_A))
+ .as("All deletes should apply to seq 0")
+ .isEqualTo(deleteFiles);
+
+ assertThat(index.forDataFile(3, FILE_A))
+ .as("All deletes should apply to seq 3")
+ .isEqualTo(deleteFiles);
+
+ assertThat(index.forDataFile(4, FILE_A))
+ .as("Last 3 deletes should apply to seq 4")
+ .isEqualTo(Arrays.copyOfRange(deleteFiles, 1, 4));
+
+ assertThat(index.forDataFile(5, FILE_A))
+ .as("Last 3 deletes should apply to seq 5")
+ .isEqualTo(Arrays.copyOfRange(deleteFiles, 1, 4));
+
+ assertThat(index.forDataFile(6, FILE_A))
+ .as("Last delete should apply to seq 6")
+ .isEqualTo(Arrays.copyOfRange(deleteFiles, 3, 4));
+
+ assertThat(index.forDataFile(7, FILE_A))
+ .as("No deletes should apply to seq 7")
+ .isEqualTo(new DataFile[0]);
+
+ assertThat(index.forDataFile(10, FILE_A))
+ .as("No deletes should apply to seq 10")
+ .isEqualTo(new DataFile[0]);
+
+ assertThat(index.forDataFile(0, FILE_B))
+ .as("No deletes should apply to FILE_B, partition not in index")
+ .hasSize(0);
+
+ assertThat(index.forDataFile(0, FILE_C))
+ .as("No deletes should apply to FILE_C, no indexed delete files")
+ .hasSize(0);
DataFile unpartitionedFileA = FILE_A.copy();
((BaseFile<?>) unpartitionedFileA).setSpecId(1);
- Assert.assertEquals(
- "No deletes should apply to FILE_A with a different specId",
- 0,
- index.forDataFile(0, unpartitionedFileA).length);
+ assertThat(index.forDataFile(0, unpartitionedFileA))
+ .as("No deletes should apply to FILE_A with a different specId")
+ .hasSize(0);
}
- @Test
+ @TestTemplate
public void testUnpartitionedTableScan() throws IOException {
- File location = temp.newFolder();
- Assert.assertTrue(location.delete());
+ File location = Files.createTempDirectory(temp, "junit").toFile();
+ assertThat(location.delete()).isTrue();
Table unpartitioned =
TestTables.create(location, "unpartitioned", SCHEMA,
PartitionSpec.unpartitioned(), 2);
@@ -259,16 +272,16 @@ public abstract class DeleteFileIndexTestBase<
unpartitioned.newRowDelta().addDeletes(unpartitionedPosDeletes).commit();
List<T> tasks =
Lists.newArrayList(newScan(unpartitioned).planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", unpartitionedFile.path(),
task.file().path());
- Assert.assertEquals("Should have one associated delete file", 1,
task.deletes().size());
- Assert.assertEquals(
- "Should have expected delete file",
- unpartitionedPosDeletes.path(),
- task.deletes().get(0).path());
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(unpartitionedFile.path());
+ assertThat(task.deletes()).as("Should have one associated delete
file").hasSize(1);
+ assertThat(task.deletes().get(0).path())
+ .as("Should have expected delete file")
+ .isEqualTo(unpartitionedPosDeletes.path());
// add a second delete file
DeleteFile unpartitionedEqDeletes =
unpartitionedEqDeletes(unpartitioned.spec());
@@ -276,80 +289,86 @@ public abstract class DeleteFileIndexTestBase<
tasks = Lists.newArrayList(newScan(unpartitioned).planFiles().iterator());
task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", unpartitionedFile.path(),
task.file().path());
- Assert.assertEquals("Should have two associated delete files", 2,
task.deletes().size());
- Assert.assertEquals(
- "Should have expected delete files",
- Sets.newHashSet(unpartitionedPosDeletes.path(),
unpartitionedEqDeletes.path()),
- Sets.newHashSet(Iterables.transform(task.deletes(),
ContentFile::path)));
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(unpartitionedFile.path());
+ assertThat(task.deletes()).as("Should have two associated delete
files").hasSize(2);
+ assertThat(Sets.newHashSet(Iterables.transform(task.deletes(),
ContentFile::path)))
+ .as("Should have expected delete files")
+ .isEqualTo(Sets.newHashSet(unpartitionedPosDeletes.path(),
unpartitionedEqDeletes.path()));
}
- @Test
+ @TestTemplate
public void testPartitionedTableWithPartitionPosDeletes() {
table.newAppend().appendFile(FILE_A).commit();
table.newRowDelta().addDeletes(FILE_A_POS_1).commit();
List<T> tasks = Lists.newArrayList(newScan(table).planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", FILE_A.path(),
task.file().path());
- Assert.assertEquals("Should have one associated delete file", 1,
task.deletes().size());
- Assert.assertEquals(
- "Should have only pos delete file", FILE_A_POS_1.path(),
task.deletes().get(0).path());
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(FILE_A.path());
+ assertThat(task.deletes()).as("Should have one associated delete
file").hasSize(1);
+ assertThat(task.deletes().get(0).path())
+ .as("Should have only pos delete file")
+ .isEqualTo(FILE_A_POS_1.path());
}
- @Test
+ @TestTemplate
public void testPartitionedTableWithPartitionEqDeletes() {
table.newAppend().appendFile(FILE_A).commit();
table.newRowDelta().addDeletes(FILE_A_EQ_1).commit();
List<T> tasks = Lists.newArrayList(newScan(table).planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", FILE_A.path(),
task.file().path());
- Assert.assertEquals("Should have one associated delete file", 1,
task.deletes().size());
- Assert.assertEquals(
- "Should have only pos delete file", FILE_A_EQ_1.path(),
task.deletes().get(0).path());
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(FILE_A.path());
+ assertThat(task.deletes()).as("Should have one associated delete
file").hasSize(1);
+ assertThat(task.deletes().get(0).path())
+ .as("Should have only pos delete file")
+ .isEqualTo(FILE_A_EQ_1.path());
}
- @Test
+ @TestTemplate
public void testPartitionedTableWithUnrelatedPartitionDeletes() {
table.newAppend().appendFile(FILE_B).commit();
table.newRowDelta().addDeletes(FILE_A_POS_1).addDeletes(FILE_A_EQ_1).commit();
List<T> tasks = Lists.newArrayList(newScan(table).planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", FILE_B.path(),
task.file().path());
- Assert.assertEquals("Should have no delete files to apply", 0,
task.deletes().size());
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(FILE_B.path());
+ assertThat(task.deletes()).as("Should have no delete files to
apply").hasSize(0);
}
- @Test
+ @TestTemplate
public void testPartitionedTableWithOlderPartitionDeletes() {
table.newRowDelta().addDeletes(FILE_A_POS_1).addDeletes(FILE_A_EQ_1).commit();
table.newAppend().appendFile(FILE_A).commit();
List<T> tasks = Lists.newArrayList(newScan(table).planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", FILE_A.path(),
task.file().path());
- Assert.assertEquals("Should have no delete files to apply", 0,
task.deletes().size());
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(FILE_A.path());
+ assertThat(task.deletes()).as("Should have no delete files to
apply").hasSize(0);
}
- @Test
+ @TestTemplate
public void testPartitionedTableScanWithGlobalDeletes() {
table.newAppend().appendFile(FILE_A).commit();
@@ -365,19 +384,19 @@ public abstract class DeleteFileIndexTestBase<
.commit();
List<T> tasks = Lists.newArrayList(newScan(table).planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", FILE_A.path(),
task.file().path());
- Assert.assertEquals("Should have one associated delete file", 1,
task.deletes().size());
- Assert.assertEquals(
- "Should have expected delete file",
- unpartitionedEqDeletes.path(),
- task.deletes().get(0).path());
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(FILE_A.path());
+ assertThat(task.deletes()).as("Should have one associated delete
file").hasSize(1);
+ assertThat(task.deletes().get(0).path())
+ .as("Should have expected delete file")
+ .isEqualTo(unpartitionedEqDeletes.path());
}
- @Test
+ @TestTemplate
public void testPartitionedTableScanWithGlobalAndPartitionDeletes() {
table.newAppend().appendFile(FILE_A).commit();
@@ -395,37 +414,39 @@ public abstract class DeleteFileIndexTestBase<
.commit();
List<T> tasks = Lists.newArrayList(newScan(table).planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", FILE_A.path(),
task.file().path());
- Assert.assertEquals("Should have two associated delete files", 2,
task.deletes().size());
- Assert.assertEquals(
- "Should have expected delete files",
- Sets.newHashSet(unpartitionedEqDeletes.path(), FILE_A_EQ_1.path()),
- Sets.newHashSet(Iterables.transform(task.deletes(),
ContentFile::path)));
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(FILE_A.path());
+ assertThat(task.deletes()).as("Should have two associated delete
files").hasSize(2);
+ assertThat(Sets.newHashSet(Iterables.transform(task.deletes(),
ContentFile::path)))
+ .as("Should have expected delete files")
+ .isEqualTo(Sets.newHashSet(unpartitionedEqDeletes.path(),
FILE_A_EQ_1.path()));
}
- @Test
+ @TestTemplate
public void testPartitionedTableSequenceNumbers() {
table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_EQ_1).addDeletes(FILE_A_POS_1).commit();
List<T> tasks = Lists.newArrayList(newScan(table).planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", FILE_A.path(),
task.file().path());
- Assert.assertEquals("Should have one associated delete file", 1,
task.deletes().size());
- Assert.assertEquals(
- "Should have only pos delete file", FILE_A_POS_1.path(),
task.deletes().get(0).path());
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(FILE_A.path());
+ assertThat(task.deletes()).as("Should have one associated delete
file").hasSize(1);
+ assertThat(task.deletes().get(0).path())
+ .as("Should have only pos delete file")
+ .isEqualTo(FILE_A_POS_1.path());
}
- @Test
+ @TestTemplate
public void testUnpartitionedTableSequenceNumbers() throws IOException {
- File location = temp.newFolder();
- Assert.assertTrue(location.delete());
+ File location = Files.createTempDirectory(temp, "junit").toFile();
+ assertThat(location.delete()).isTrue();
Table unpartitioned =
TestTables.create(location, "unpartitioned", SCHEMA,
PartitionSpec.unpartitioned(), 2);
@@ -441,30 +462,29 @@ public abstract class DeleteFileIndexTestBase<
.addDeletes(unpartitionedEqDeletes(unpartitioned.spec()))
.commit();
- Assert.assertEquals(
- "Table should contain 2 delete files",
- 2,
- (long)
+ assertThat(
unpartitioned
.currentSnapshot()
.deleteManifests(unpartitioned.io())
.get(0)
- .addedFilesCount());
+ .addedFilesCount())
+ .as("Table should contain 2 delete files")
+ .isEqualTo(2);
List<FileScanTask> tasks =
Lists.newArrayList(unpartitioned.newScan().planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", unpartitionedFile.path(),
task.file().path());
- Assert.assertEquals("Should have one associated delete file", 1,
task.deletes().size());
- Assert.assertEquals(
- "Should have only pos delete file",
- unpartitionedPosDeleteFile.path(),
- task.deletes().get(0).path());
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(unpartitionedFile.path());
+ assertThat(task.deletes()).as("Should have one associated delete
file").hasSize(1);
+ assertThat(task.deletes().get(0).path())
+ .as("Should have only pos delete file")
+ .isEqualTo(unpartitionedPosDeleteFile.path());
}
- @Test
+ @TestTemplate
public void testPartitionedTableWithExistingDeleteFile() {
table.updateProperties().set(TableProperties.MANIFEST_MERGE_ENABLED,
"false").commit();
@@ -480,47 +500,58 @@ public abstract class DeleteFileIndexTestBase<
.set(TableProperties.MANIFEST_MERGE_ENABLED, "true")
.commit();
- Assert.assertEquals(
- "Should have two delete manifests",
- 2,
- table.currentSnapshot().deleteManifests(table.io()).size());
+ assertThat(table.currentSnapshot().deleteManifests(table.io()))
+ .as("Should have two delete manifests")
+ .hasSize(2);
// merge delete manifests
table.newAppend().appendFile(FILE_B).commit();
- Assert.assertEquals(
- "Should have one delete manifest",
- 1,
- table.currentSnapshot().deleteManifests(table.io()).size());
- Assert.assertEquals(
- "Should have zero added delete file",
- 0,
-
table.currentSnapshot().deleteManifests(table.io()).get(0).addedFilesCount().intValue());
- Assert.assertEquals(
- "Should have zero deleted delete file",
- 0,
-
table.currentSnapshot().deleteManifests(table.io()).get(0).deletedFilesCount().intValue());
- Assert.assertEquals(
- "Should have two existing delete files",
- 2,
-
table.currentSnapshot().deleteManifests(table.io()).get(0).existingFilesCount().intValue());
+ assertThat(table.currentSnapshot().deleteManifests(table.io()))
+ .as("Should have one delete manifest")
+ .hasSize(1);
+
+ assertThat(
+
table.currentSnapshot().deleteManifests(table.io()).get(0).addedFilesCount().intValue())
+ .as("Should have zero added delete file")
+ .isEqualTo(0);
+
+ assertThat(
+ table
+ .currentSnapshot()
+ .deleteManifests(table.io())
+ .get(0)
+ .deletedFilesCount()
+ .intValue())
+ .as("Should have zero deleted delete file")
+ .isEqualTo(0);
+
+ assertThat(
+ table
+ .currentSnapshot()
+ .deleteManifests(table.io())
+ .get(0)
+ .existingFilesCount()
+ .intValue())
+ .as("Should have two existing delete files")
+ .isEqualTo(2);
List<T> tasks =
Lists.newArrayList(
newScan(table).filter(equal(bucket("data", BUCKETS_NUMBER),
0)).planFiles().iterator());
- Assert.assertEquals("Should have one task", 1, tasks.size());
+ assertThat(tasks).as("Should have one task").hasSize(1);
FileScanTask task = (FileScanTask) tasks.get(0);
- Assert.assertEquals(
- "Should have the correct data file path", FILE_A.path(),
task.file().path());
- Assert.assertEquals("Should have two associated delete files", 2,
task.deletes().size());
- Assert.assertEquals(
- "Should have expected delete files",
- Sets.newHashSet(FILE_A_EQ_1.path(), FILE_A_POS_1.path()),
- Sets.newHashSet(Iterables.transform(task.deletes(),
ContentFile::path)));
+ assertThat(task.file().path())
+ .as("Should have the correct data file path")
+ .isEqualTo(FILE_A.path());
+ assertThat(task.deletes()).as("Should have two associated delete
files").hasSize(2);
+ assertThat(Sets.newHashSet(Iterables.transform(task.deletes(),
ContentFile::path)))
+ .as("Should have expected delete files")
+ .isEqualTo(Sets.newHashSet(FILE_A_EQ_1.path(), FILE_A_POS_1.path()));
}
- @Test
+ @TestTemplate
public void testPositionDeletesGroup() {
DeleteFile file1 = withDataSequenceNumber(1, partitionedPosDeletes(SPEC,
FILE_A.partition()));
DeleteFile file2 = withDataSequenceNumber(2, partitionedPosDeletes(SPEC,
FILE_A.partition()));
@@ -554,7 +585,7 @@ public abstract class DeleteFileIndexTestBase<
assertThatThrownBy(() ->
group.add(file1)).isInstanceOf(IllegalStateException.class);
}
- @Test
+ @TestTemplate
public void testEqualityDeletesGroup() {
DeleteFile file1 = withDataSequenceNumber(1, partitionedEqDeletes(SPEC,
FILE_A.partition()));
DeleteFile file2 = withDataSequenceNumber(2, partitionedEqDeletes(SPEC,
FILE_A.partition()));
diff --git a/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java
b/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java
index 995a07f2eb..ad92d0f662 100644
--- a/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java
@@ -24,67 +24,67 @@ import static org.junit.Assert.assertEquals;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
+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;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.io.TempDir;
+@ExtendWith(ParameterizedTestExtension.class)
public abstract class FilterFilesTestBase<
ScanT extends Scan<ScanT, T, G>, T extends ScanTask, G extends
ScanTaskGroup<T>> {
- public final int formatVersion;
-
- public FilterFilesTestBase(int formatVersion) {
- this.formatVersion = formatVersion;
- }
+ @Parameter(index = 0)
+ protected int formatVersion;
protected abstract ScanT newScan(Table table);
- @Rule public TemporaryFolder temp = new TemporaryFolder();
+ @TempDir protected Path temp;
private final Schema schema =
new Schema(
required(1, "id", Types.IntegerType.get()), required(2, "data",
Types.StringType.get()));
private File tableDir = null;
- @Before
+ @BeforeEach
public void setupTableDir() throws IOException {
- this.tableDir = temp.newFolder();
+ this.tableDir = Files.createTempDirectory(temp, "junit").toFile();
}
- @After
+ @AfterEach
public void cleanupTables() {
TestTables.clearTables();
}
- @Test
+ @TestTemplate
public void testFilterFilesUnpartitionedTable() {
PartitionSpec spec = PartitionSpec.unpartitioned();
Table table = TestTables.create(tableDir, "test", schema, spec,
formatVersion);
testFilterFiles(table);
}
- @Test
+ @TestTemplate
public void testCaseInsensitiveFilterFilesUnpartitionedTable() {
PartitionSpec spec = PartitionSpec.unpartitioned();
Table table = TestTables.create(tableDir, "test", schema, spec,
formatVersion);
testCaseInsensitiveFilterFiles(table);
}
- @Test
+ @TestTemplate
public void testFilterFilesPartitionedTable() {
PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data",
16).build();
Table table = TestTables.create(tableDir, "test", schema, spec,
formatVersion);
testFilterFiles(table);
}
- @Test
+ @TestTemplate
public void testCaseInsensitiveFilterFilesPartitionedTable() {
PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data",
16).build();
Table table = TestTables.create(tableDir, "test", schema, spec,
formatVersion);
diff --git
a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java
b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java
index a8f98f82cc..3ba74c3b4b 100644
---
a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java
+++
b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java
@@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat;
import java.io.IOException;
import java.time.Duration;
+import java.util.Arrays;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.iceberg.expressions.Expressions;
@@ -34,21 +35,24 @@ import org.apache.iceberg.metrics.ScanMetricsResult;
import org.apache.iceberg.metrics.ScanReport;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.assertj.core.api.InstanceOfAssertFactories;
-import org.junit.Test;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+@ExtendWith(ParameterizedTestExtension.class)
public abstract class ScanPlanningAndReportingTestBase<
ScanT extends Scan<ScanT, T, G>, T extends ScanTask, G extends
ScanTaskGroup<T>>
- extends TableTestBase {
+ extends TestBase {
private final TestMetricsReporter reporter = new TestMetricsReporter();
- public ScanPlanningAndReportingTestBase() {
- super(2);
+ @Parameters(name = "formatVersion = {0}")
+ public static List<Object> parameters() {
+ return Arrays.asList(2);
}
protected abstract ScanT newScan(Table table);
- @Test
+ @TestTemplate
public void noDuplicatesInScanContext() {
TableScanContext context = TableScanContext.empty();
assertThat(context.metricsReporter()).isInstanceOf(LoggingMetricsReporter.class);
@@ -76,7 +80,7 @@ public abstract class ScanPlanningAndReportingTestBase<
.containsExactlyInAnyOrder(LoggingMetricsReporter.instance(), first,
second);
}
- @Test
+ @TestTemplate
public void scanningWithMultipleReporters() throws IOException {
String tableName = "scan-with-multiple-reporters";
Table table =
@@ -106,7 +110,7 @@ public abstract class ScanPlanningAndReportingTestBase<
assertThat(reportedCount.get()).isEqualTo(2);
}
- @Test
+ @TestTemplate
public void scanningWithMultipleDataManifests() throws IOException {
String tableName = "multiple-data-manifests";
Table table =
@@ -169,7 +173,7 @@ public abstract class ScanPlanningAndReportingTestBase<
assertThat(result.skippedDeleteFiles().value()).isEqualTo(0);
}
- @Test
+ @TestTemplate
public void scanningWithDeletes() throws IOException {
Table table =
TestTables.create(
@@ -212,7 +216,7 @@ public abstract class ScanPlanningAndReportingTestBase<
assertThat(result.positionalDeleteFiles().value()).isEqualTo(2);
}
- @Test
+ @TestTemplate
public void scanningWithSkippedDataFiles() throws IOException {
String tableName = "scan-planning-with-skipped-data-files";
Table table =
@@ -252,7 +256,7 @@ public abstract class ScanPlanningAndReportingTestBase<
assertThat(result.totalDeleteFileSizeInBytes().value()).isEqualTo(0L);
}
- @Test
+ @TestTemplate
public void scanningWithSkippedDeleteFiles() throws IOException {
String tableName = "scan-planning-with-skipped-delete-files";
Table table =
@@ -296,7 +300,7 @@ public abstract class ScanPlanningAndReportingTestBase<
assertThat(result.positionalDeleteFiles().value()).isEqualTo(0);
}
- @Test
+ @TestTemplate
public void scanningWithEqualityAndPositionalDeleteFiles() throws
IOException {
String tableName = "scan-planning-with-eq-and-pos-delete-files";
Table table =
diff --git a/core/src/test/java/org/apache/iceberg/ScanTestBase.java
b/core/src/test/java/org/apache/iceberg/ScanTestBase.java
index 48a8ccbaa9..51d692c1b4 100644
--- a/core/src/test/java/org/apache/iceberg/ScanTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/ScanTestBase.java
@@ -19,66 +19,54 @@
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 static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.assertj.core.api.Assumptions.assumeThat;
import java.io.File;
import java.io.IOException;
+import java.nio.file.Files;
import java.util.Collections;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.io.CloseableIterable;
-import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.types.Types;
-import org.assertj.core.api.Assertions;
-import org.junit.Assert;
-import org.junit.Assume;
-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)
+@ExtendWith(ParameterizedTestExtension.class)
public abstract class ScanTestBase<
ScanT extends Scan<ScanT, T, G>, T extends ScanTask, G extends
ScanTaskGroup<T>>
- extends TableTestBase {
-
- @Parameterized.Parameters(name = "formatVersion = {0}")
- public static Object[] parameters() {
- return new Object[] {1, 2};
- }
-
- public ScanTestBase(int formatVersion) {
- super(formatVersion);
- }
+ extends TestBase {
protected abstract ScanT newScan();
- @Test
+ @TestTemplate
public void testTableScanHonorsSelect() {
ScanT scan = newScan().select(Collections.singletonList("id"));
Schema expectedSchema = new Schema(required(1, "id",
Types.IntegerType.get()));
- assertEquals(
- "A tableScan.select() should prune the schema",
- expectedSchema.asStruct(),
- scan.schema().asStruct());
+ assertThat(scan.schema().asStruct())
+ .as("A tableScan.select() should prune the schema")
+ .isEqualTo(expectedSchema.asStruct());
}
- @Test
+ @TestTemplate
public void testTableBothProjectAndSelect() {
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
() ->
newScan().select(Collections.singletonList("id")).project(SCHEMA.select("data")))
.isInstanceOf(IllegalStateException.class)
.hasMessage("Cannot set projection schema when columns are selected");
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
() ->
newScan().project(SCHEMA.select("data")).select(Collections.singletonList("id")))
.isInstanceOf(IllegalStateException.class)
.hasMessage("Cannot select columns when projection schema is set");
}
- @Test
+ @TestTemplate
public void testTableScanHonorsSelectWithoutCaseSensitivity() {
ScanT scan1 =
newScan().caseSensitive(false).select(Collections.singletonList("ID"));
// order of refinements shouldn't matter
@@ -86,29 +74,29 @@ public abstract class ScanTestBase<
Schema expectedSchema = new Schema(required(1, "id",
Types.IntegerType.get()));
- assertEquals(
- "A tableScan.select() should prune the schema without case
sensitivity",
- expectedSchema.asStruct(),
- scan1.schema().asStruct());
+ assertThat(scan1.schema().asStruct())
+ .as("A tableScan.select() should prune the schema without case
sensitivity")
+ .isEqualTo(expectedSchema.asStruct());
- assertEquals(
- "A tableScan.select() should prune the schema regardless of scan
refinement order",
- expectedSchema.asStruct(),
- scan2.schema().asStruct());
+ assertThat(scan2.schema().asStruct())
+ .as("A tableScan.select() should prune the schema regardless of scan
refinement order")
+ .isEqualTo(expectedSchema.asStruct());
}
- @Test
+ @TestTemplate
public void testTableScanHonorsIgnoreResiduals() throws IOException {
table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit();
ScanT scan1 = newScan().filter(Expressions.equal("id", 5));
try (CloseableIterable<G> groups = scan1.planTasks()) {
- Assert.assertTrue("Tasks should not be empty", Iterables.size(groups) >
0);
+ assertThat(groups).as("Tasks should not be empty").isNotEmpty();
for (G group : groups) {
for (T task : group.tasks()) {
Expression residual = ((ContentScanTask<?>) task).residual();
- Assert.assertNotEquals("Residuals must be preserved",
Expressions.alwaysTrue(), residual);
+ assertThat(residual)
+ .as("Residuals must be preserved")
+ .isNotEqualTo(Expressions.alwaysTrue());
}
}
}
@@ -116,17 +104,19 @@ public abstract class ScanTestBase<
ScanT scan2 = newScan().filter(Expressions.equal("id",
5)).ignoreResiduals();
try (CloseableIterable<G> groups = scan2.planTasks()) {
- Assert.assertTrue("Tasks should not be empty", Iterables.size(groups) >
0);
+ assertThat(groups).as("Tasks should not be empty").isNotEmpty();
for (G group : groups) {
for (T task : group.tasks()) {
Expression residual = ((ContentScanTask<?>) task).residual();
- Assert.assertEquals("Residuals must be ignored",
Expressions.alwaysTrue(), residual);
+ assertThat(residual)
+ .as("Residuals must be preserved")
+ .isEqualTo(Expressions.alwaysTrue());
}
}
}
}
- @Test
+ @TestTemplate
public void testTableScanWithPlanExecutor() {
table.newFastAppend().appendFile(FILE_A).commit();
table.newFastAppend().appendFile(FILE_B).commit();
@@ -144,20 +134,22 @@ public abstract class ScanTestBase<
true); // daemon threads will be terminated abruptly
when the JVM exits
return thread;
}));
- Assert.assertEquals(2, Iterables.size(scan.planFiles()));
- Assert.assertTrue("Thread should be created in provided pool",
planThreadsIndex.get() > 0);
+ assertThat(scan.planFiles()).hasSize(2);
+ assertThat(planThreadsIndex.get())
+ .as("Thread should be created in provided pool")
+ .isGreaterThan(0);
}
- @Test
+ @TestTemplate
public void testReAddingPartitionField() throws Exception {
- Assume.assumeTrue(formatVersion == 2);
+ assumeThat(formatVersion).isEqualTo(2);
Schema schema =
new Schema(
required(1, "a", Types.IntegerType.get()),
required(2, "b", Types.StringType.get()),
required(3, "data", Types.IntegerType.get()));
PartitionSpec initialSpec =
PartitionSpec.builderFor(schema).identity("a").build();
- File dir = temp.newFolder();
+ File dir = Files.createTempDirectory(temp, "junit").toFile();
dir.delete();
this.table = TestTables.create(dir, "test_part_evolution", schema,
initialSpec, formatVersion);
table
@@ -208,29 +200,29 @@ public abstract class ScanTestBase<
TableScan scan1 = table.newScan().filter(Expressions.equal("b", "1"));
try (CloseableIterable<CombinedScanTask> tasks = scan1.planTasks()) {
- Assert.assertTrue("There should be 1 combined task",
Iterables.size(tasks) == 1);
+ assertThat(tasks).as("There should be 1 combined task").hasSize(1);
for (CombinedScanTask combinedScanTask : tasks) {
- Assert.assertEquals(
- "All 4 files should match b=1 filter", 4,
combinedScanTask.files().size());
+ assertThat(combinedScanTask.files()).as("All 4 files should match b=1
filter").hasSize(4);
}
}
TableScan scan2 = table.newScan().filter(Expressions.equal("a", 2));
try (CloseableIterable<CombinedScanTask> tasks = scan2.planTasks()) {
- Assert.assertTrue("There should be 1 combined task",
Iterables.size(tasks) == 1);
+ assertThat(tasks).as("There should be 1 combined task").hasSize(1);
for (CombinedScanTask combinedScanTask : tasks) {
- Assert.assertEquals(
- "a=2 and file without a in spec should match", 2,
combinedScanTask.files().size());
+ assertThat(combinedScanTask.files())
+ .as("a=2 and file without a in spec should match")
+ .hasSize(2);
}
}
}
- @Test
+ @TestTemplate
public void testDataFileSorted() throws Exception {
Schema schema =
new Schema(
required(1, "a", Types.IntegerType.get()), required(2, "b",
Types.StringType.get()));
- File dir = temp.newFolder();
+ File dir = Files.createTempDirectory(temp, "junit").toFile();
dir.delete();
this.table =
TestTables.create(
@@ -250,7 +242,7 @@ public abstract class ScanTestBase<
TableScan scan = table.newScan();
try (CloseableIterable<FileScanTask> tasks = scan.planFiles()) {
for (FileScanTask fileScanTask : tasks) {
- Assertions.assertThat(fileScanTask.file().sortOrderId()).isEqualTo(1);
+ assertThat(fileScanTask.file().sortOrderId()).isEqualTo(1);
}
}
}
diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java
b/core/src/test/java/org/apache/iceberg/TestBase.java
index 6fc048ded8..ebd527ded3 100644
--- a/core/src/test/java/org/apache/iceberg/TestBase.java
+++ b/core/src/test/java/org/apache/iceberg/TestBase.java
@@ -173,8 +173,8 @@ public class TestBase {
public TestTables.TestTable table = null;
@Parameters(name = "formatVersion = {0}")
- protected static List<Object[]> parameters() {
- return Arrays.asList(new Object[] {1}, new Object[] {2});
+ protected static List<Object> parameters() {
+ return Arrays.asList(1, 2);
}
@Parameter protected int formatVersion;
diff --git
a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java
b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java
index c40cc35d24..7b011b9134 100644
--- a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java
+++ b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java
@@ -18,23 +18,23 @@
*/
package org.apache.iceberg;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
-import org.assertj.core.api.Assertions;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+@ExtendWith(ParameterizedTestExtension.class)
public class TestBaseIncrementalAppendScan
extends ScanTestBase<IncrementalAppendScan, FileScanTask,
CombinedScanTask> {
- public TestBaseIncrementalAppendScan(int formatVersion) {
- super(formatVersion);
- }
@Override
protected IncrementalAppendScan newScan() {
return table.newIncrementalAppendScan();
}
- @Test
+ @TestTemplate
public void testFromSnapshotInclusive() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -44,21 +44,21 @@ public class TestBaseIncrementalAppendScan
long snapshotCId = table.currentSnapshot().snapshotId();
IncrementalAppendScan scan = newScan().fromSnapshotInclusive(snapshotAId);
- Assert.assertEquals(3, Iterables.size(scan.planFiles()));
+ assertThat(scan.planFiles()).hasSize(3);
IncrementalAppendScan scanWithToSnapshot =
newScan().fromSnapshotInclusive(snapshotAId).toSnapshot(snapshotCId);
- Assert.assertEquals(3, Iterables.size(scanWithToSnapshot.planFiles()));
+ assertThat(scanWithToSnapshot.planFiles()).hasSize(3);
}
- @Test
+ @TestTemplate
public void fromSnapshotInclusiveWithNonExistingRef() {
- Assertions.assertThatThrownBy(() ->
newScan().fromSnapshotInclusive("nonExistingRef"))
+ assertThatThrownBy(() -> newScan().fromSnapshotInclusive("nonExistingRef"))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot find ref: nonExistingRef");
}
- @Test
+ @TestTemplate
public void fromSnapshotInclusiveWithTag() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -77,31 +77,30 @@ public class TestBaseIncrementalAppendScan
---- snapshotAId(tag:t1) ---- snapshotMainB(tag:t2) ---- currentSnapshot
*/
IncrementalAppendScan scan =
newScan().fromSnapshotInclusive(tagSnapshotAName);
- Assertions.assertThat(scan.planFiles()).hasSize(5);
+ assertThat(scan.planFiles()).hasSize(5);
IncrementalAppendScan scanWithToSnapshot =
newScan().fromSnapshotInclusive(tagSnapshotAName).toSnapshot(tagSnapshotBName);
- Assertions.assertThat(scanWithToSnapshot.planFiles()).hasSize(3);
+ assertThat(scanWithToSnapshot.planFiles()).hasSize(3);
}
- @Test
+ @TestTemplate
public void fromSnapshotInclusiveWithBranchShouldFail() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
String branchName = "b1";
table.manageSnapshots().createBranch(branchName, snapshotAId).commit();
- Assertions.assertThatThrownBy(() ->
newScan().fromSnapshotInclusive(branchName))
+ assertThatThrownBy(() -> newScan().fromSnapshotInclusive(branchName))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage(String.format("Ref %s is not a tag", branchName));
- Assertions.assertThatThrownBy(
- () ->
newScan().fromSnapshotInclusive(snapshotAId).toSnapshot(branchName))
+ assertThatThrownBy(() ->
newScan().fromSnapshotInclusive(snapshotAId).toSnapshot(branchName))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage(String.format("Ref %s is not a tag", branchName));
}
- @Test
+ @TestTemplate
public void testUseBranch() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -138,40 +137,40 @@ public class TestBaseIncrementalAppendScan
snapshotBranchC(branch:b1)
*/
IncrementalAppendScan scan =
newScan().fromSnapshotInclusive(tagSnapshotAName);
- Assertions.assertThat(scan.planFiles()).hasSize(5);
+ assertThat(scan.planFiles()).hasSize(5);
IncrementalAppendScan scan2 =
newScan().fromSnapshotInclusive(tagSnapshotAName).useBranch(branchName);
- Assertions.assertThat(scan2.planFiles()).hasSize(3);
+ assertThat(scan2.planFiles()).hasSize(3);
IncrementalAppendScan scan3 =
newScan().toSnapshot(snapshotBranchBId).useBranch(branchName);
- Assertions.assertThat(scan3.planFiles()).hasSize(2);
+ assertThat(scan3.planFiles()).hasSize(2);
IncrementalAppendScan scan4 =
newScan().toSnapshot(snapshotBranchCId).useBranch(branchName);
- Assertions.assertThat(scan4.planFiles()).hasSize(3);
+ assertThat(scan4.planFiles()).hasSize(3);
IncrementalAppendScan scan5 =
newScan()
.fromSnapshotExclusive(tagSnapshotAName)
.toSnapshot(snapshotBranchBId)
.useBranch(branchName);
- Assertions.assertThat(scan5.planFiles()).hasSize(1);
+ assertThat(scan5.planFiles()).hasSize(1);
}
- @Test
+ @TestTemplate
public void testUseBranchWithTagShouldFail() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
String tagSnapshotAName = "t1";
table.manageSnapshots().createTag(tagSnapshotAName, snapshotAId).commit();
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
() ->
newScan().fromSnapshotInclusive(snapshotAId).useBranch(tagSnapshotAName))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage(String.format("Ref %s is not a branch", tagSnapshotAName));
}
- @Test
+ @TestTemplate
public void testUseBranchWithInvalidSnapshotShouldFail() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -193,12 +192,12 @@ public class TestBaseIncrementalAppendScan
\files:FILE_C
snapshotBranchB(branch:b1)
*/
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
() ->
newScan().toSnapshot(snapshotMainBId).useBranch(branchName).planFiles())
.isInstanceOf(IllegalArgumentException.class)
.hasMessageContaining("End snapshot is not a valid snapshot on the
current branch");
- Assertions.assertThatThrownBy(
+ assertThatThrownBy(
() ->
newScan().fromSnapshotInclusive(snapshotMainBId).useBranch(branchName).planFiles())
.isInstanceOf(IllegalArgumentException.class)
@@ -208,14 +207,14 @@ public class TestBaseIncrementalAppendScan
snapshotMainBId, snapshotBranchBId));
}
- @Test
+ @TestTemplate
public void testUseBranchWithNonExistingRef() {
- Assertions.assertThatThrownBy(() -> newScan().useBranch("nonExistingRef"))
+ assertThatThrownBy(() -> newScan().useBranch("nonExistingRef"))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot find ref: nonExistingRef");
}
- @Test
+ @TestTemplate
public void testFromSnapshotExclusive() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -225,14 +224,14 @@ public class TestBaseIncrementalAppendScan
long snapshotCId = table.currentSnapshot().snapshotId();
IncrementalAppendScan scan = newScan().fromSnapshotExclusive(snapshotAId);
- Assert.assertEquals(2, Iterables.size(scan.planFiles()));
+ assertThat(scan.planFiles()).hasSize(2);
IncrementalAppendScan scanWithToSnapshot =
newScan().fromSnapshotExclusive(snapshotAId).toSnapshot(snapshotBId);
- Assert.assertEquals(1, Iterables.size(scanWithToSnapshot.planFiles()));
+ assertThat(scanWithToSnapshot.planFiles()).hasSize(1);
}
- @Test
+ @TestTemplate
public void testFromSnapshotExclusiveForExpiredParent() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -245,21 +244,21 @@ public class TestBaseIncrementalAppendScan
table.expireSnapshots().expireOlderThan(expireTimestampSnapshotA).commit();
IncrementalAppendScan scan = newScan().fromSnapshotExclusive(snapshotAId);
- Assert.assertEquals(2, Iterables.size(scan.planFiles()));
+ assertThat(scan.planFiles()).hasSize(2);
IncrementalAppendScan scanWithToSnapshot =
newScan().fromSnapshotExclusive(snapshotAId).toSnapshot(snapshotBId);
- Assert.assertEquals(1, Iterables.size(scanWithToSnapshot.planFiles()));
+ assertThat(scanWithToSnapshot.planFiles()).hasSize(1);
}
- @Test
+ @TestTemplate
public void fromSnapshotExclusiveWithNonExistingRef() {
- Assertions.assertThatThrownBy(() ->
newScan().fromSnapshotExclusive("nonExistingRef"))
+ assertThatThrownBy(() -> newScan().fromSnapshotExclusive("nonExistingRef"))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot find ref: nonExistingRef");
}
- @Test
+ @TestTemplate
public void testFromSnapshotExclusiveWithTag() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -278,26 +277,26 @@ public class TestBaseIncrementalAppendScan
---- snapshotAId(tag:t1) ---- snapshotMainB(tag:t2) ---- currentSnapshot
*/
IncrementalAppendScan scan =
newScan().fromSnapshotExclusive(tagSnapshotAName);
- Assertions.assertThat(scan.planFiles()).hasSize(4);
+ assertThat(scan.planFiles()).hasSize(4);
IncrementalAppendScan scanWithToSnapshot =
newScan().fromSnapshotExclusive(tagSnapshotAName).toSnapshot(tagSnapshotBName);
- Assertions.assertThat(scanWithToSnapshot.planFiles()).hasSize(2);
+ assertThat(scanWithToSnapshot.planFiles()).hasSize(2);
}
- @Test
+ @TestTemplate
public void fromSnapshotExclusiveWithBranchShouldFail() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
String branchName = "b1";
table.manageSnapshots().createBranch(branchName, snapshotAId).commit();
- Assertions.assertThatThrownBy(() ->
newScan().fromSnapshotExclusive(branchName))
+ assertThatThrownBy(() -> newScan().fromSnapshotExclusive(branchName))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage(String.format("Ref %s is not a tag", branchName));
}
- @Test
+ @TestTemplate
public void testToSnapshot() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -307,10 +306,10 @@ public class TestBaseIncrementalAppendScan
long snapshotCId = table.currentSnapshot().snapshotId();
IncrementalAppendScan scan = newScan().toSnapshot(snapshotBId);
- Assert.assertEquals(2, Iterables.size(scan.planFiles()));
+ assertThat(scan.planFiles()).hasSize(2);
}
- @Test
+ @TestTemplate
public void testToSnapshotWithTag() {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -339,20 +338,20 @@ public class TestBaseIncrementalAppendScan
snapshotBranchB(branch:b1, tag:t2)
*/
IncrementalAppendScan scan = newScan().toSnapshot(tagSnapshotMainBName);
- Assertions.assertThat(scan.planFiles()).hasSize(2);
+ assertThat(scan.planFiles()).hasSize(2);
IncrementalAppendScan scan2 = newScan().toSnapshot(tagSnapshotBranchBName);
- Assertions.assertThat(scan2.planFiles()).hasSize(3);
+ assertThat(scan2.planFiles()).hasSize(3);
}
- @Test
+ @TestTemplate
public void testToSnapshotWithNonExistingRef() {
- Assertions.assertThatThrownBy(() -> newScan().toSnapshot("nonExistingRef"))
+ assertThatThrownBy(() -> newScan().toSnapshot("nonExistingRef"))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot find ref: nonExistingRef");
}
- @Test
+ @TestTemplate
public void testToSnapshotWithBranchShouldFail() {
table.newFastAppend().appendFile(FILE_A).commit();
table.newFastAppend().appendFile(FILE_B).commit();
@@ -361,12 +360,12 @@ public class TestBaseIncrementalAppendScan
String branchName = "b1";
table.manageSnapshots().createBranch(branchName, snapshotId).commit();
- Assertions.assertThatThrownBy(() -> newScan().toSnapshot(branchName))
+ assertThatThrownBy(() -> newScan().toSnapshot(branchName))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage(String.format("Ref %s is not a tag", branchName));
}
- @Test
+ @TestTemplate
public void testMultipleRootSnapshots() throws Exception {
table.newFastAppend().appendFile(FILE_A).commit();
long snapshotAId = table.currentSnapshot().snapshotId();
@@ -388,12 +387,12 @@ public class TestBaseIncrementalAppendScan
// scan should discover snapshot C and D
IncrementalAppendScan scan = newScan().toSnapshot(snapshotDId);
- Assert.assertEquals(2, Iterables.size(scan.planFiles()));
+ assertThat(scan.planFiles()).hasSize(2);
// scan should fail because snapshot B is not an ancestor of snapshot D
IncrementalAppendScan scanShouldFail =
newScan().fromSnapshotExclusive(snapshotBId).toSnapshot(snapshotDId);
- Assertions.assertThatThrownBy(() ->
Iterables.size(scanShouldFail.planFiles()))
+ assertThatThrownBy(() -> Iterables.size(scanShouldFail.planFiles()))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage(
String.format(
@@ -403,7 +402,7 @@ public class TestBaseIncrementalAppendScan
// scan should fail because snapshot B is not an ancestor of snapshot D
IncrementalAppendScan scanShouldFailInclusive =
newScan().fromSnapshotInclusive(snapshotBId).toSnapshot(snapshotDId);
- Assertions.assertThatThrownBy(() ->
Iterables.size(scanShouldFailInclusive.planFiles()))
+ assertThatThrownBy(() ->
Iterables.size(scanShouldFailInclusive.planFiles()))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage(
String.format(
diff --git
a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java
b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java
index dcda2f354a..bb647627a0 100644
---
a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java
+++
b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java
@@ -20,6 +20,8 @@ package org.apache.iceberg;
import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED;
import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
import java.io.IOException;
import java.util.Comparator;
@@ -32,24 +34,20 @@ import
org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.assertj.core.api.Assertions;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Test;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+@ExtendWith(ParameterizedTestExtension.class)
public class TestBaseIncrementalChangelogScan
extends ScanTestBase<
IncrementalChangelogScan, ChangelogScanTask,
ScanTaskGroup<ChangelogScanTask>> {
- public TestBaseIncrementalChangelogScan(int formatVersion) {
- super(formatVersion);
- }
-
@Override
protected IncrementalChangelogScan newScan() {
return table.newIncrementalChangelogScan();
}
- @Test
+ @TestTemplate
public void testDataFilters() {
table.newFastAppend().appendFile(FILE_A).commit();
@@ -60,7 +58,7 @@ public class TestBaseIncrementalChangelogScan
Snapshot snap2 = table.currentSnapshot();
- Assert.assertEquals("Must be 2 data manifests", 2,
snap2.dataManifests(table.io()).size());
+ assertThat(snap2.dataManifests(table.io())).as("Must be 2 data
manifests").hasSize(2);
withUnavailableLocations(
ImmutableList.of(snap1DataManifest.path()),
@@ -70,17 +68,17 @@ public class TestBaseIncrementalChangelogScan
List<ChangelogScanTask> tasks = plan(scan);
- Assert.assertEquals("Must have 1 task", 1, tasks.size());
+ assertThat(tasks).as("Must have 1 task").hasSize(1);
AddedRowsScanTask t1 = (AddedRowsScanTask)
Iterables.getOnlyElement(tasks);
- Assert.assertEquals("Ordinal must match", 1, t1.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap2.snapshotId(),
t1.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_B.path(),
t1.file().path());
- Assert.assertTrue("Must be no deletes", t1.deletes().isEmpty());
+ assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(1);
+ assertThat(t1.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap2.snapshotId());
+ assertThat(t1.file().path()).as("Data file must
match").isEqualTo(FILE_B.path());
+ assertThat(t1.deletes()).as("Must be no deletes").isEmpty();
});
}
- @Test
+ @TestTemplate
public void testOverwrites() {
table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit();
@@ -95,22 +93,22 @@ public class TestBaseIncrementalChangelogScan
List<ChangelogScanTask> tasks = plan(scan);
- Assert.assertEquals("Must have 2 tasks", 2, tasks.size());
+ assertThat(tasks).as("Must have 2 tasks").hasSize(2);
AddedRowsScanTask t1 = (AddedRowsScanTask) tasks.get(0);
- Assert.assertEquals("Ordinal must match", 0, t1.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap2.snapshotId(),
t1.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_A2.path(),
t1.file().path());
- Assert.assertTrue("Must be no deletes", t1.deletes().isEmpty());
+ assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0);
+ assertThat(t1.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap2.snapshotId());
+ assertThat(t1.file().path()).as("Data file must
match").isEqualTo(FILE_A2.path());
+ assertThat(t1.deletes()).as("Must be no deletes").isEmpty();
DeletedDataFileScanTask t2 = (DeletedDataFileScanTask) tasks.get(1);
- Assert.assertEquals("Ordinal must match", 0, t2.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap2.snapshotId(),
t2.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_A.path(),
t2.file().path());
- Assert.assertTrue("Must be no deletes", t2.existingDeletes().isEmpty());
+ assertThat(t2.changeOrdinal()).as("Ordinal must match").isEqualTo(0);
+ assertThat(t2.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap2.snapshotId());
+ assertThat(t2.file().path()).as("Data file must
match").isEqualTo(FILE_A.path());
+ assertThat(t2.existingDeletes()).as("Must be no deletes").isEmpty();
}
- @Test
+ @TestTemplate
public void testFileDeletes() {
table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit();
@@ -125,16 +123,16 @@ public class TestBaseIncrementalChangelogScan
List<ChangelogScanTask> tasks = plan(scan);
- Assert.assertEquals("Must have 1 tasks", 1, tasks.size());
+ assertThat(tasks).as("Must have 1 task").hasSize(1);
DeletedDataFileScanTask t1 = (DeletedDataFileScanTask)
Iterables.getOnlyElement(tasks);
- Assert.assertEquals("Ordinal must match", 0, t1.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap2.snapshotId(),
t1.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_A.path(),
t1.file().path());
- Assert.assertTrue("Must be no deletes", t1.existingDeletes().isEmpty());
+ assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0);
+ assertThat(t1.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap2.snapshotId());
+ assertThat(t1.file().path()).as("Data file must
match").isEqualTo(FILE_A.path());
+ assertThat(t1.existingDeletes()).as("Must be no deletes").isEmpty();
}
- @Test
+ @TestTemplate
public void testExistingEntriesInNewDataManifestsAreIgnored() {
table
.updateProperties()
@@ -151,23 +149,23 @@ public class TestBaseIncrementalChangelogScan
Snapshot snap3 = table.currentSnapshot();
ManifestFile manifest =
Iterables.getOnlyElement(snap3.dataManifests(table.io()));
- Assert.assertTrue("Manifest must have existing files",
manifest.hasExistingFiles());
+ assertThat(manifest.hasExistingFiles()).as("Manifest must have existing
files").isTrue();
IncrementalChangelogScan scan =
newScan().fromSnapshotInclusive(snap3.snapshotId()).toSnapshot(snap3.snapshotId());
List<ChangelogScanTask> tasks = plan(scan);
- Assert.assertEquals("Must have 1 task", 1, tasks.size());
+ assertThat(tasks).as("Must have 1 task").hasSize(1);
AddedRowsScanTask t1 = (AddedRowsScanTask) Iterables.getOnlyElement(tasks);
- Assert.assertEquals("Ordinal must match", 0, t1.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap3.snapshotId(),
t1.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_C.path(),
t1.file().path());
- Assert.assertTrue("Must be no deletes", t1.deletes().isEmpty());
+ assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0);
+ assertThat(t1.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap3.snapshotId());
+ assertThat(t1.file().path()).as("Data file must
match").isEqualTo(FILE_C.path());
+ assertThat(t1.deletes()).as("Must be no deletes").isEmpty();
}
- @Test
+ @TestTemplate
public void testManifestRewritesAreIgnored() throws IOException {
table.newAppend().appendFile(FILE_A).commit();
@@ -199,28 +197,28 @@ public class TestBaseIncrementalChangelogScan
List<ChangelogScanTask> tasks = plan(newScan());
- Assert.assertEquals("Must have 3 tasks", 3, tasks.size());
+ assertThat(tasks).as("Must have 3 tasks").hasSize(3);
AddedRowsScanTask t1 = (AddedRowsScanTask) tasks.get(0);
- Assert.assertEquals("Ordinal must match", 0, t1.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap1.snapshotId(),
t1.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_A.path(),
t1.file().path());
- Assert.assertTrue("Must be no deletes", t1.deletes().isEmpty());
+ assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0);
+ assertThat(t1.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap1.snapshotId());
+ assertThat(t1.file().path()).as("Data file must
match").isEqualTo(FILE_A.path());
+ assertThat(t1.deletes()).as("Must be no deletes").isEmpty();
AddedRowsScanTask t2 = (AddedRowsScanTask) tasks.get(1);
- Assert.assertEquals("Ordinal must match", 1, t2.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap2.snapshotId(),
t2.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_B.path(),
t2.file().path());
- Assert.assertTrue("Must be no deletes", t2.deletes().isEmpty());
+ assertThat(t2.changeOrdinal()).as("Ordinal must match").isEqualTo(1);
+ assertThat(t2.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap2.snapshotId());
+ assertThat(t2.file().path()).as("Data file must
match").isEqualTo(FILE_B.path());
+ assertThat(t2.deletes()).as("Must be no deletes").isEmpty();
AddedRowsScanTask t3 = (AddedRowsScanTask) tasks.get(2);
- Assert.assertEquals("Ordinal must match", 2, t3.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap4.snapshotId(),
t3.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_C.path(),
t3.file().path());
- Assert.assertTrue("Must be no deletes", t3.deletes().isEmpty());
+ assertThat(t3.changeOrdinal()).as("Ordinal must match").isEqualTo(2);
+ assertThat(t3.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap4.snapshotId());
+ assertThat(t3.file().path()).as("Data file must
match").isEqualTo(FILE_C.path());
+ assertThat(t3.deletes()).as("Must be no deletes").isEmpty();
}
- @Test
+ @TestTemplate
public void testDataFileRewrites() {
table.newAppend().appendFile(FILE_A).commit();
@@ -234,24 +232,24 @@ public class TestBaseIncrementalChangelogScan
List<ChangelogScanTask> tasks = plan(newScan());
- Assert.assertEquals("Must have 2 tasks", 2, tasks.size());
+ assertThat(tasks).as("Must have 2 tasks").hasSize(2);
AddedRowsScanTask t1 = (AddedRowsScanTask) tasks.get(0);
- Assert.assertEquals("Ordinal must match", 0, t1.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap1.snapshotId(),
t1.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_A.path(),
t1.file().path());
- Assert.assertTrue("Must be no deletes", t1.deletes().isEmpty());
+ assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0);
+ assertThat(t1.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap1.snapshotId());
+ assertThat(t1.file().path()).as("Data file must
match").isEqualTo(FILE_A.path());
+ assertThat(t1.deletes()).as("Must be no deletes").isEmpty();
AddedRowsScanTask t2 = (AddedRowsScanTask) tasks.get(1);
- Assert.assertEquals("Ordinal must match", 1, t2.changeOrdinal());
- Assert.assertEquals("Snapshot must match", snap2.snapshotId(),
t2.commitSnapshotId());
- Assert.assertEquals("Data file must match", FILE_B.path(),
t2.file().path());
- Assert.assertTrue("Must be no deletes", t2.deletes().isEmpty());
+ assertThat(t2.changeOrdinal()).as("Ordinal must match").isEqualTo(1);
+ assertThat(t2.commitSnapshotId()).as("Snapshot must
match").isEqualTo(snap2.snapshotId());
+ assertThat(t2.file().path()).as("Data file must
match").isEqualTo(FILE_B.path());
+ assertThat(t2.deletes()).as("Must be no deletes").isEmpty();
}
- @Test
+ @TestTemplate
public void testDeleteFilesAreNotSupported() {
- Assume.assumeTrue(formatVersion == 2);
+ assumeThat(formatVersion).isEqualTo(2);
table.newFastAppend().appendFile(FILE_A2).appendFile(FILE_B).commit();
diff --git a/core/src/test/java/org/apache/iceberg/TestLocalDataTableScan.java
b/core/src/test/java/org/apache/iceberg/TestLocalDataTableScan.java
index 897cbed488..73f183387d 100644
--- a/core/src/test/java/org/apache/iceberg/TestLocalDataTableScan.java
+++ b/core/src/test/java/org/apache/iceberg/TestLocalDataTableScan.java
@@ -21,10 +21,6 @@ package org.apache.iceberg;
public class TestLocalDataTableScan
extends DataTableScanTestBase<TableScan, FileScanTask, CombinedScanTask> {
- public TestLocalDataTableScan(int formatVersion) {
- super(formatVersion);
- }
-
@Override
protected TableScan useRef(TableScan scan, String ref) {
return scan.useRef(ref);
diff --git a/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java
b/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java
index b7ff71461c..275b89df36 100644
--- a/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java
+++ b/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java
@@ -18,20 +18,17 @@
*/
package org.apache.iceberg;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import java.util.Arrays;
+import java.util.List;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public class TestLocalFilterFiles
extends FilterFilesTestBase<TableScan, FileScanTask, CombinedScanTask> {
- @Parameterized.Parameters(name = "formatVersion = {0}")
- public static Object[] parameters() {
- return new Object[] {1, 2};
- }
-
- public TestLocalFilterFiles(int formatVersion) {
- super(formatVersion);
+ @Parameters(name = "formatVersion = {0}")
+ public static List<Object> parameters() {
+ return Arrays.asList(1, 2);
}
@Override
diff --git
a/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java
b/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java
index a725a40aba..5718eb3703 100644
---
a/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java
+++
b/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java
@@ -62,7 +62,7 @@ public class TestGenericSortedPosDeleteWriter extends
TestBase {
private Record gRecord;
@Parameters(name = "formatVersion = {0}, fileFormat = {1}")
- public static List<Object[]> parameters() {
+ public static List<Object> parameters() {
return Arrays.asList(
new Object[] {FORMAT_V2, FileFormat.AVRO},
new Object[] {FORMAT_V2, FileFormat.ORC},
diff --git
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java
index 47b8dbb1d9..404ba72846 100644
---
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java
+++
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java
@@ -21,46 +21,41 @@ package org.apache.iceberg;
import static org.apache.iceberg.PlanningMode.DISTRIBUTED;
import static org.apache.iceberg.PlanningMode.LOCAL;
+import java.util.Arrays;
+import java.util.List;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.internal.SQLConf;
-import org.junit.Before;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public abstract class SparkDistributedDataScanTestBase
extends DataTableScanTestBase<BatchScan, ScanTask,
ScanTaskGroup<ScanTask>> {
@Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}")
- public static Object[] parameters() {
- return new Object[][] {
- new Object[] {1, LOCAL, LOCAL},
- new Object[] {1, LOCAL, DISTRIBUTED},
- new Object[] {1, DISTRIBUTED, LOCAL},
- new Object[] {1, DISTRIBUTED, DISTRIBUTED},
- new Object[] {2, LOCAL, LOCAL},
- new Object[] {2, LOCAL, DISTRIBUTED},
- new Object[] {2, DISTRIBUTED, LOCAL},
- new Object[] {2, DISTRIBUTED, DISTRIBUTED}
- };
+ public static List<Object> parameters() {
+ return Arrays.asList(
+ new Object[] {1, LOCAL, LOCAL},
+ new Object[] {1, LOCAL, DISTRIBUTED},
+ new Object[] {1, DISTRIBUTED, LOCAL},
+ new Object[] {1, DISTRIBUTED, DISTRIBUTED},
+ new Object[] {2, LOCAL, LOCAL},
+ new Object[] {2, LOCAL, DISTRIBUTED},
+ new Object[] {2, DISTRIBUTED, LOCAL},
+ new Object[] {2, DISTRIBUTED, DISTRIBUTED});
}
protected static SparkSession spark = null;
- private final PlanningMode dataMode;
- private final PlanningMode deleteMode;
+ @Parameter(index = 1)
+ private PlanningMode dataMode;
- public SparkDistributedDataScanTestBase(
- int formatVersion, PlanningMode dataPlanningMode, PlanningMode
deletePlanningMode) {
- super(formatVersion);
- this.dataMode = dataPlanningMode;
- this.deleteMode = deletePlanningMode;
- }
+ @Parameter(index = 2)
+ private PlanningMode deleteMode;
- @Before
+ @BeforeEach
public void configurePlanningModes() {
table
.updateProperties()
diff --git
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java
index 8ed37db642..9361c63176 100644
---
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java
+++
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java
@@ -21,42 +21,39 @@ package org.apache.iceberg;
import static org.apache.iceberg.PlanningMode.DISTRIBUTED;
import static org.apache.iceberg.PlanningMode.LOCAL;
+import java.util.Arrays;
+import java.util.List;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.internal.SQLConf;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public class TestSparkDistributedDataScanDeletes
extends DeleteFileIndexTestBase<BatchScan, ScanTask,
ScanTaskGroup<ScanTask>> {
- @Parameterized.Parameters(name = "dataMode = {0}, deleteMode = {1}")
- public static Object[] parameters() {
- return new Object[][] {
- new Object[] {LOCAL, LOCAL},
- new Object[] {LOCAL, DISTRIBUTED},
- new Object[] {DISTRIBUTED, LOCAL},
- new Object[] {DISTRIBUTED, DISTRIBUTED}
- };
+ @Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}")
+ public static List<Object> parameters() {
+ return Arrays.asList(
+ new Object[] {2, LOCAL, LOCAL},
+ new Object[] {2, LOCAL, DISTRIBUTED},
+ new Object[] {2, DISTRIBUTED, LOCAL},
+ new Object[] {2, LOCAL, DISTRIBUTED});
}
private static SparkSession spark = null;
- private final PlanningMode dataMode;
- private final PlanningMode deleteMode;
+ @Parameter(index = 1)
+ private PlanningMode dataMode;
- public TestSparkDistributedDataScanDeletes(
- PlanningMode dataPlanningMode, PlanningMode deletePlanningMode) {
- this.dataMode = dataPlanningMode;
- this.deleteMode = deletePlanningMode;
- }
+ @Parameter(index = 2)
+ private PlanningMode deleteMode;
- @Before
+ @BeforeEach
public void configurePlanningModes() {
table
.updateProperties()
@@ -65,7 +62,7 @@ public class TestSparkDistributedDataScanDeletes
.commit();
}
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
TestSparkDistributedDataScanDeletes.spark =
SparkSession.builder()
@@ -75,7 +72,7 @@ public class TestSparkDistributedDataScanDeletes
.getOrCreate();
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = TestSparkDistributedDataScanDeletes.spark;
TestSparkDistributedDataScanDeletes.spark = null;
diff --git
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java
index 510c130a58..a218f965ea 100644
---
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java
+++
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java
@@ -25,13 +25,11 @@ import
org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.internal.SQLConf;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public class TestSparkDistributedDataScanFilterFiles
extends FilterFilesTestBase<BatchScan, ScanTask, ScanTaskGroup<ScanTask>> {
@@ -51,17 +49,13 @@ public class TestSparkDistributedDataScanFilterFiles
private static SparkSession spark = null;
- private final PlanningMode dataMode;
- private final PlanningMode deleteMode;
+ @Parameter(index = 1)
+ private PlanningMode dataMode;
- public TestSparkDistributedDataScanFilterFiles(
- int formatVersion, PlanningMode dataPlanningMode, PlanningMode
deletePlanningMode) {
- super(formatVersion);
- this.dataMode = dataPlanningMode;
- this.deleteMode = deletePlanningMode;
- }
+ @Parameter(index = 2)
+ private PlanningMode deleteMode;
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
TestSparkDistributedDataScanFilterFiles.spark =
SparkSession.builder()
@@ -71,7 +65,7 @@ public class TestSparkDistributedDataScanFilterFiles
.getOrCreate();
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = TestSparkDistributedDataScanFilterFiles.spark;
TestSparkDistributedDataScanFilterFiles.spark = null;
diff --git
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java
index ba1096ee36..b8bd6fb867 100644
---
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java
+++
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java
@@ -19,24 +19,19 @@
package org.apache.iceberg;
import org.apache.spark.sql.SparkSession;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
public class TestSparkDistributedDataScanJavaSerialization
extends SparkDistributedDataScanTestBase {
- public TestSparkDistributedDataScanJavaSerialization(
- int formatVersion, PlanningMode dataPlanningMode, PlanningMode
deletePlanningMode) {
- super(formatVersion, dataPlanningMode, deletePlanningMode);
- }
-
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
SparkDistributedDataScanTestBase.spark =
initSpark("org.apache.spark.serializer.JavaSerializer");
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = SparkDistributedDataScanTestBase.spark;
SparkDistributedDataScanTestBase.spark = null;
diff --git
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java
index 7a795eb477..08d66cccb6 100644
---
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java
+++
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java
@@ -19,24 +19,19 @@
package org.apache.iceberg;
import org.apache.spark.sql.SparkSession;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
public class TestSparkDistributedDataScanKryoSerialization
extends SparkDistributedDataScanTestBase {
- public TestSparkDistributedDataScanKryoSerialization(
- int formatVersion, PlanningMode dataPlanningMode, PlanningMode
deletePlanningMode) {
- super(formatVersion, dataPlanningMode, deletePlanningMode);
- }
-
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
SparkDistributedDataScanTestBase.spark =
initSpark("org.apache.spark.serializer.KryoSerializer");
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = SparkDistributedDataScanTestBase.spark;
SparkDistributedDataScanTestBase.spark = null;
diff --git
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java
index 1ea4f990b2..acd4688440 100644
---
a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java
+++
b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java
@@ -21,41 +21,38 @@ package org.apache.iceberg;
import static org.apache.iceberg.PlanningMode.DISTRIBUTED;
import static org.apache.iceberg.PlanningMode.LOCAL;
+import java.util.Arrays;
+import java.util.List;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.internal.SQLConf;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public class TestSparkDistributedDataScanReporting
extends ScanPlanningAndReportingTestBase<BatchScan, ScanTask,
ScanTaskGroup<ScanTask>> {
- @Parameterized.Parameters(name = "dataMode = {0}, deleteMode = {1}")
- public static Object[] parameters() {
- return new Object[][] {
- new Object[] {LOCAL, LOCAL},
- new Object[] {LOCAL, DISTRIBUTED},
- new Object[] {DISTRIBUTED, LOCAL},
- new Object[] {DISTRIBUTED, DISTRIBUTED}
- };
+ @Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}")
+ public static List<Object> parameters() {
+ return Arrays.asList(
+ new Object[] {2, LOCAL, LOCAL},
+ new Object[] {2, LOCAL, DISTRIBUTED},
+ new Object[] {2, DISTRIBUTED, LOCAL},
+ new Object[] {2, DISTRIBUTED, DISTRIBUTED});
}
private static SparkSession spark = null;
- private final PlanningMode dataMode;
- private final PlanningMode deleteMode;
+ @Parameter(index = 1)
+ private PlanningMode dataMode;
- public TestSparkDistributedDataScanReporting(
- PlanningMode dataPlanningMode, PlanningMode deletePlanningMode) {
- this.dataMode = dataPlanningMode;
- this.deleteMode = deletePlanningMode;
- }
+ @Parameter(index = 2)
+ private PlanningMode deleteMode;
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
TestSparkDistributedDataScanReporting.spark =
SparkSession.builder()
@@ -65,7 +62,7 @@ public class TestSparkDistributedDataScanReporting
.getOrCreate();
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = TestSparkDistributedDataScanReporting.spark;
TestSparkDistributedDataScanReporting.spark = null;
diff --git
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java
index 47b8dbb1d9..404ba72846 100644
---
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java
+++
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java
@@ -21,46 +21,41 @@ package org.apache.iceberg;
import static org.apache.iceberg.PlanningMode.DISTRIBUTED;
import static org.apache.iceberg.PlanningMode.LOCAL;
+import java.util.Arrays;
+import java.util.List;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.internal.SQLConf;
-import org.junit.Before;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public abstract class SparkDistributedDataScanTestBase
extends DataTableScanTestBase<BatchScan, ScanTask,
ScanTaskGroup<ScanTask>> {
@Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}")
- public static Object[] parameters() {
- return new Object[][] {
- new Object[] {1, LOCAL, LOCAL},
- new Object[] {1, LOCAL, DISTRIBUTED},
- new Object[] {1, DISTRIBUTED, LOCAL},
- new Object[] {1, DISTRIBUTED, DISTRIBUTED},
- new Object[] {2, LOCAL, LOCAL},
- new Object[] {2, LOCAL, DISTRIBUTED},
- new Object[] {2, DISTRIBUTED, LOCAL},
- new Object[] {2, DISTRIBUTED, DISTRIBUTED}
- };
+ public static List<Object> parameters() {
+ return Arrays.asList(
+ new Object[] {1, LOCAL, LOCAL},
+ new Object[] {1, LOCAL, DISTRIBUTED},
+ new Object[] {1, DISTRIBUTED, LOCAL},
+ new Object[] {1, DISTRIBUTED, DISTRIBUTED},
+ new Object[] {2, LOCAL, LOCAL},
+ new Object[] {2, LOCAL, DISTRIBUTED},
+ new Object[] {2, DISTRIBUTED, LOCAL},
+ new Object[] {2, DISTRIBUTED, DISTRIBUTED});
}
protected static SparkSession spark = null;
- private final PlanningMode dataMode;
- private final PlanningMode deleteMode;
+ @Parameter(index = 1)
+ private PlanningMode dataMode;
- public SparkDistributedDataScanTestBase(
- int formatVersion, PlanningMode dataPlanningMode, PlanningMode
deletePlanningMode) {
- super(formatVersion);
- this.dataMode = dataPlanningMode;
- this.deleteMode = deletePlanningMode;
- }
+ @Parameter(index = 2)
+ private PlanningMode deleteMode;
- @Before
+ @BeforeEach
public void configurePlanningModes() {
table
.updateProperties()
diff --git
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java
index 8ed37db642..9361c63176 100644
---
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java
+++
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java
@@ -21,42 +21,39 @@ package org.apache.iceberg;
import static org.apache.iceberg.PlanningMode.DISTRIBUTED;
import static org.apache.iceberg.PlanningMode.LOCAL;
+import java.util.Arrays;
+import java.util.List;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.internal.SQLConf;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public class TestSparkDistributedDataScanDeletes
extends DeleteFileIndexTestBase<BatchScan, ScanTask,
ScanTaskGroup<ScanTask>> {
- @Parameterized.Parameters(name = "dataMode = {0}, deleteMode = {1}")
- public static Object[] parameters() {
- return new Object[][] {
- new Object[] {LOCAL, LOCAL},
- new Object[] {LOCAL, DISTRIBUTED},
- new Object[] {DISTRIBUTED, LOCAL},
- new Object[] {DISTRIBUTED, DISTRIBUTED}
- };
+ @Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}")
+ public static List<Object> parameters() {
+ return Arrays.asList(
+ new Object[] {2, LOCAL, LOCAL},
+ new Object[] {2, LOCAL, DISTRIBUTED},
+ new Object[] {2, DISTRIBUTED, LOCAL},
+ new Object[] {2, LOCAL, DISTRIBUTED});
}
private static SparkSession spark = null;
- private final PlanningMode dataMode;
- private final PlanningMode deleteMode;
+ @Parameter(index = 1)
+ private PlanningMode dataMode;
- public TestSparkDistributedDataScanDeletes(
- PlanningMode dataPlanningMode, PlanningMode deletePlanningMode) {
- this.dataMode = dataPlanningMode;
- this.deleteMode = deletePlanningMode;
- }
+ @Parameter(index = 2)
+ private PlanningMode deleteMode;
- @Before
+ @BeforeEach
public void configurePlanningModes() {
table
.updateProperties()
@@ -65,7 +62,7 @@ public class TestSparkDistributedDataScanDeletes
.commit();
}
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
TestSparkDistributedDataScanDeletes.spark =
SparkSession.builder()
@@ -75,7 +72,7 @@ public class TestSparkDistributedDataScanDeletes
.getOrCreate();
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = TestSparkDistributedDataScanDeletes.spark;
TestSparkDistributedDataScanDeletes.spark = null;
diff --git
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java
index 510c130a58..a218f965ea 100644
---
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java
+++
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java
@@ -25,13 +25,11 @@ import
org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.internal.SQLConf;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public class TestSparkDistributedDataScanFilterFiles
extends FilterFilesTestBase<BatchScan, ScanTask, ScanTaskGroup<ScanTask>> {
@@ -51,17 +49,13 @@ public class TestSparkDistributedDataScanFilterFiles
private static SparkSession spark = null;
- private final PlanningMode dataMode;
- private final PlanningMode deleteMode;
+ @Parameter(index = 1)
+ private PlanningMode dataMode;
- public TestSparkDistributedDataScanFilterFiles(
- int formatVersion, PlanningMode dataPlanningMode, PlanningMode
deletePlanningMode) {
- super(formatVersion);
- this.dataMode = dataPlanningMode;
- this.deleteMode = deletePlanningMode;
- }
+ @Parameter(index = 2)
+ private PlanningMode deleteMode;
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
TestSparkDistributedDataScanFilterFiles.spark =
SparkSession.builder()
@@ -71,7 +65,7 @@ public class TestSparkDistributedDataScanFilterFiles
.getOrCreate();
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = TestSparkDistributedDataScanFilterFiles.spark;
TestSparkDistributedDataScanFilterFiles.spark = null;
diff --git
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java
index ba1096ee36..b8bd6fb867 100644
---
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java
+++
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java
@@ -19,24 +19,19 @@
package org.apache.iceberg;
import org.apache.spark.sql.SparkSession;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
public class TestSparkDistributedDataScanJavaSerialization
extends SparkDistributedDataScanTestBase {
- public TestSparkDistributedDataScanJavaSerialization(
- int formatVersion, PlanningMode dataPlanningMode, PlanningMode
deletePlanningMode) {
- super(formatVersion, dataPlanningMode, deletePlanningMode);
- }
-
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
SparkDistributedDataScanTestBase.spark =
initSpark("org.apache.spark.serializer.JavaSerializer");
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = SparkDistributedDataScanTestBase.spark;
SparkDistributedDataScanTestBase.spark = null;
diff --git
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java
index 7a795eb477..08d66cccb6 100644
---
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java
+++
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java
@@ -19,24 +19,19 @@
package org.apache.iceberg;
import org.apache.spark.sql.SparkSession;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
public class TestSparkDistributedDataScanKryoSerialization
extends SparkDistributedDataScanTestBase {
- public TestSparkDistributedDataScanKryoSerialization(
- int formatVersion, PlanningMode dataPlanningMode, PlanningMode
deletePlanningMode) {
- super(formatVersion, dataPlanningMode, deletePlanningMode);
- }
-
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
SparkDistributedDataScanTestBase.spark =
initSpark("org.apache.spark.serializer.KryoSerializer");
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = SparkDistributedDataScanTestBase.spark;
SparkDistributedDataScanTestBase.spark = null;
diff --git
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java
index 1ea4f990b2..acd4688440 100644
---
a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java
+++
b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java
@@ -21,41 +21,38 @@ package org.apache.iceberg;
import static org.apache.iceberg.PlanningMode.DISTRIBUTED;
import static org.apache.iceberg.PlanningMode.LOCAL;
+import java.util.Arrays;
+import java.util.List;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.internal.SQLConf;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.extension.ExtendWith;
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
public class TestSparkDistributedDataScanReporting
extends ScanPlanningAndReportingTestBase<BatchScan, ScanTask,
ScanTaskGroup<ScanTask>> {
- @Parameterized.Parameters(name = "dataMode = {0}, deleteMode = {1}")
- public static Object[] parameters() {
- return new Object[][] {
- new Object[] {LOCAL, LOCAL},
- new Object[] {LOCAL, DISTRIBUTED},
- new Object[] {DISTRIBUTED, LOCAL},
- new Object[] {DISTRIBUTED, DISTRIBUTED}
- };
+ @Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}")
+ public static List<Object> parameters() {
+ return Arrays.asList(
+ new Object[] {2, LOCAL, LOCAL},
+ new Object[] {2, LOCAL, DISTRIBUTED},
+ new Object[] {2, DISTRIBUTED, LOCAL},
+ new Object[] {2, DISTRIBUTED, DISTRIBUTED});
}
private static SparkSession spark = null;
- private final PlanningMode dataMode;
- private final PlanningMode deleteMode;
+ @Parameter(index = 1)
+ private PlanningMode dataMode;
- public TestSparkDistributedDataScanReporting(
- PlanningMode dataPlanningMode, PlanningMode deletePlanningMode) {
- this.dataMode = dataPlanningMode;
- this.deleteMode = deletePlanningMode;
- }
+ @Parameter(index = 2)
+ private PlanningMode deleteMode;
- @BeforeClass
+ @BeforeAll
public static void startSpark() {
TestSparkDistributedDataScanReporting.spark =
SparkSession.builder()
@@ -65,7 +62,7 @@ public class TestSparkDistributedDataScanReporting
.getOrCreate();
}
- @AfterClass
+ @AfterAll
public static void stopSpark() {
SparkSession currentSpark = TestSparkDistributedDataScanReporting.spark;
TestSparkDistributedDataScanReporting.spark = null;