Skip to content

Commit

Permalink
Spark 3.5: Migrate tests in SQL directory to JUnit5 (#9401)
Browse files Browse the repository at this point in the history
  • Loading branch information
chinmay-bhat authored Jan 10, 2024
1 parent d1a3c10 commit 53a1c86
Show file tree
Hide file tree
Showing 26 changed files with 1,502 additions and 1,462 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,53 +18,49 @@
*/
package org.apache.iceberg.spark.sql;

import static org.assertj.core.api.Assertions.assertThat;

import java.util.Arrays;
import java.util.List;
import java.util.Map;
import org.apache.iceberg.Table;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.spark.SparkCatalogTestBase;
import org.apache.iceberg.spark.CatalogTestBase;
import org.apache.iceberg.spark.source.SimpleRecord;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
import org.apache.spark.sql.functions;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;

public abstract class PartitionedWritesTestBase extends SparkCatalogTestBase {
public PartitionedWritesTestBase(
String catalogName, String implementation, Map<String, String> config) {
super(catalogName, implementation, config);
}
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.TestTemplate;

public abstract class PartitionedWritesTestBase extends CatalogTestBase {

@Before
@BeforeEach
public void createTables() {
sql(
"CREATE TABLE %s (id bigint, data string) USING iceberg PARTITIONED BY (truncate(id, 3))",
tableName);
sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName);
}

@After
@AfterEach
public void removeTables() {
sql("DROP TABLE IF EXISTS %s", tableName);
}

@Test
@TestTemplate
public void testInsertAppend() {
Assert.assertEquals(
"Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 5 rows after insert")
.isEqualTo(3L);

sql("INSERT INTO %s VALUES (4, 'd'), (5, 'e')", commitTarget());

Assert.assertEquals(
"Should have 5 rows after insert",
5L,
scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 5 rows after insert")
.isEqualTo(5L);

List<Object[]> expected =
ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e"));
Expand All @@ -75,18 +71,18 @@ public void testInsertAppend() {
sql("SELECT * FROM %s ORDER BY id", selectTarget()));
}

@Test
@TestTemplate
public void testInsertOverwrite() {
Assert.assertEquals(
"Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 5 rows after insert")
.isEqualTo(3L);

// 4 and 5 replace 3 in the partition (id - (id % 3)) = 3
sql("INSERT OVERWRITE %s VALUES (4, 'd'), (5, 'e')", commitTarget());

Assert.assertEquals(
"Should have 4 rows after overwrite",
4L,
scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 4 rows after overwrite")
.isEqualTo(4L);

List<Object[]> expected =
ImmutableList.of(row(1L, "a"), row(2L, "b"), row(4L, "d"), row(5L, "e"));
Expand All @@ -97,20 +93,20 @@ public void testInsertOverwrite() {
sql("SELECT * FROM %s ORDER BY id", selectTarget()));
}

@Test
@TestTemplate
public void testDataFrameV2Append() throws NoSuchTableException {
Assert.assertEquals(
"Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 3 rows")
.isEqualTo(3L);

List<SimpleRecord> data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e"));
Dataset<Row> ds = spark.createDataFrame(data, SimpleRecord.class);

ds.writeTo(commitTarget()).append();

Assert.assertEquals(
"Should have 5 rows after insert",
5L,
scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 5 rows after insert")
.isEqualTo(5L);

List<Object[]> expected =
ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e"));
Expand All @@ -121,20 +117,20 @@ public void testDataFrameV2Append() throws NoSuchTableException {
sql("SELECT * FROM %s ORDER BY id", selectTarget()));
}

@Test
@TestTemplate
public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException {
Assert.assertEquals(
"Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 3 rows")
.isEqualTo(3L);

List<SimpleRecord> data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e"));
Dataset<Row> ds = spark.createDataFrame(data, SimpleRecord.class);

ds.writeTo(commitTarget()).overwritePartitions();

Assert.assertEquals(
"Should have 4 rows after overwrite",
4L,
scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 4 rows after overwrite")
.isEqualTo(4L);

List<Object[]> expected =
ImmutableList.of(row(1L, "a"), row(2L, "b"), row(4L, "d"), row(5L, "e"));
Expand All @@ -145,20 +141,20 @@ public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException {
sql("SELECT * FROM %s ORDER BY id", selectTarget()));
}

@Test
@TestTemplate
public void testDataFrameV2Overwrite() throws NoSuchTableException {
Assert.assertEquals(
"Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 3 rows")
.isEqualTo(3L);

List<SimpleRecord> data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e"));
Dataset<Row> ds = spark.createDataFrame(data, SimpleRecord.class);

ds.writeTo(commitTarget()).overwrite(functions.col("id").$less(3));

Assert.assertEquals(
"Should have 3 rows after overwrite",
3L,
scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 3 rows after overwrite")
.isEqualTo(3L);

List<Object[]> expected = ImmutableList.of(row(3L, "c"), row(4L, "d"), row(5L, "e"));

Expand All @@ -168,10 +164,11 @@ public void testDataFrameV2Overwrite() throws NoSuchTableException {
sql("SELECT * FROM %s ORDER BY id", selectTarget()));
}

@Test
@TestTemplate
public void testViewsReturnRecentResults() {
Assert.assertEquals(
"Should have 3 rows", 3L, scalarSql("SELECT count(*) FROM %s", selectTarget()));
assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget()))
.as("Should have 3 rows")
.isEqualTo(3L);

Dataset<Row> query = spark.sql("SELECT * FROM " + commitTarget() + " WHERE id = 1");
query.createOrReplaceTempView("tmp");
Expand Down Expand Up @@ -207,7 +204,7 @@ protected void assertPartitionMetadata(
rowsToJava(actualPartitionRows.collectAsList()));
}

@Test
@TestTemplate
public void testWriteWithOutputSpec() throws NoSuchTableException {
Table table = validationCatalog.loadTable(tableIdent);

Expand Down
Loading

0 comments on commit 53a1c86

Please sign in to comment.