Skip to content

Commit

Permalink
Test nested fields with all storage formats in Hive connector
Browse files Browse the repository at this point in the history
  • Loading branch information
findepi committed Sep 22, 2021
1 parent 783016c commit c35a87f
Showing 1 changed file with 103 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,15 @@

import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.inject.Inject;
import io.airlift.units.DataSize;
import io.trino.plugin.hive.HiveTimestampPrecision;
import io.trino.tempto.BeforeTestWithContext;
import io.trino.tempto.ProductTest;
import io.trino.tempto.assertions.QueryAssert.Row;
import io.trino.tempto.query.QueryExecutionException;
import io.trino.tempto.query.QueryExecutor.QueryParam;
import io.trino.tempto.query.QueryResult;
import io.trino.testng.services.Flaky;
Expand Down Expand Up @@ -221,11 +223,13 @@ public class TestHiveStorageFormats
"2021-01-01 00:00:00.000000",
"2021-01-01 00:00:00.000000000"));

private static Set<String> storageFormats()
@DataProvider
public static String[] storageFormats()
{
return Stream.of(storageFormatsWithConfiguration())
.map(StorageFormat::getName)
.collect(toImmutableSet());
.distinct()
.toArray(String[]::new);
}

@DataProvider
Expand Down Expand Up @@ -295,7 +299,7 @@ public void verifyDataProviderCompleteness()
.filter(format -> !"JSON".equals(format))
.collect(toImmutableSet());

Assertions.assertThat(storageFormats())
Assertions.assertThat(ImmutableSet.copyOf(storageFormats()))
.isEqualTo(allFormatsToTest);
}

Expand Down Expand Up @@ -515,6 +519,97 @@ public void testOrcTableCreatedInTrino()
onTrino().executeQuery("DROP TABLE orc_table_created_in_trino");
}

@Test(dataProvider = "storageFormats", groups = STORAGE_FORMATS_DETAILED)
public void testNestedFieldsWrittenByHive(String format)
{
testNestedFields(format, Engine.HIVE);
}

@Test(dataProvider = "storageFormats", groups = STORAGE_FORMATS_DETAILED)
public void testNestedFieldsWrittenByTrino(String format)
{
testNestedFields(format, Engine.TRINO);
}

private void testNestedFields(String format, Engine writer)
{
String tableName = "test_nested_fields_written_by_" + writer.name().toLowerCase(ENGLISH);
onTrino().executeQuery("DROP TABLE IF EXISTS " + tableName);
onTrino().executeQuery("CREATE TABLE " + tableName + " (" +
" r row(a int), " +
" rr row(r row(a int)), " +
" ra row(a array(int)), " +
" dummy varchar) WITH (format='" + format + "')");

switch (writer) {
case HIVE:
writer.queryExecutor().executeQuery("INSERT INTO " + tableName + " SELECT " +
"named_struct('a', 42), " +
"named_struct('r', named_struct('a', 43)), " +
"named_struct('a', array(11, 22, 33)), " +
"'dummy value' " +
"FROM dummy");
break;
case TRINO:
writer.queryExecutor().executeQuery("INSERT INTO " + tableName + " VALUES (" +
"row(42), " +
"row(row(43)), " +
"row(ARRAY[11, 22, 33]), " +
"'dummy value')");
break;
default:
throw new IllegalStateException("Unsupported writer: " + writer);
}

assertThat(onTrino().executeQuery("SELECT * FROM " + tableName))
.containsOnly(row(
rowBuilder().addField("a", 42).build(),
rowBuilder()
.addField("r", rowBuilder().addField("a", 43).build())
.build(),
rowBuilder()
.addField("a", List.of(11, 22, 33))
.build(),
"dummy value"));

// with dereference
assertThat(onTrino().executeQuery("SELECT r.a, rr.r.a, ra.a[2] FROM " + tableName))
.containsOnly(row(42, 43, 22));

// with dereference in predicate
assertThat(onTrino().executeQuery("SELECT dummy FROM " + tableName + " WHERE r.a = 42 AND rr.r.a = 43 AND ra.a[2] = 22"))
.containsOnly(row("dummy value"));

// verify with Hive if data written by Trino
if (writer != Engine.HIVE) {
QueryResult queryResult = null;
try {
queryResult = onHive().executeQuery("SELECT * FROM " + tableName);
verify(queryResult != null);
}
catch (QueryExecutionException e) {
if ("AVRO".equals(format)) {
// TODO (https://github.com/trinodb/trino/issues/9285) Some versions of Hive cannot read Avro nested structs written by Trino
Assertions.assertThat(e.getCause())
.hasToString("java.sql.SQLException: java.io.IOException: org.apache.avro.AvroTypeException: Found default.record_1, expecting union");
}
else {
throw e;
}
}
if (queryResult != null) {
assertThat(queryResult)
.containsOnly(row(
"{\"a\":42}",
"{\"r\":{\"a\":43}}",
"{\"a\":[11,22,33]}",
"dummy value"));
}
}

onTrino().executeQuery("DROP TABLE " + tableName);
}

@Test(groups = STORAGE_FORMATS_DETAILED)
public void testOrcStructsWithNonLowercaseFields()
throws SQLException
Expand Down Expand Up @@ -1018,4 +1113,9 @@ public String getReadType(HiveTimestampPrecision precision)
return format("timestamp(%s)", precision.getPrecision());
}
}

private static io.trino.jdbc.Row.Builder rowBuilder()
{
return io.trino.jdbc.Row.builder();
}
}

0 comments on commit c35a87f

Please sign in to comment.