diff --git a/presto-accumulo/src/test/java/io/prestosql/plugin/accumulo/TestAccumuloDistributedQueries.java b/presto-accumulo/src/test/java/io/prestosql/plugin/accumulo/TestAccumuloDistributedQueries.java index c9f124fb730f..fc0941ee2cdc 100644 --- a/presto-accumulo/src/test/java/io/prestosql/plugin/accumulo/TestAccumuloDistributedQueries.java +++ b/presto-accumulo/src/test/java/io/prestosql/plugin/accumulo/TestAccumuloDistributedQueries.java @@ -23,6 +23,8 @@ import org.testng.SkipException; import org.testng.annotations.Test; +import java.util.Optional; + import static io.prestosql.plugin.accumulo.AccumuloQueryRunner.createAccumuloQueryRunner; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -358,4 +360,17 @@ public void testCommentTable() // Accumulo connector currently does not support comment on table assertQueryFails("COMMENT ON TABLE orders IS 'hello'", "This connector does not support setting table comments"); } + + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeName = dataMappingTestSetup.getPrestoTypeName(); + if (typeName.startsWith("decimal(") + || typeName.equals("timestamp with time zone") + || typeName.startsWith("char(")) { + return Optional.of(dataMappingTestSetup.asUnsupported()); + } + + return Optional.of(dataMappingTestSetup); + } } diff --git a/presto-cassandra/src/test/java/io/prestosql/plugin/cassandra/TestCassandraDistributedQueries.java b/presto-cassandra/src/test/java/io/prestosql/plugin/cassandra/TestCassandraDistributedQueries.java index b7bd79d3be4c..83ec47aa0990 100644 --- a/presto-cassandra/src/test/java/io/prestosql/plugin/cassandra/TestCassandraDistributedQueries.java +++ b/presto-cassandra/src/test/java/io/prestosql/plugin/cassandra/TestCassandraDistributedQueries.java @@ -171,4 +171,10 @@ protected TestTable createTableWithDefaultColumns() { throw new SkipException("Cassandra connector does not support column default values"); } + + @Override + public void testDataMappingSmokeTest(DataMappingTestSetup dataMappingTestSetup) + { + // Cassandra connector currently does not support create table + } } diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHiveDistributedQueries.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHiveDistributedQueries.java index b0df04717838..92ae861499c7 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHiveDistributedQueries.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHiveDistributedQueries.java @@ -20,6 +20,8 @@ import org.testng.SkipException; import org.testng.annotations.Test; +import java.util.Optional; + import static com.google.common.collect.Iterables.getOnlyElement; import static io.prestosql.sql.tree.ExplainType.Type.LOGICAL; import static io.prestosql.tpch.TpchTable.getTables; @@ -57,5 +59,17 @@ public void testExplainOfCreateTableAs() assertEquals(getOnlyElement(result.getOnlyColumnAsSet()), getExplainPlan(query, LOGICAL)); } + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeName = dataMappingTestSetup.getPrestoTypeName(); + if (typeName.equals("time") + || typeName.equals("timestamp with time zone")) { + return Optional.of(dataMappingTestSetup.asUnsupported()); + } + + return Optional.of(dataMappingTestSetup); + } + // Hive specific tests should normally go in TestHiveIntegrationSmokeTest } diff --git a/presto-iceberg/src/test/java/io/prestosql/plugin/iceberg/TestIcebergDistributed.java b/presto-iceberg/src/test/java/io/prestosql/plugin/iceberg/TestIcebergDistributed.java index 87b69b09ff3f..70704d54b636 100644 --- a/presto-iceberg/src/test/java/io/prestosql/plugin/iceberg/TestIcebergDistributed.java +++ b/presto-iceberg/src/test/java/io/prestosql/plugin/iceberg/TestIcebergDistributed.java @@ -19,6 +19,8 @@ import io.prestosql.testing.sql.TestTable; import org.testng.SkipException; +import java.util.Optional; + import static io.prestosql.plugin.iceberg.IcebergQueryRunner.createIcebergQueryRunner; public class TestIcebergDistributed @@ -79,4 +81,25 @@ public void testInsertWithCoercion() { // Iceberg does not support parameterized varchar } + + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeName = dataMappingTestSetup.getPrestoTypeName(); + if (typeName.equals("tinyint") + || typeName.equals("smallint") + || typeName.equals("timestamp") + || typeName.startsWith("char(")) { + return Optional.of(dataMappingTestSetup.asUnsupported()); + } + + if (typeName.startsWith("decimal(") + || typeName.equals("time") + || typeName.equals("timestamp with time zone")) { + // TODO this should either work or fail cleanly + return Optional.empty(); + } + + return Optional.of(dataMappingTestSetup); + } } diff --git a/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoPageSink.java b/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoPageSink.java index 5b5915c37ef0..1e1b1e91521b 100644 --- a/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoPageSink.java +++ b/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoPageSink.java @@ -28,12 +28,13 @@ import io.prestosql.spi.connector.SchemaTableName; import io.prestosql.spi.type.BigintType; import io.prestosql.spi.type.BooleanType; +import io.prestosql.spi.type.CharType; import io.prestosql.spi.type.DateType; import io.prestosql.spi.type.DecimalType; -import io.prestosql.spi.type.Decimals; import io.prestosql.spi.type.DoubleType; import io.prestosql.spi.type.IntegerType; import io.prestosql.spi.type.NamedTypeSignature; +import io.prestosql.spi.type.RealType; import io.prestosql.spi.type.SmallintType; import io.prestosql.spi.type.TimeType; import io.prestosql.spi.type.TimestampType; @@ -46,8 +47,6 @@ import org.bson.types.Binary; import org.bson.types.ObjectId; -import java.math.BigDecimal; -import java.math.BigInteger; import java.util.ArrayList; import java.util.Collection; import java.util.Date; @@ -63,8 +62,11 @@ import static io.prestosql.plugin.mongodb.TypeUtils.isMapType; import static io.prestosql.plugin.mongodb.TypeUtils.isRowType; import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.prestosql.spi.type.Chars.padSpaces; import static io.prestosql.spi.type.DateTimeEncoding.unpackMillisUtc; +import static io.prestosql.spi.type.Decimals.readBigDecimal; import static io.prestosql.spi.type.Varchars.isVarcharType; +import static java.lang.Float.intBitsToFloat; import static java.lang.Math.toIntExact; import static java.util.Collections.unmodifiableList; import static java.util.Collections.unmodifiableMap; @@ -141,12 +143,18 @@ private Object getObjectValue(Type type, Block block, int position) if (type.equals(TinyintType.TINYINT)) { return SignedBytes.checkedCast(type.getLong(block, position)); } + if (type.equals(RealType.REAL)) { + return intBitsToFloat(toIntExact(type.getLong(block, position))); + } if (type.equals(DoubleType.DOUBLE)) { return type.getDouble(block, position); } if (isVarcharType(type)) { return type.getSlice(block, position).toStringUtf8(); } + if (type instanceof CharType) { + return padSpaces(type.getSlice(block, position), ((CharType) type)).toStringUtf8(); + } if (type.equals(VarbinaryType.VARBINARY)) { return new Binary(type.getSlice(block, position).getBytes()); } @@ -167,17 +175,7 @@ private Object getObjectValue(Type type, Block block, int position) return new Date(millisUtc); } if (type instanceof DecimalType) { - // TODO: decimal type might not support yet - // TODO: this code is likely wrong and should switch to Decimals.readBigDecimal() - DecimalType decimalType = (DecimalType) type; - BigInteger unscaledValue; - if (decimalType.isShort()) { - unscaledValue = BigInteger.valueOf(decimalType.getLong(block, position)); - } - else { - unscaledValue = Decimals.decodeUnscaledValue(decimalType.getSlice(block, position)); - } - return new BigDecimal(unscaledValue); + return readBigDecimal((DecimalType) type, block, position); } if (isArrayType(type)) { Type elementType = type.getTypeParameters().get(0); diff --git a/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoPageSource.java b/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoPageSource.java index 813b86fe2ac8..bb18faf0d7a7 100644 --- a/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoPageSource.java +++ b/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoPageSource.java @@ -13,6 +13,8 @@ */ package io.prestosql.plugin.mongodb; +import com.google.common.primitives.Shorts; +import com.google.common.primitives.SignedBytes; import com.mongodb.client.MongoCursor; import io.airlift.slice.Slice; import io.prestosql.spi.Page; @@ -21,12 +23,15 @@ import io.prestosql.spi.block.Block; import io.prestosql.spi.block.BlockBuilder; import io.prestosql.spi.connector.ConnectorPageSource; +import io.prestosql.spi.type.CharType; +import io.prestosql.spi.type.DecimalType; import io.prestosql.spi.type.Type; import io.prestosql.spi.type.TypeSignatureParameter; import io.prestosql.spi.type.VarbinaryType; import io.prestosql.spi.type.VarcharType; import org.bson.Document; import org.bson.types.Binary; +import org.bson.types.Decimal128; import org.bson.types.ObjectId; import org.joda.time.chrono.ISOChronology; @@ -47,10 +52,20 @@ import static io.prestosql.plugin.mongodb.TypeUtils.isRowType; import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.Chars.truncateToLengthAndTrimSpaces; +import static io.prestosql.spi.type.DateTimeEncoding.packDateTimeWithZone; import static io.prestosql.spi.type.DateType.DATE; +import static io.prestosql.spi.type.Decimals.encodeScaledValue; +import static io.prestosql.spi.type.Decimals.encodeShortScaledValue; import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.SmallintType.SMALLINT; import static io.prestosql.spi.type.TimeType.TIME; +import static io.prestosql.spi.type.TimeZoneKey.UTC_KEY; import static io.prestosql.spi.type.TimestampType.TIMESTAMP; +import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static java.lang.Float.floatToIntBits; import static java.lang.String.join; import static java.util.stream.Collectors.toList; @@ -150,6 +165,19 @@ else if (javaType == long.class) { else if (type.equals(INTEGER)) { type.writeLong(output, ((Number) value).intValue()); } + else if (type.equals(SMALLINT)) { + type.writeLong(output, Shorts.checkedCast(((Number) value).longValue())); + } + else if (type.equals(TINYINT)) { + type.writeLong(output, SignedBytes.checkedCast(((Number) value).longValue())); + } + else if (type.equals(REAL)) { + //noinspection NumericCastThatLosesPrecision + type.writeLong(output, floatToIntBits(((float) ((Number) value).doubleValue()))); + } + else if (type instanceof DecimalType) { + type.writeLong(output, encodeShortScaledValue(((Decimal128) value).bigDecimalValue(), ((DecimalType) type).getScale())); + } else if (type.equals(DATE)) { long utcMillis = ((Date) value).getTime(); type.writeLong(output, TimeUnit.MILLISECONDS.toDays(utcMillis)); @@ -158,8 +186,12 @@ else if (type.equals(TIME)) { type.writeLong(output, UTC_CHRONOLOGY.millisOfDay().get(((Date) value).getTime())); } else if (type.equals(TIMESTAMP)) { + // TODO provide correct TIMESTAMP mapping, and respecting session.isLegacyTimestamp() type.writeLong(output, ((Date) value).getTime()); } + else if (type.equals(TIMESTAMP_WITH_TIME_ZONE)) { + type.writeLong(output, packDateTimeWithZone(((Date) value).getTime(), UTC_KEY)); + } else { throw new PrestoException(GENERIC_INTERNAL_ERROR, "Unhandled type for " + javaType.getSimpleName() + ":" + type.getTypeSignature()); } @@ -178,6 +210,7 @@ else if (javaType == Block.class) { } } catch (ClassCastException ignore) { + // TODO remove (fail clearly), or hide behind a toggle // returns null instead of raising exception output.appendNull(); } @@ -199,6 +232,9 @@ private void writeSlice(BlockBuilder output, Type type, Object value) if (type instanceof VarcharType) { type.writeSlice(output, utf8Slice(toVarcharValue(value))); } + else if (type instanceof CharType) { + type.writeSlice(output, truncateToLengthAndTrimSpaces(utf8Slice((String) value), ((CharType) type))); + } else if (type.equals(OBJECT_ID)) { type.writeSlice(output, wrappedBuffer(((ObjectId) value).toByteArray())); } @@ -210,6 +246,9 @@ else if (type instanceof VarbinaryType) { output.appendNull(); } } + else if (type instanceof DecimalType) { + type.writeSlice(output, encodeScaledValue(((Decimal128) value).bigDecimalValue(), ((DecimalType) type).getScale())); + } else { throw new PrestoException(GENERIC_INTERNAL_ERROR, "Unhandled type for Slice: " + type.getTypeSignature()); } diff --git a/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoSession.java b/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoSession.java index 4a5846dda1dc..b71b25c03afc 100644 --- a/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoSession.java +++ b/presto-mongodb/src/main/java/io/prestosql/plugin/mongodb/MongoSession.java @@ -19,6 +19,9 @@ import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.common.primitives.Primitives; +import com.google.common.primitives.Shorts; +import com.google.common.primitives.SignedBytes; import com.google.common.util.concurrent.UncheckedExecutionException; import com.mongodb.MongoClient; import com.mongodb.client.FindIterable; @@ -37,6 +40,7 @@ import io.prestosql.spi.predicate.Domain; import io.prestosql.spi.predicate.Range; import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.type.IntegerType; import io.prestosql.spi.type.NamedTypeSignature; import io.prestosql.spi.type.RowFieldName; import io.prestosql.spi.type.StandardTypes; @@ -44,6 +48,7 @@ import io.prestosql.spi.type.TypeManager; import io.prestosql.spi.type.TypeSignature; import io.prestosql.spi.type.TypeSignatureParameter; +import io.prestosql.spi.type.VarcharType; import org.bson.Document; import org.bson.types.ObjectId; @@ -58,15 +63,20 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Throwables.throwIfInstanceOf; import static com.google.common.base.Verify.verify; +import static com.google.common.collect.ImmutableList.toImmutableList; import static io.prestosql.plugin.mongodb.ObjectIdType.OBJECT_ID; import static io.prestosql.spi.type.BigintType.BIGINT; import static io.prestosql.spi.type.BooleanType.BOOLEAN; import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.SmallintType.SMALLINT; import static io.prestosql.spi.type.TimestampType.TIMESTAMP; +import static io.prestosql.spi.type.TinyintType.TINYINT; import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType; +import static java.lang.Math.toIntExact; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.HOURS; @@ -250,39 +260,48 @@ static Document buildQuery(TupleDomain tupleDomain) if (tupleDomain.getDomains().isPresent()) { for (Map.Entry entry : tupleDomain.getDomains().get().entrySet()) { MongoColumnHandle column = (MongoColumnHandle) entry.getKey(); - query.putAll(buildPredicate(column, entry.getValue())); + Optional predicate = buildPredicate(column, entry.getValue()); + predicate.ifPresent(query::putAll); } } return query; } - private static Document buildPredicate(MongoColumnHandle column, Domain domain) + private static Optional buildPredicate(MongoColumnHandle column, Domain domain) { String name = column.getName(); Type type = column.getType(); if (domain.getValues().isNone() && domain.isNullAllowed()) { - return documentOf(name, isNullPredicate()); + return Optional.of(documentOf(name, isNullPredicate())); } if (domain.getValues().isAll() && !domain.isNullAllowed()) { - return documentOf(name, isNotNullPredicate()); + return Optional.of(documentOf(name, isNotNullPredicate())); } List singleValues = new ArrayList<>(); List disjuncts = new ArrayList<>(); for (Range range : domain.getValues().getRanges().getOrderedRanges()) { if (range.isSingleValue()) { - singleValues.add(translateValue(range.getSingleValue(), type)); + Optional translated = translateValue(range.getSingleValue(), type); + if (!translated.isPresent()) { + return Optional.empty(); + } + singleValues.add(translated.get()); } else { Document rangeConjuncts = new Document(); if (!range.getLow().isLowerUnbounded()) { + Optional translated = translateValue(range.getLow().getValue(), type); + if (!translated.isPresent()) { + return Optional.empty(); + } switch (range.getLow().getBound()) { case ABOVE: - rangeConjuncts.put(GT_OP, translateValue(range.getLow().getValue(), type)); + rangeConjuncts.put(GT_OP, translated.get()); break; case EXACTLY: - rangeConjuncts.put(GTE_OP, translateValue(range.getLow().getValue(), type)); + rangeConjuncts.put(GTE_OP, translated.get()); break; case BELOW: throw new IllegalArgumentException("Low Marker should never use BELOW bound: " + range); @@ -291,14 +310,18 @@ private static Document buildPredicate(MongoColumnHandle column, Domain domain) } } if (!range.getHigh().isUpperUnbounded()) { + Optional translated = translateValue(range.getHigh().getValue(), type); + if (!translated.isPresent()) { + return Optional.empty(); + } switch (range.getHigh().getBound()) { case ABOVE: throw new IllegalArgumentException("High Marker should never use ABOVE bound: " + range); case EXACTLY: - rangeConjuncts.put(LTE_OP, translateValue(range.getHigh().getValue(), type)); + rangeConjuncts.put(LTE_OP, translated.get()); break; case BELOW: - rangeConjuncts.put(LT_OP, translateValue(range.getHigh().getValue(), type)); + rangeConjuncts.put(LT_OP, translated.get()); break; default: throw new AssertionError("Unhandled bound: " + range.getHigh().getBound()); @@ -322,23 +345,42 @@ else if (singleValues.size() > 1) { disjuncts.add(isNullPredicate()); } - return orPredicate(disjuncts.stream() + return Optional.of(orPredicate(disjuncts.stream() .map(disjunct -> new Document(name, disjunct)) - .collect(toList())); + .collect(toImmutableList()))); } - private static Object translateValue(Object source, Type type) + private static Optional translateValue(Object prestoNativeValue, Type type) { - if (source instanceof Slice) { - if (type instanceof ObjectIdType) { - return new ObjectId(((Slice) source).getBytes()); - } - else { - return ((Slice) source).toStringUtf8(); - } + requireNonNull(prestoNativeValue, "prestoNativeValue is null"); + requireNonNull(type, "type is null"); + checkArgument(Primitives.wrap(type.getJavaType()).isInstance(prestoNativeValue), "%s (%s) is not a valid representation for %s", prestoNativeValue, prestoNativeValue.getClass(), type); + + if (type == TINYINT) { + return Optional.of((long) SignedBytes.checkedCast(((Long) prestoNativeValue))); + } + + if (type == SMALLINT) { + return Optional.of((long) Shorts.checkedCast(((Long) prestoNativeValue))); + } + + if (type == IntegerType.INTEGER) { + return Optional.of((long) toIntExact(((Long) prestoNativeValue))); + } + + if (type == BIGINT) { + return Optional.of(prestoNativeValue); + } + + if (type instanceof ObjectIdType) { + return Optional.of(new ObjectId(((Slice) prestoNativeValue).getBytes())); + } + + if (type instanceof VarcharType) { + return Optional.of(((Slice) prestoNativeValue).toStringUtf8()); } - return source; + return Optional.empty(); } private static Document documentOf(String key, Object value) diff --git a/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/MongoQueryRunner.java b/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/MongoQueryRunner.java index 6aebbb8af649..a50d4bc12312 100644 --- a/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/MongoQueryRunner.java +++ b/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/MongoQueryRunner.java @@ -39,15 +39,17 @@ private MongoQueryRunner() {} public static DistributedQueryRunner createMongoQueryRunner(MongoServer server, TpchTable... tables) throws Exception { - return createMongoQueryRunner(server, ImmutableList.copyOf(tables)); + return createMongoQueryRunner(server, ImmutableMap.of(), ImmutableList.copyOf(tables)); } - public static DistributedQueryRunner createMongoQueryRunner(MongoServer server, Iterable> tables) + public static DistributedQueryRunner createMongoQueryRunner(MongoServer server, Map extraProperties, Iterable> tables) throws Exception { DistributedQueryRunner queryRunner = null; try { - queryRunner = DistributedQueryRunner.builder(createSession()).build(); + queryRunner = DistributedQueryRunner.builder(createSession()) + .setExtraProperties(extraProperties) + .build(); queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); @@ -85,7 +87,10 @@ public static void main(String[] args) throws Exception { Logging.initialize(); - DistributedQueryRunner queryRunner = createMongoQueryRunner(new MongoServer(), TpchTable.getTables()); + DistributedQueryRunner queryRunner = createMongoQueryRunner( + new MongoServer(), + ImmutableMap.of("http-server.http.port", "8080"), + TpchTable.getTables()); Thread.sleep(10); Logger log = Logger.get(MongoQueryRunner.class); log.info("======== SERVER STARTED ========"); diff --git a/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/TestMongoDistributedQueries.java b/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/TestMongoDistributedQueries.java index 67933167f8d0..498c0df7dca7 100644 --- a/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/TestMongoDistributedQueries.java +++ b/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/TestMongoDistributedQueries.java @@ -13,17 +13,22 @@ */ package io.prestosql.plugin.mongodb; -import io.prestosql.testing.AbstractTestQueries; +import io.prestosql.testing.AbstractTestDistributedQueries; import io.prestosql.testing.QueryRunner; +import io.prestosql.testing.sql.TestTable; import io.prestosql.tpch.TpchTable; +import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; +import org.testng.SkipException; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; +import java.util.Optional; + import static io.prestosql.plugin.mongodb.MongoQueryRunner.createMongoQueryRunner; @Test public class TestMongoDistributedQueries - extends AbstractTestQueries + extends AbstractTestDistributedQueries { private MongoServer server; @@ -32,7 +37,7 @@ protected QueryRunner createQueryRunner() throws Exception { this.server = new MongoServer(); - return createMongoQueryRunner(server, TpchTable.getTables()); + return createMongoQueryRunner(server, ImmutableMap.of(), TpchTable.getTables()); } @AfterClass(alwaysRun = true) @@ -40,4 +45,85 @@ public final void destroy() { server.close(); } + + @Override + protected boolean supportsViews() + { + return false; + } + + @Override + public void testCreateTable() + { + // TODO https://github.com/prestosql/presto/issues/3082 + throw new SkipException("Fix DROP TABLE"); + } + + @Override + public void testCreateTableAsSelect() + { + // TODO https://github.com/prestosql/presto/issues/3082 + throw new SkipException("Fix DROP TABLE"); + } + + @Override + public void testCreateSchema() + { + // the connector does not support creating schemas + } + + @Override + public void testRenameTable() + { + // the connector does not support renaming tables + } + + @Override + public void testAddColumn() + { + // the connector does not support adding columns + } + + @Override + public void testRenameColumn() + { + // the connector does not support renaming columns + } + + @Override + public void testDropColumn() + { + // the connector does not support dropping columns + } + + @Override + public void testDelete() + { + // the connector does not support delete + } + + @Override + public void testCommentTable() + { + // the connector does not support comment on table + assertQueryFails("COMMENT ON TABLE orders IS 'hello'", "This connector does not support setting table comments"); + } + + @Override + protected TestTable createTableWithDefaultColumns() + { + throw new SkipException("test disabled for Mongo"); + } + + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeName = dataMappingTestSetup.getPrestoTypeName(); + if (typeName.equals("time")) { + // TODO this should either work or fail cleanly + return Optional.empty(); + } + + return Optional.of(dataMappingTestSetup); + } } diff --git a/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/TestMongoSession.java b/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/TestMongoSession.java index afbc49468eb6..f412ef497346 100644 --- a/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/TestMongoSession.java +++ b/presto-mongodb/src/test/java/io/prestosql/plugin/mongodb/TestMongoSession.java @@ -37,6 +37,7 @@ public class TestMongoSession { private static final MongoColumnHandle COL1 = new MongoColumnHandle("col1", BIGINT, false); private static final MongoColumnHandle COL2 = new MongoColumnHandle("col2", createUnboundedVarcharType(), false); + private static final MongoColumnHandle COL3 = new MongoColumnHandle("col3", createUnboundedVarcharType(), false); @Test public void testBuildQuery() @@ -56,12 +57,12 @@ public void testBuildQuery() public void testBuildQueryStringType() { TupleDomain tupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of( - COL1, Domain.create(ValueSet.ofRanges(range(createUnboundedVarcharType(), utf8Slice("hello"), false, utf8Slice("world"), true)), false), + COL3, Domain.create(ValueSet.ofRanges(range(createUnboundedVarcharType(), utf8Slice("hello"), false, utf8Slice("world"), true)), false), COL2, Domain.create(ValueSet.ofRanges(greaterThanOrEqual(createUnboundedVarcharType(), utf8Slice("a value"))), false))); Document query = MongoSession.buildQuery(tupleDomain); Document expected = new Document() - .append(COL1.getName(), new Document().append("$gt", "hello").append("$lte", "world")) + .append(COL3.getName(), new Document().append("$gt", "hello").append("$lte", "world")) .append(COL2.getName(), new Document("$gte", "a value")); assertEquals(query, expected); } diff --git a/presto-mysql/src/test/java/io/prestosql/plugin/mysql/TestMySqlDistributedQueries.java b/presto-mysql/src/test/java/io/prestosql/plugin/mysql/TestMySqlDistributedQueries.java index cd840cc2227a..99e1e5c442cf 100644 --- a/presto-mysql/src/test/java/io/prestosql/plugin/mysql/TestMySqlDistributedQueries.java +++ b/presto-mysql/src/test/java/io/prestosql/plugin/mysql/TestMySqlDistributedQueries.java @@ -21,6 +21,8 @@ import io.prestosql.tpch.TpchTable; import org.testng.annotations.AfterClass; +import java.util.Optional; + import static io.prestosql.plugin.mysql.MySqlQueryRunner.createMySqlQueryRunner; import static io.prestosql.spi.type.VarcharType.VARCHAR; import static io.prestosql.testing.MaterializedResult.resultBuilder; @@ -129,5 +131,23 @@ public void testDelete() // delete is not supported } + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeName = dataMappingTestSetup.getPrestoTypeName(); + if (typeName.equals("time") + || typeName.equals("timestamp with time zone")) { + return Optional.of(dataMappingTestSetup.asUnsupported()); + } + + if (typeName.equals("real") + || typeName.equals("timestamp")) { + // TODO this should either work or fail cleanly + return Optional.empty(); + } + + return Optional.of(dataMappingTestSetup); + } + // MySQL specific tests should normally go in TestMySqlIntegrationSmokeTest } diff --git a/presto-phoenix/src/test/java/io/prestosql/plugin/phoenix/TestPhoenixDistributedQueries.java b/presto-phoenix/src/test/java/io/prestosql/plugin/phoenix/TestPhoenixDistributedQueries.java index 9da7579f8e2e..af9147499bf3 100644 --- a/presto-phoenix/src/test/java/io/prestosql/plugin/phoenix/TestPhoenixDistributedQueries.java +++ b/presto-phoenix/src/test/java/io/prestosql/plugin/phoenix/TestPhoenixDistributedQueries.java @@ -132,4 +132,11 @@ public void testCreateSchema() { throw new SkipException("test disabled until issue fixed"); // TODO https://github.com/prestosql/presto/issues/2348 } + + @Override + public void testDataMappingSmokeTest(DataMappingTestSetup dataMappingTestSetup) + { + // TODO enable the test + throw new SkipException("test fails on Phoenix"); + } } diff --git a/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorDistributedQueries.java b/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorDistributedQueries.java index 77e323c87a05..c15cd70a4b5c 100644 --- a/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorDistributedQueries.java +++ b/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorDistributedQueries.java @@ -19,6 +19,8 @@ import io.prestosql.testing.sql.TestTable; import org.testng.SkipException; +import java.util.Optional; + import static io.prestosql.plugin.raptor.legacy.RaptorQueryRunner.createRaptorQueryRunner; public class TestRaptorDistributedQueries @@ -55,4 +57,21 @@ public void testCreateSchema() { // schema creation is not supported } + + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeName = dataMappingTestSetup.getPrestoTypeName(); + if (typeName.equals("tinyint") + || typeName.equals("real") + || typeName.startsWith("decimal(") + || typeName.equals("time") + || typeName.equals("timestamp with time zone") + || typeName.startsWith("char(")) { + // TODO this should either work or fail cleanly + return Optional.empty(); + } + + return Optional.of(dataMappingTestSetup); + } } diff --git a/presto-sqlserver/src/test/java/io/prestosql/plugin/sqlserver/TestSqlServerDistributedQueries.java b/presto-sqlserver/src/test/java/io/prestosql/plugin/sqlserver/TestSqlServerDistributedQueries.java index 9af45c29f731..5c8737eb032f 100644 --- a/presto-sqlserver/src/test/java/io/prestosql/plugin/sqlserver/TestSqlServerDistributedQueries.java +++ b/presto-sqlserver/src/test/java/io/prestosql/plugin/sqlserver/TestSqlServerDistributedQueries.java @@ -21,6 +21,8 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.Test; +import java.util.Optional; + import static io.prestosql.plugin.sqlserver.SqlServerQueryRunner.createSqlServerQueryRunner; @Test @@ -90,5 +92,23 @@ public void testDelete() // delete is not supported } + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeName = dataMappingTestSetup.getPrestoTypeName(); + if (typeName.equals("time") + || typeName.equals("timestamp") + || typeName.equals("timestamp with time zone")) { + return Optional.of(dataMappingTestSetup.asUnsupported()); + } + + if (typeName.equals("varbinary")) { + // TODO this should either work or fail cleanly + return Optional.empty(); + } + + return Optional.of(dataMappingTestSetup); + } + // SQLServer specific tests should normally go in TestSqlServerIntegrationSmokeTest } diff --git a/presto-testing/src/main/java/io/prestosql/testing/AbstractTestDistributedQueries.java b/presto-testing/src/main/java/io/prestosql/testing/AbstractTestDistributedQueries.java index da52a4a7591d..4d5021e7e198 100644 --- a/presto-testing/src/main/java/io/prestosql/testing/AbstractTestDistributedQueries.java +++ b/presto-testing/src/main/java/io/prestosql/testing/AbstractTestDistributedQueries.java @@ -14,6 +14,7 @@ package io.prestosql.testing; import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -26,15 +27,21 @@ import io.prestosql.execution.QueryInfo; import io.prestosql.execution.QueryManager; import io.prestosql.server.BasicQueryInfo; +import io.prestosql.spi.PrestoException; import io.prestosql.spi.security.Identity; import io.prestosql.testing.sql.TestTable; import org.intellij.lang.annotations.Language; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import java.util.List; +import java.util.Optional; import java.util.function.Supplier; +import java.util.regex.Pattern; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.Iterables.getOnlyElement; +import static com.google.common.collect.MoreCollectors.toOptional; import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; import static io.airlift.units.Duration.nanosSince; import static io.prestosql.SystemSessionProperties.QUERY_MAX_MEMORY; @@ -57,14 +64,17 @@ import static io.prestosql.testing.TestingAccessControlManager.privilege; import static io.prestosql.testing.TestingSession.TESTING_CATALOG; import static io.prestosql.testing.assertions.Assert.assertEquals; +import static io.prestosql.testing.sql.TestTable.randomTableSuffix; import static java.lang.String.format; import static java.lang.Thread.currentThread; import static java.util.Collections.nCopies; +import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -1169,4 +1179,152 @@ public void testInsertForDefaultColumn() } protected abstract TestTable createTableWithDefaultColumns(); + + @Test(dataProvider = "testDataMappingSmokeTestDataProvider") + public void testDataMappingSmokeTest(DataMappingTestSetup dataMappingTestSetup) + { + String prestoTypeName = dataMappingTestSetup.getPrestoTypeName(); + String sampleValueLiteral = dataMappingTestSetup.getSampleValueLiteral(); + String highValueLiteral = dataMappingTestSetup.getHighValueLiteral(); + + String tableName = "test_data_mapping_smoke_" + prestoTypeName.replaceAll("[^a-zA-Z0-9]", "_") + "_" + randomTableSuffix(); + + Runnable setup = () -> { + String createTable = format("CREATE TABLE %s(id bigint, value %s)", tableName, prestoTypeName); + assertUpdate(createTable); + assertUpdate( + format("INSERT INTO %s VALUES (10000, NULL), (10001, %s), (99999, %s)", tableName, sampleValueLiteral, highValueLiteral), + 3); + }; + if (dataMappingTestSetup.isUnsupportedType()) { + String typeNameBase = prestoTypeName.replaceFirst("\\(.*", ""); + String expectedMessagePart = format("(%1$s.*not (yet )?supported)|((?i)unsupported.*%1$s)|((?i)not supported.*%1$s)", Pattern.quote(typeNameBase)); + assertThatThrownBy(setup::run) + .hasMessageFindingMatch(expectedMessagePart) + .satisfies(e -> assertThat(getPrestoExceptionCause(e)).hasMessageFindingMatch(expectedMessagePart)); + return; + } + setup.run(); + + // without pushdown, i.e. test read data mapping + assertQuery("SELECT id FROM " + tableName + " WHERE rand() = 42 OR value IS NULL", "VALUES 10000"); + assertQuery("SELECT id FROM " + tableName + " WHERE rand() = 42 OR value IS NOT NULL", "VALUES (10001), (99999)"); + assertQuery("SELECT id FROM " + tableName + " WHERE rand() = 42 OR value = " + sampleValueLiteral, "VALUES 10001"); + assertQuery("SELECT id FROM " + tableName + " WHERE rand() = 42 OR value = " + highValueLiteral, "VALUES 99999"); + + assertQuery("SELECT id FROM " + tableName + " WHERE value IS NULL", "VALUES 10000"); + assertQuery("SELECT id FROM " + tableName + " WHERE value IS NOT NULL", "VALUES (10001), (99999)"); + assertQuery("SELECT id FROM " + tableName + " WHERE value = " + sampleValueLiteral, "VALUES 10001"); + assertQuery("SELECT id FROM " + tableName + " WHERE value != " + sampleValueLiteral, "VALUES 99999"); + assertQuery("SELECT id FROM " + tableName + " WHERE value <= " + sampleValueLiteral, "VALUES 10001"); + assertQuery("SELECT id FROM " + tableName + " WHERE value > " + sampleValueLiteral, "VALUES 99999"); + assertQuery("SELECT id FROM " + tableName + " WHERE value <= " + highValueLiteral, "VALUES (10001), (99999)"); + + assertUpdate("DROP TABLE " + tableName); + } + + @DataProvider + public final Object[][] testDataMappingSmokeTestDataProvider() + { + return testDataMappingSmokeTestData().stream() + .map(this::filterDataMappingSmokeTestData) + .filter(Optional::isPresent) + .map(Optional::get) + .map(dataMappingTestSetup -> new Object[] {dataMappingTestSetup}) + .toArray(Object[][]::new); + } + + private List testDataMappingSmokeTestData() + { + return ImmutableList.builder() + .add(new DataMappingTestSetup("tinyint", "37", "127")) + .add(new DataMappingTestSetup("smallint", "32123", "32767")) + .add(new DataMappingTestSetup("integer", "1274942432", "2147483647")) + .add(new DataMappingTestSetup("bigint", "312739231274942432", "9223372036854775807")) + .add(new DataMappingTestSetup("real", "REAL '567.123'", "REAL '999999.999'")) + .add(new DataMappingTestSetup("double", "DOUBLE '1234567890123.123'", "DOUBLE '9999999999999.999'")) + .add(new DataMappingTestSetup("decimal(5,3)", "12.345", "99.999")) + .add(new DataMappingTestSetup("decimal(15,3)", "123456789012.345", "999999999999.99")) + .add(new DataMappingTestSetup("date", "DATE '2020-02-12'", "DATE '9999-12-31'")) + .add(new DataMappingTestSetup("time", "TIME '15:03:00'", "TIME '23:59:59.999'")) + .add(new DataMappingTestSetup("timestamp", "TIMESTAMP '2020-02-12 15:03:00'", "TIMESTAMP '2199-12-31 23:59:59.999'")) + .add(new DataMappingTestSetup("timestamp with time zone", "TIMESTAMP '2020-02-12 15:03:00 +01:00'", "TIMESTAMP '9999-12-31 23:59:59.999 +12:00'")) + .add(new DataMappingTestSetup("char(3)", "'ab'", "'zzz'")) + .add(new DataMappingTestSetup("varchar(3)", "'de'", "'zzz'")) + .add(new DataMappingTestSetup("varchar", "'łąka for the win'", "'ŻŻŻŻŻŻŻŻŻŻ'")) + .add(new DataMappingTestSetup("varbinary", "X'12ab3f'", "X'ffffffffffffffffffff'")) + .build(); + } + + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + return Optional.of(dataMappingTestSetup); + } + + private static RuntimeException getPrestoExceptionCause(Throwable e) + { + return Throwables.getCausalChain(e).stream() + .filter(cause -> cause.toString().startsWith(PrestoException.class.getName() + ": ")) // e.g. io.prestosql.client.FailureInfo + .map(RuntimeException.class::cast) + .collect(toOptional()) + .orElseThrow(() -> new IllegalArgumentException("Exception does not have PrestoException cause", e)); + } + + protected static final class DataMappingTestSetup + { + private final String prestoTypeName; + private final String sampleValueLiteral; + private final String highValueLiteral; + + private final boolean unsupportedType; + + public DataMappingTestSetup(String prestoTypeName, String sampleValueLiteral, String highValueLiteral) + { + this(prestoTypeName, sampleValueLiteral, highValueLiteral, false); + } + + private DataMappingTestSetup(String prestoTypeName, String sampleValueLiteral, String highValueLiteral, boolean unsupportedType) + { + this.prestoTypeName = requireNonNull(prestoTypeName, "prestoTypeName is null"); + this.sampleValueLiteral = requireNonNull(sampleValueLiteral, "sampleValueLiteral is null"); + this.highValueLiteral = requireNonNull(highValueLiteral, "highValueLiteral is null"); + this.unsupportedType = unsupportedType; + } + + public String getPrestoTypeName() + { + return prestoTypeName; + } + + public String getSampleValueLiteral() + { + return sampleValueLiteral; + } + + public String getHighValueLiteral() + { + return highValueLiteral; + } + + public boolean isUnsupportedType() + { + return unsupportedType; + } + + public DataMappingTestSetup asUnsupported() + { + return new DataMappingTestSetup( + prestoTypeName, + sampleValueLiteral, + highValueLiteral, + true); + } + + @Override + public String toString() + { + // toString is brief because it's used for test case labels in IDE + return prestoTypeName + (unsupportedType ? "!" : ""); + } + } }