diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java index 4f59b8ad5af82..66127ebb6f93d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java @@ -261,6 +261,7 @@ public class FeaturesConfig private boolean pushRemoteExchangeThroughGroupId; private boolean isOptimizeMultipleApproxPercentileOnSameFieldEnabled = true; private boolean nativeExecutionEnabled; + private boolean disableTimeStampWithTimeZoneForNative = true; private String nativeExecutionExecutablePath = "./presto_server"; private String nativeExecutionProgramArguments = ""; private boolean nativeExecutionProcessReuseEnabled = true; @@ -2588,6 +2589,19 @@ public boolean isNativeExecutionEnabled() return this.nativeExecutionEnabled; } + @Config("disable-timestamp-with-timezone-for-native-execution") + @ConfigDescription("Disable timestamp with timezone type on native engine") + public FeaturesConfig setDisableTimeStampWithTimeZoneForNative(boolean disableTimeStampWithTimeZoneForNative) + { + this.disableTimeStampWithTimeZoneForNative = disableTimeStampWithTimeZoneForNative; + return this; + } + + public boolean isDisableTimeStampWithTimeZoneForNative() + { + return this.disableTimeStampWithTimeZoneForNative; + } + @Config("native-execution-executable-path") @ConfigDescription("Native execution executable file path") public FeaturesConfig setNativeExecutionExecutablePath(String nativeExecutionExecutablePath) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/CheckNoTimestampWithTimezoneType.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/CheckNoTimestampWithTimezoneType.java new file mode 100644 index 0000000000000..b05e00f21da3b --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/CheckNoTimestampWithTimezoneType.java @@ -0,0 +1,191 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.sql.planner.sanity; + +import com.facebook.presto.Session; +import com.facebook.presto.common.type.ArrayType; +import com.facebook.presto.common.type.MapType; +import com.facebook.presto.common.type.RowType; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.expressions.DefaultRowExpressionTraversalVisitor; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.spi.WarningCollector; +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.ProjectNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.spi.relation.CallExpression; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.InputReferenceExpression; +import com.facebook.presto.spi.relation.IntermediateFormExpression; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.SpecialFormExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.parser.SqlParser; +import com.facebook.presto.sql.planner.SimplePlanVisitor; +import com.facebook.presto.sql.planner.TypeProvider; +import com.facebook.presto.sql.planner.plan.WindowNode; + +import java.util.List; + +import static com.facebook.presto.common.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; +import static com.google.common.base.Preconditions.checkState; + +public class CheckNoTimestampWithTimezoneType + implements PlanChecker.Checker +{ + private static final String errorMessage = "Timestamp with Timezone type is not supported in Prestissimo"; + + @Override + public void validate(PlanNode planNode, Session session, Metadata metadata, SqlParser sqlParser, TypeProvider types, WarningCollector warningCollector) + { + planNode.accept(new Visitor(), null); + } + + private static class Visitor + extends SimplePlanVisitor + { + private final NoTimeStampWithTimeZoneTypeChecker noTimeStampWithTimeZoneTypeChecker; + + public Visitor() + { + this.noTimeStampWithTimeZoneTypeChecker = new NoTimeStampWithTimeZoneTypeChecker(); + } + + @Override + public Void visitPlan(PlanNode node, Void context) + { + checkState(node.getOutputVariables().stream().noneMatch(x -> hasTimestampWithTimezoneType(x.getType())), errorMessage); + return super.visitPlan(node, context); + } + + @Override + public Void visitAggregation(AggregationNode node, Void context) + { + visitPlan(node, context); + node.getAggregations().forEach((variable, aggregation) -> { + aggregation.getCall().accept(noTimeStampWithTimeZoneTypeChecker, null); + if (aggregation.getFilter().isPresent()) { + aggregation.getFilter().get().accept(noTimeStampWithTimeZoneTypeChecker, null); + } + }); + + return null; + } + + @Override + public Void visitWindow(WindowNode node, Void context) + { + visitPlan(node, context); + node.getWindowFunctions().forEach((variable, function) -> { + function.getFunctionCall().accept(noTimeStampWithTimeZoneTypeChecker, null); + }); + + return null; + } + + @Override + public Void visitProject(ProjectNode node, Void context) + { + visitPlan(node, context); + node.getAssignments().getMap().forEach((variable, expression) -> { + expression.accept(noTimeStampWithTimeZoneTypeChecker, null); + }); + + return null; + } + + @Override + public Void visitValues(ValuesNode node, Void context) + { + visitPlan(node, context); + for (List row : node.getRows()) { + row.forEach(x -> x.accept(noTimeStampWithTimeZoneTypeChecker, null)); + } + return null; + } + + @Override + public Void visitFilter(FilterNode node, Void context) + { + visitPlan(node, context); + node.getPredicate().accept(noTimeStampWithTimeZoneTypeChecker, null); + return null; + } + } + + private static class NoTimeStampWithTimeZoneTypeChecker + extends DefaultRowExpressionTraversalVisitor + { + @Override + public Void visitConstant(ConstantExpression literal, Void context) + { + checkState(!hasTimestampWithTimezoneType(literal.getType()), errorMessage); + return null; + } + + @Override + public Void visitVariableReference(VariableReferenceExpression reference, Void context) + { + checkState(!hasTimestampWithTimezoneType(reference.getType()), errorMessage); + return null; + } + + @Override + public Void visitInputReference(InputReferenceExpression input, Void context) + { + checkState(!hasTimestampWithTimezoneType(input.getType()), errorMessage); + return null; + } + + @Override + public Void visitCall(CallExpression call, Void context) + { + checkState(!hasTimestampWithTimezoneType(call.getType()), errorMessage); + return super.visitCall(call, context); + } + + @Override + public Void visitSpecialForm(SpecialFormExpression specialForm, Void context) + { + checkState(!hasTimestampWithTimezoneType(specialForm.getType()), errorMessage); + return super.visitSpecialForm(specialForm, context); + } + + @Override + public Void visitIntermediateFormExpression(IntermediateFormExpression expression, Void context) + { + checkState(!hasTimestampWithTimezoneType(expression.getType()), errorMessage); + return super.visitIntermediateFormExpression(expression, context); + } + } + + private static boolean hasTimestampWithTimezoneType(Type type) + { + if (type.equals(TIMESTAMP_WITH_TIME_ZONE)) { + return true; + } + if (type instanceof ArrayType) { + return hasTimestampWithTimezoneType(((ArrayType) type).getElementType()); + } + else if (type instanceof MapType) { + return hasTimestampWithTimezoneType(((MapType) type).getKeyType()) || hasTimestampWithTimezoneType(((MapType) type).getValueType()); + } + else if (type instanceof RowType) { + return ((RowType) type).getTypeParameters().stream().anyMatch(CheckNoTimestampWithTimezoneType::hasTimestampWithTimezoneType); + } + return false; + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/PlanChecker.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/PlanChecker.java index fe0e2be4f29dd..492ea797165cc 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/PlanChecker.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/PlanChecker.java @@ -40,8 +40,8 @@ public PlanChecker(FeaturesConfig featuresConfig) public PlanChecker(FeaturesConfig featuresConfig, boolean forceSingleNode) { - checkers = ImmutableListMultimap.builder() - .putAll( + ImmutableListMultimap.Builder builder = ImmutableListMultimap.builder(); + builder.putAll( Stage.INTERMEDIATE, new ValidateDependenciesChecker(), new NoDuplicatePlanNodeIdsChecker(), @@ -69,8 +69,11 @@ public PlanChecker(FeaturesConfig featuresConfig, boolean forceSingleNode) new VerifyNoIntermediateFormExpression(), new VerifyProjectionLocality(), new DynamicFiltersChecker(), - new WarnOnScanWithoutPartitionPredicate(featuresConfig)) - .build(); + new WarnOnScanWithoutPartitionPredicate(featuresConfig)); + if (featuresConfig.isNativeExecutionEnabled() && featuresConfig.isDisableTimeStampWithTimeZoneForNative()) { + builder.put(Stage.FINAL, new CheckNoTimestampWithTimezoneType()); + } + checkers = builder.build(); } public void validateFinalPlan(PlanNode planNode, Session session, Metadata metadata, SqlParser sqlParser, TypeProvider types, WarningCollector warningCollector) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java index 0d95e7c58d305..50a736912d8fb 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java @@ -226,6 +226,7 @@ public void testDefaults() .setPushRemoteExchangeThroughGroupId(false) .setOptimizeMultipleApproxPercentileOnSameFieldEnabled(true) .setNativeExecutionEnabled(false) + .setDisableTimeStampWithTimeZoneForNative(true) .setNativeExecutionExecutablePath("./presto_server") .setNativeExecutionProgramArguments("") .setNativeExecutionProcessReuseEnabled(true) @@ -443,6 +444,7 @@ public void testExplicitPropertyMappings() .put("optimizer.push-remote-exchange-through-group-id", "true") .put("optimizer.optimize-multiple-approx-percentile-on-same-field", "false") .put("native-execution-enabled", "true") + .put("disable-timestamp-with-timezone-for-native-execution", "false") .put("native-execution-executable-path", "/bin/echo") .put("native-execution-program-arguments", "--v 1") .put("native-execution-process-reuse-enabled", "false") @@ -658,6 +660,7 @@ public void testExplicitPropertyMappings() .setPushRemoteExchangeThroughGroupId(true) .setOptimizeMultipleApproxPercentileOnSameFieldEnabled(false) .setNativeExecutionEnabled(true) + .setDisableTimeStampWithTimeZoneForNative(false) .setNativeExecutionExecutablePath("/bin/echo") .setNativeExecutionProgramArguments("--v 1") .setNativeExecutionProcessReuseEnabled(false) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestCheckNoTimestampWithTimezoneType.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestCheckNoTimestampWithTimezoneType.java new file mode 100644 index 0000000000000..08e211f9da4d9 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestCheckNoTimestampWithTimezoneType.java @@ -0,0 +1,118 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.sql.planner.sanity; + +import com.facebook.presto.Session; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.spi.WarningCollector; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.PlanNodeIdAllocator; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.parser.SqlParser; +import com.facebook.presto.sql.planner.TypeProvider; +import com.facebook.presto.sql.planner.assertions.BasePlanTest; +import com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.function.Function; + +import static com.facebook.presto.SessionTestUtils.TEST_SESSION; +import static com.facebook.presto.common.type.BooleanType.BOOLEAN; +import static com.facebook.presto.common.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.sql.planner.iterative.rule.test.PlanBuilder.assignment; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; + +public class TestCheckNoTimestampWithTimezoneType + extends BasePlanTest +{ + private Session testSession; + private Metadata metadata; + private SqlParser sqlParser; + private PlanNodeIdAllocator idAllocator = new PlanNodeIdAllocator(); + + @BeforeClass + public void setup() + { + Session.SessionBuilder sessionBuilder = testSessionBuilder() + .setCatalog("local") + .setSchema("tiny"); + testSession = sessionBuilder.build(); + metadata = getQueryRunner().getMetadata(); + sqlParser = getQueryRunner().getSqlParser(); + } + + @AfterClass(alwaysRun = true) + public void tearDown() + { + testSession = null; + metadata = null; + sqlParser = null; + idAllocator = null; + } + + @Test(expectedExceptions = IllegalStateException.class, expectedExceptionsMessageRegExp = "Timestamp with Timezone type is not supported in Prestissimo") + public void testValidateProjectFail() + { + validatePlan( + p -> { + VariableReferenceExpression col = p.variable("col", VARCHAR); + VariableReferenceExpression col2 = p.variable("col2", TIMESTAMP_WITH_TIME_ZONE); + return p.project( + assignment(col2, p.rowExpression("cast(col as timestamp with time zone)")), + p.values(col)); + }); + } + + @Test(expectedExceptions = IllegalStateException.class, expectedExceptionsMessageRegExp = "Timestamp with Timezone type is not supported in Prestissimo") + public void testValidateProjectAssignmentFail() + { + validatePlan( + p -> { + VariableReferenceExpression col = p.variable("col", VARCHAR); + VariableReferenceExpression col1 = p.variable("col1", VARCHAR); + VariableReferenceExpression col2 = p.variable("col2", BOOLEAN); + return p.project( + assignment(col2, p.rowExpression("cast(col as timestamp with time zone) > cast(col1 as timestamp with time zone)")), + p.values(col, col1)); + }); + } + + @Test(expectedExceptions = IllegalStateException.class, expectedExceptionsMessageRegExp = "Timestamp with Timezone type is not supported in Prestissimo") + public void testValidateValueFail() + { + validatePlan( + p -> { + VariableReferenceExpression col = p.variable("col", TIMESTAMP_WITH_TIME_ZONE); + VariableReferenceExpression col2 = p.variable("col2", VARCHAR); + return p.project( + assignment(col2, p.rowExpression("cast(col as varchar)")), + p.values(col)); + }); + } + + private void validatePlan(Function planProvider) + { + PlanBuilder builder = new PlanBuilder(TEST_SESSION, idAllocator, metadata); + PlanNode planNode = planProvider.apply(builder); + TypeProvider types = builder.getTypes(); + getQueryRunner().inTransaction(testSession, session -> { + session.getCatalog().ifPresent(catalog -> metadata.getCatalogHandle(session, catalog)); + new CheckNoTimestampWithTimezoneType().validate(planNode, session, metadata, sqlParser, types, WarningCollector.NOOP); + return null; + }); + } +} diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeAggregations.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeAggregations.java index 0fc3a59f54388..5d6b6d625b6e9 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeAggregations.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeAggregations.java @@ -324,7 +324,7 @@ public void testChecksum() assertQuery("SELECT checksum(quantities) FROM orders_ex"); assertQuery("SELECT checksum(quantity_by_linenumber) FROM orders_ex"); assertQuery("SELECT shipmode, checksum(extendedprice) FROM lineitem GROUP BY shipmode"); - assertQuery("SELECT checksum(from_unixtime(orderkey, '+01:00')) FROM lineitem WHERE orderkey < 20"); + assertQueryFails("SELECT checksum(from_unixtime(orderkey, '+01:00')) FROM lineitem WHERE orderkey < 20", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); // test DECIMAL data assertQuery("SELECT checksum(a), checksum(b) FROM (VALUES (DECIMAL '1.234', DECIMAL '611180549424.4633133')) AS t(a, b)"); diff --git a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java index da07785e2da53..01db5755a681e 100644 --- a/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java +++ b/presto-native-execution/src/test/java/com/facebook/presto/nativeworker/AbstractTestNativeGeneralQueries.java @@ -520,13 +520,13 @@ public void testCast() assertQuery("SELECT cast(row(orderkey, comment) as row(\"123\" varchar, \"456\" varchar)) FROM orders"); // Cast timestamp with time zone - assertQuery("SELECT cast(from_unixtime(orderkey) as timestamp with time zone) from orders"); - assertQuery(legacyTimestampDisabled, "SELECT cast(from_unixtime(orderkey) as timestamp with time zone) from orders"); + assertQueryFails("SELECT cast(from_unixtime(orderkey) as timestamp with time zone) from orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); + assertQueryFails(legacyTimestampDisabled, "SELECT cast(from_unixtime(orderkey) as timestamp with time zone) from orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); // Cast timestamp with time zone to timestamp - assertQuery("SELECT cast(from_unixtime(orderkey, '+01:00') as timestamp), " + - "cast(from_unixtime(orderkey, 'America/Los_Angeles') as timestamp) from orders"); - assertQuery(legacyTimestampDisabled, "SELECT cast(from_unixtime(orderkey, '+01:00') as timestamp), " + - "cast(from_unixtime(orderkey, 'America/Los_Angeles') as timestamp) from orders"); + assertQueryFails("SELECT cast(from_unixtime(orderkey, '+01:00') as timestamp), " + + "cast(from_unixtime(orderkey, 'America/Los_Angeles') as timestamp) from orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); + assertQueryFails(legacyTimestampDisabled, "SELECT cast(from_unixtime(orderkey, '+01:00') as timestamp), " + + "cast(from_unixtime(orderkey, 'America/Los_Angeles') as timestamp) from orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); } @Test @@ -1203,22 +1203,22 @@ public void testArithmetic() @Test public void testTimestampWithTimeZone() { - assertQuery("SELECT from_unixtime(orderkey, '+01:00'), from_unixtime(orderkey, '-05:00'), from_unixtime(orderkey, 'Europe/Moscow') FROM orders"); - assertQuery("SELECT from_unixtime(orderkey, '+01:00'), count(1) FROM orders GROUP BY 1"); + assertQueryFails("SELECT from_unixtime(orderkey, '+01:00'), from_unixtime(orderkey, '-05:00'), from_unixtime(orderkey, 'Europe/Moscow') FROM orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); + assertQueryFails("SELECT from_unixtime(orderkey, '+01:00'), count(1) FROM orders GROUP BY 1", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); - assertQuery("SELECT parse_datetime(cast(1970 + nationkey as varchar) || '-01-02+00:' || cast(10 + (3 * nationkey) % 50 as varchar), 'YYYY-MM-dd+HH:mm'), parse_datetime(cast(1970 + nationkey as varchar) || '-01-02+00:' || cast(10 + (3 * nationkey) % 50 as varchar) || '+14:00', 'YYYY-MM-dd+HH:mmZZ') FROM nation"); + assertQueryFails("SELECT parse_datetime(cast(1970 + nationkey as varchar) || '-01-02+00:' || cast(10 + (3 * nationkey) % 50 as varchar), 'YYYY-MM-dd+HH:mm'), parse_datetime(cast(1970 + nationkey as varchar) || '-01-02+00:' || cast(10 + (3 * nationkey) % 50 as varchar) || '+14:00', 'YYYY-MM-dd+HH:mmZZ') FROM nation", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); - assertQuery("SELECT to_unixtime(from_unixtime(orderkey, '+01:00')), to_unixtime(from_unixtime(orderkey, '-05:00')), to_unixtime(from_unixtime(orderkey, 'Europe/Moscow')) FROM orders"); - assertQuery("SELECT to_unixtime(from_unixtime(orderkey, '+01:00')), count(1) FROM orders GROUP BY 1"); - assertQuery("SELECT to_unixtime(parse_datetime(cast(1970 + nationkey as varchar) || '-01-02+00:' || cast(10 + (3 * nationkey) % 50 as varchar), 'YYYY-MM-dd+HH:mm')), to_unixtime(parse_datetime(cast(1970 + nationkey as varchar) || '-01-02+00:' || cast(10 + (3 * nationkey) % 50 as varchar) || '+14:00', 'YYYY-MM-dd+HH:mmZZ')) FROM nation"); - assertQuery("SELECT timestamp '2012-10-31 01:00 UTC' AT TIME ZONE 'America/Los_Angeles'"); - assertQuery("SELECT ARRAY[timestamp '2018-02-06 23:00:00.000 Australia/Melbourne', null, timestamp '2012-10-31 01:00 UTC' AT TIME ZONE 'America/Los_Angeles']"); + assertQueryFails("SELECT to_unixtime(from_unixtime(orderkey, '+01:00')), to_unixtime(from_unixtime(orderkey, '-05:00')), to_unixtime(from_unixtime(orderkey, 'Europe/Moscow')) FROM orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); + assertQueryFails("SELECT to_unixtime(from_unixtime(orderkey, '+01:00')), count(1) FROM orders GROUP BY 1", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); + assertQueryFails("SELECT to_unixtime(parse_datetime(cast(1970 + nationkey as varchar) || '-01-02+00:' || cast(10 + (3 * nationkey) % 50 as varchar), 'YYYY-MM-dd+HH:mm')), to_unixtime(parse_datetime(cast(1970 + nationkey as varchar) || '-01-02+00:' || cast(10 + (3 * nationkey) % 50 as varchar) || '+14:00', 'YYYY-MM-dd+HH:mmZZ')) FROM nation", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); + assertQueryFails("SELECT timestamp '2012-10-31 01:00 UTC' AT TIME ZONE 'America/Los_Angeles'", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); + assertQueryFails("SELECT ARRAY[timestamp '2018-02-06 23:00:00.000 Australia/Melbourne', null, timestamp '2012-10-31 01:00 UTC' AT TIME ZONE 'America/Los_Angeles']", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); - assertQuery("SELECT orderkey, year(from_unixtime(orderkey, '+01:00')), quarter(from_unixtime(orderkey, '-07:00')), month(from_unixtime(orderkey, '+00:00')), day(from_unixtime(orderkey, '-13:00')), day_of_week(from_unixtime(orderkey, '+03:00')), day_of_year(from_unixtime(orderkey, '-13:00')), year_of_week(from_unixtime(orderkey, '+14:00')), hour(from_unixtime(orderkey, '+01:00')), minute(from_unixtime(orderkey, '+01:00')), second(from_unixtime(orderkey, '-07:00')), millisecond(from_unixtime(orderkey, '+03:00')) FROM orders"); - assertQuery("SELECT orderkey, date_trunc('year', from_unixtime(orderkey, '-03:00')), date_trunc('quarter', from_unixtime(orderkey, '+14:00')), date_trunc('month', from_unixtime(orderkey, '+03:00')), date_trunc('day', from_unixtime(orderkey, '-07:00')), date_trunc('hour', from_unixtime(orderkey, '-09:30')), date_trunc('minute', from_unixtime(orderkey, '+05:30')), date_trunc('second', from_unixtime(orderkey, '+00:00')) FROM orders"); + assertQueryFails("SELECT orderkey, year(from_unixtime(orderkey, '+01:00')), quarter(from_unixtime(orderkey, '-07:00')), month(from_unixtime(orderkey, '+00:00')), day(from_unixtime(orderkey, '-13:00')), day_of_week(from_unixtime(orderkey, '+03:00')), day_of_year(from_unixtime(orderkey, '-13:00')), year_of_week(from_unixtime(orderkey, '+14:00')), hour(from_unixtime(orderkey, '+01:00')), minute(from_unixtime(orderkey, '+01:00')), second(from_unixtime(orderkey, '-07:00')), millisecond(from_unixtime(orderkey, '+03:00')) FROM orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); + assertQueryFails("SELECT orderkey, date_trunc('year', from_unixtime(orderkey, '-03:00')), date_trunc('quarter', from_unixtime(orderkey, '+14:00')), date_trunc('month', from_unixtime(orderkey, '+03:00')), date_trunc('day', from_unixtime(orderkey, '-07:00')), date_trunc('hour', from_unixtime(orderkey, '-09:30')), date_trunc('minute', from_unixtime(orderkey, '+05:30')), date_trunc('second', from_unixtime(orderkey, '+00:00')) FROM orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); - assertQuery("SELECT timezone_hour(from_unixtime(orderkey, 'Asia/Oral')) FROM orders"); - assertQuery("SELECT timezone_minute(from_unixtime(orderkey, 'Asia/Kolkata')) FROM orders"); + assertQueryFails("SELECT timezone_hour(from_unixtime(orderkey, 'Asia/Oral')) FROM orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); + assertQueryFails("SELECT timezone_minute(from_unixtime(orderkey, 'Asia/Kolkata')) FROM orders", ".*Timestamp with Timezone type is not supported in Prestissimo.*"); } @Test