Skip to content

Commit

Permalink
Simplify all types test
Browse files Browse the repository at this point in the history
Use query-based test instead of much more verbose
`MaterializedResult`-based assertion.
  • Loading branch information
findepi committed Oct 22, 2021
1 parent 9619b0f commit c4368dd
Showing 1 changed file with 23 additions and 64 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,6 @@
import io.trino.spi.predicate.TupleDomain;
import io.trino.spi.statistics.ColumnStatistics;
import io.trino.spi.statistics.TableStatistics;
import io.trino.spi.type.ArrayType;
import io.trino.spi.type.MapType;
import io.trino.spi.type.RowType;
import io.trino.spi.type.TypeOperators;
import io.trino.sql.analyzer.FeaturesConfig;
import io.trino.testing.BaseConnectorTest;
import io.trino.testing.MaterializedResult;
Expand All @@ -66,9 +62,6 @@

import java.io.File;
import java.io.OutputStream;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
Expand All @@ -91,16 +84,12 @@
import static io.trino.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE;
import static io.trino.plugin.hive.HdfsEnvironment.HdfsContext;
import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT;
import static io.trino.plugin.hive.HiveTestUtils.SESSION;
import static io.trino.plugin.iceberg.IcebergQueryRunner.createIcebergQueryRunner;
import static io.trino.plugin.iceberg.IcebergSplitManager.ICEBERG_DOMAIN_COMPACTION_THRESHOLD;
import static io.trino.spi.predicate.Domain.multipleValues;
import static io.trino.spi.predicate.Domain.singleValue;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.DoubleType.DOUBLE;
import static io.trino.spi.type.IntegerType.INTEGER;
import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MICROS;
import static io.trino.spi.type.VarbinaryType.VARBINARY;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static io.trino.testing.MaterializedResult.resultBuilder;
import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder;
Expand Down Expand Up @@ -2132,59 +2121,29 @@ public void testAllAvailableTypes()
", a_map MAP(INTEGER, VARCHAR)" +
")");

assertUpdate(
"INSERT INTO test_all_types VALUES(" +
"true, " +
"1, " +
"BIGINT '1', " +
"REAL '1.0', " +
"DOUBLE '1.0', " +
"CAST(1.0 as DECIMAL(5,2)), " +
"CAST(11.0 as DECIMAL(38,20)), " +
"DATE '2021-07-24'," +
"TIME '02:43:57.348000', " +
"TIMESTAMP '2021-07-24 03:43:57.348000'," +
"TIMESTAMP '2021-07-24 04:43:57.348000' AT TIME ZONE 'America/Los_Angeles', " +
"'onefsadfdsf', " +
"X'000102f0feff', " +
"(CAST(ROW(null, 'this is a random value') AS ROW(int, varchar))), " +
"array['uno', 'dos', 'tres'], " +
"map(array[1,2], array['ek', 'one']))", 1);

@Language("SQL") String expectedResult = "" +
"VALUES(" +
" true" +
", 1" +
", 1" +
", CAST('1.0' AS REAL)" +
", CAST('1.0' AS DOUBLE)" +
", CAST('1.0' AS DECIMAL(5,2))" +
", CAST('11.0' AS DECIMAL(38,20))" +
", CAST('2021-07-24' AS DATE)" +
", CAST('02:43:57.348000' AS TIME(6))" +
", CAST('2021-07-24 03:43:57.348000' AS TIMESTAMP(6))" +
", 'onefsadfdsf')";
assertQuery("SELECT a_bool, a_int, a_big, a_real, a_double, a_short_decimal, a_long_decimal, a_date, a_time, a_timestamp, a_string FROM test_all_types", expectedResult);
query("SELECT a_timestamptz, a_fixed, a_binary, a_row, a_array, a_map FROM test_all_types").assertThat()
.matches(
resultBuilder(
SESSION,
TIMESTAMP_TZ_MICROS,
VARBINARY,
VARBINARY,
RowType.from(
ImmutableList.of(
new RowType.Field(Optional.of("id"), INTEGER),
new RowType.Field(Optional.of("vc"), VARCHAR))),
new ArrayType(VARCHAR),
new MapType(INTEGER, VARCHAR, new TypeOperators()))
.row(
ZonedDateTime.of(2021, 7, 23, 15, 43, 57, 348000000, ZoneId.of("UTC")),
new byte[] {00, 01, 02, -16, -2, -1},
new MaterializedRow(Arrays.asList(null, "this is a random value")),
Arrays.asList("uno", "dos", "tres"),
ImmutableMap.of(1, "ek", 2, "one"))
.build());
String values = "VALUES (" +
"true, " +
"1, " +
"BIGINT '1', " +
"REAL '1.0', " +
"DOUBLE '1.0', " +
"CAST(1.0 as DECIMAL(5,2)), " +
"CAST(11.0 as DECIMAL(38,20)), " +
"DATE '2021-07-24'," +
"TIME '02:43:57.348000', " +
"TIMESTAMP '2021-07-24 03:43:57.348000'," +
"TIMESTAMP '2021-07-24 04:43:57.348000 UTC', " +
"VARCHAR 'onefsadfdsf', " +
"X'000102f0feff', " +
"(CAST(ROW(null, 'this is a random value') AS ROW(id int, vc varchar))), " +
"array[VARCHAR 'uno', 'dos', 'tres'], " +
"map(array[1,2], array['ek', VARCHAR 'one']))";
assertUpdate("INSERT INTO test_all_types " + values, 1);

assertThat(query("SELECT * FROM test_all_types"))
.matches(values);

assertUpdate("DROP TABLE test_all_types");
}

@Test
Expand Down

0 comments on commit c4368dd

Please sign in to comment.