You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ro...@apache.org on 2022/09/29 17:50:11 UTC

[pinot] branch master updated: add parsing for AT TIME ZONE (#9477)

This is an automated email from the ASF dual-hosted git repository.

rongr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new dc84310077 add parsing for AT TIME ZONE (#9477)
dc84310077 is described below

commit dc84310077765ce367d380c97aa67f9e4caeb0bf
Author: Almog Gavra <al...@gmail.com>
AuthorDate: Thu Sep 29 10:50:04 2022 -0700

    add parsing for AT TIME ZONE (#9477)
    
    * add parsing for AT TIME ZONE
    
    This commit adds the custom Calcite extension for
    parsing AT TIME ZONE expressions, but does not yet
    support executing these queries.
---
 pinot-common/src/main/codegen/config.fmpp          |  5 ++
 .../src/main/codegen/includes/parserImpls.ftl      | 17 ++++++
 .../pinot/sql/parsers/parser/SqlAtTimeZone.java    | 63 ++++++++++++++++++++++
 .../pinot/sql/parsers/CalciteSqlCompilerTest.java  | 57 ++++++++++++++++++++
 .../apache/pinot/queries/TimestampQueriesTest.java | 12 +++++
 .../apache/pinot/query/QueryCompilationTest.java   |  2 +
 6 files changed, 156 insertions(+)

diff --git a/pinot-common/src/main/codegen/config.fmpp b/pinot-common/src/main/codegen/config.fmpp
index 0b1f558e89..e6955766bc 100644
--- a/pinot-common/src/main/codegen/config.fmpp
+++ b/pinot-common/src/main/codegen/config.fmpp
@@ -528,6 +528,11 @@ data: {
       "SqlInsertFromFile()"
     ]
 
+    # List of custom function syntax to add
+    extraBinaryExpressions: [
+      "SqlAtTimeZone"
+    ]
+
     # List of files in @includes directory that have parser method
     implementationFiles: [
       "parserImpls.ftl"
diff --git a/pinot-common/src/main/codegen/includes/parserImpls.ftl b/pinot-common/src/main/codegen/includes/parserImpls.ftl
index 3b85f19aff..449d8ab3b9 100644
--- a/pinot-common/src/main/codegen/includes/parserImpls.ftl
+++ b/pinot-common/src/main/codegen/includes/parserImpls.ftl
@@ -102,3 +102,20 @@ SqlNodeList SqlStmtsEof() :
         return stmts;
     }
 }
+
+void SqlAtTimeZone(List<Object> list, ExprContext exprContext, Span s) :
+{
+    List<Object> list2;
+    SqlOperator op;
+}
+{
+    {
+        checkNonQueryExpression(exprContext);
+        s.clear().add(this);
+    }
+    <AT> <TIME> <ZONE> { op = SqlAtTimeZone.INSTANCE; }
+    list2 = Expression2(ExprContext.ACCEPT_SUB_QUERY) {
+        list.add(new SqlParserUtil.ToTreeListItem(op, s.pos()));
+        list.addAll(list2);
+    }
+}
diff --git a/pinot-common/src/main/java/org/apache/pinot/sql/parsers/parser/SqlAtTimeZone.java b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/parser/SqlAtTimeZone.java
new file mode 100644
index 0000000000..fbeb01b7ca
--- /dev/null
+++ b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/parser/SqlAtTimeZone.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.pinot.sql.parsers.parser;
+
+import java.util.Arrays;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+import static java.util.Objects.requireNonNull;
+
+
+public class SqlAtTimeZone extends SqlSpecialOperator {
+
+  public static final SqlAtTimeZone INSTANCE = new SqlAtTimeZone();
+
+  private SqlAtTimeZone() {
+    super(
+        "AT_TIME_ZONE",
+        SqlKind.OTHER_FUNCTION,
+        32, // put this at the same precedence as LIKE (see SqlLikeOperator)
+        false,
+        opBinding -> opBinding.getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP),
+        null,
+        OperandTypes.family(SqlTypeFamily.TIME)
+    );
+  }
+
+  @Override
+  public ReduceResult reduceExpr(int ordinal, TokenSequence list) {
+    SqlNode left = list.node(ordinal - 1);
+    SqlNode right = list.node(ordinal + 1);
+    return new ReduceResult(ordinal - 1,
+        ordinal + 2,
+        createCall(
+            SqlParserPos.sum(
+                Arrays.asList(requireNonNull(left, "left").getParserPosition(),
+                    requireNonNull(right, "right").getParserPosition(),
+                    list.pos(ordinal))),
+            left,
+            right));
+  }
+}
diff --git a/pinot-common/src/test/java/org/apache/pinot/sql/parsers/CalciteSqlCompilerTest.java b/pinot-common/src/test/java/org/apache/pinot/sql/parsers/CalciteSqlCompilerTest.java
index 5f69fd7001..c662af0468 100644
--- a/pinot-common/src/test/java/org/apache/pinot/sql/parsers/CalciteSqlCompilerTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/sql/parsers/CalciteSqlCompilerTest.java
@@ -2681,6 +2681,63 @@ public class CalciteSqlCompilerTest {
     Assert.assertEquals(sqlNodeAndOptions.getSqlType(), PinotSqlType.DML);
   }
 
+
+  @Test
+  public void shouldParseBasicAtTimeZoneExtension() {
+    // Given:
+    String sql = "SELECT ts AT TIME ZONE 'pst' FROM myTable;";
+
+    // When:
+    PinotQuery pinotQuery = CalciteSqlParser.compileToPinotQuery(sql);
+
+    // Then:
+    Assert.assertEquals(pinotQuery.getSelectListSize(), 1);
+    Function fun = pinotQuery.getSelectList().get(0).getFunctionCall();
+    Assert.assertEquals(fun.operator, "attimezone");
+    Assert.assertEquals(fun.operands.size(), 2);
+    Assert.assertEquals(fun.operands.get(0).getIdentifier().name, "ts");
+    Assert.assertEquals(fun.operands.get(1).getLiteral().getStringValue(), "pst");
+  }
+
+  @Test
+  public void shouldParseNestedTimeExprAtTimeZoneExtension() {
+    // Given:
+    String sql = "SELECT ts + 123 AT TIME ZONE 'pst' FROM myTable;";
+
+    // When:
+    PinotQuery pinotQuery = CalciteSqlParser.compileToPinotQuery(sql);
+
+    // Then:
+    Assert.assertEquals(pinotQuery.getSelectListSize(), 1);
+    Function fun = pinotQuery.getSelectList().get(0).getFunctionCall();
+    Assert.assertEquals(fun.operator, "attimezone");
+    Assert.assertEquals(fun.operands.size(), 2);
+    Assert.assertEquals(fun.operands.get(0).getFunctionCall().operator, "plus");
+    Assert.assertEquals(fun.operands.get(0).getFunctionCall().operands.size(), 2);
+    Assert.assertEquals(fun.operands.get(0).getFunctionCall().operands.get(0).getIdentifier().getName(), "ts");
+    Assert.assertEquals(fun.operands.get(0).getFunctionCall().operands.get(1).getLiteral().getLongValue(), 123L);
+    Assert.assertEquals(fun.operands.get(1).getLiteral().getStringValue(), "pst");
+  }
+
+  @Test
+  public void shouldParseOutsideExprAtTimeZoneExtension() {
+    // Given:
+    String sql = "SELECT ts AT TIME ZONE 'pst' > 123 FROM myTable;";
+
+    // When:
+    PinotQuery pinotQuery = CalciteSqlParser.compileToPinotQuery(sql);
+
+    // Then:
+    Assert.assertEquals(pinotQuery.getSelectListSize(), 1);
+    Function fun = pinotQuery.getSelectList().get(0).getFunctionCall();
+    Assert.assertEquals(fun.operator, "GREATER_THAN");
+    Assert.assertEquals(fun.operands.size(), 2);
+    Assert.assertEquals(fun.operands.get(0).getFunctionCall().operator, "attimezone");
+    Assert.assertEquals(fun.operands.get(0).getFunctionCall().operands.size(), 2);
+    Assert.assertEquals(fun.operands.get(0).getFunctionCall().operands.get(0).getIdentifier().getName(), "ts");
+    Assert.assertEquals(fun.operands.get(0).getFunctionCall().operands.get(1).getLiteral().getStringValue(), "pst");
+  }
+
   private static SqlNodeAndOptions testSqlWithCustomSqlParser(String sqlString) {
     try (StringReader inStream = new StringReader(sqlString)) {
       SqlParserImpl sqlParser = CalciteSqlParser.newSqlParser(inStream);
diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
index 0bded079b3..c96c993ed2 100644
--- a/pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/queries/TimestampQueriesTest.java
@@ -40,6 +40,7 @@ import org.apache.pinot.spi.config.table.TableType;
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.exception.BadQueryRequestException;
 import org.apache.pinot.spi.utils.ReadMode;
 import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
 import org.testng.annotations.AfterClass;
@@ -222,6 +223,17 @@ public class TimestampQueriesTest extends BaseQueriesTest {
     }
   }
 
+  @Test(
+      expectedExceptions = BadQueryRequestException.class,
+      expectedExceptionsMessageRegExp = ".*attimezone not found.*"
+  )
+  public void shouldThrowOnAtTimeZone() {
+    // this isn't yet implemented but the syntax is supported, make sure the
+    // degradation experience is clean
+    String query = "SELECT timestampColumn AT TIME ZONE 'pst' FROM testTable";
+    getBrokerResponse(query);
+  }
+
   @AfterClass
   public void tearDown()
       throws IOException {
diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java
index 0531cfaba8..9c220b4438 100644
--- a/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java
+++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/QueryCompilationTest.java
@@ -257,6 +257,8 @@ public class QueryCompilationTest extends QueryEnvironmentTestBase {
         new Object[]{"SELECT a.col1, SUM(a.col3) FROM a", "'a.col1' is not being grouped"},
         // empty IN clause fails compilation
         new Object[]{"SELECT a.col1 FROM a WHERE a.col1 IN ()", "Encountered \"\" at line"},
+        // AT TIME ZONE should fail
+        new Object[]{"SELECT a.col1 AT TIME ZONE 'PST' FROM a", "No match found for function signature AT_TIME_ZONE"},
     };
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org