You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by GitBox <gi...@apache.org> on 2020/08/10 12:45:09 UTC

[GitHub] [calcite] danny0405 opened a new pull request #2103: [CALCITE-4171] Support named parameters for table window functions

danny0405 opened a new pull request #2103:
URL: https://github.com/apache/calcite/pull/2103


   * Changes SqlArgumentAssignmentOperator to allow non-scala query as operand
   * In SqlCallBinding, matches the permuted operand by name with name matcher
   * Refactor SqlWindowTableFunction and its sub-class to reuse same logic
   * With explicit named params, the non-specified param would be replaced
     with a DEFAULT call


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r468072910



##########
File path: core/src/main/java/org/apache/calcite/sql/fun/SqlArgumentAssignmentOperator.java
##########
@@ -47,4 +47,8 @@
     writer.keyword(getName());
     call.operand(0).unparse(writer, getRightPrec(), rightPrec);
   }
+
+  @Override public boolean argumentMustBeScalar(int ordinal) {
+    return false;

Review comment:
       This is really nice. I actually tried to add named parameter support but failed on make non-scalar work.
   
   I wasn't aware this can be as simple as override this parameter in SqlArgumentAssignmentOperator.java




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] danny0405 commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469634129



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -98,8 +173,8 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
     return typeFactory.builder()
         .kind(inputRowType.getStructKind())
         .addAll(inputRowType.getFieldList())
-        .add("window_start", timestampType)
-        .add("window_end", timestampType)
+        .add("wstart", timestampType)
+        .add("wend", timestampType)

Review comment:
       Well, I didn’t notice that, can you share the discussion?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469747848



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -98,8 +173,8 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
     return typeFactory.builder()
         .kind(inputRowType.getStructKind())
         .addAll(inputRowType.getFieldList())
-        .add("window_start", timestampType)
-        .add("window_end", timestampType)
+        .add("wstart", timestampType)
+        .add("wend", timestampType)

Review comment:
       ok. I might or might not discussed this in Calcite (because I cannot find a JIRA that about it), but
   
   1. This is the acknowledge from Kenneth Knowles, one of the paper authors: https://docs.google.com/document/d/138uA7VTpbF84CFrd--cz3YVe0-AQ9ALnsavaSE2JeE4/edit?disco=AAAAEAjWvxg
   
   2. I started to use window_start and window_end in https://issues.apache.org/jira/browse/CALCITE-3272. Julian, as another paper author, didn't against it I think.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469619227



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -66,7 +86,53 @@ protected boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding call
     }
   }
 
-  protected void validateColumnNames(SqlValidator validator,
+  /**
+   * Validate the heading operands are in the form:
+   * (ROW, DESCRIPTOR, DESCRIPTOR ..., other params).
+   *
+   * @param callBinding The call binding
+   * @param descriptors The number of descriptors following the first operand (e.g. the table)
+   *
+   * @return true if validation passes
+   */
+  protected boolean validateTableWithFollowingDescriptors(
+      SqlCallBinding callBinding, int descriptors) {
+    final SqlNode operand0 = callBinding.operand(0);
+    final SqlValidator validator = callBinding.getValidator();
+    final RelDataType type = validator.getValidatedNodeType(operand0);
+    if (type.getSqlTypeName() != SqlTypeName.ROW) {
+      return false;
+    }
+    for (int i = 1; i < descriptors + 1; i++) {
+      final SqlNode operand = callBinding.operand(i);

Review comment:
       Hi @danny0405  
   
   any suggestions on https://issues.apache.org/jira/browse/CALCITE-3780?focusedCommentId=17169620&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17169620
   
   Your change will assume that the structure of window table functions be `row, (ROW, DESCRIPTOR, DESCRIPTOR ..., other params)`, in the comment we are thinking to make key descriptor of SESSION as optional, which might mean it will be moved to the last of the arguments so it becomes `SESSION(data TABLE, ts DESCRIPTOR, gap interval, key DESCRIPTOR optional)`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] danny0405 commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469812407



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -66,7 +86,53 @@ protected boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding call
     }
   }
 
-  protected void validateColumnNames(SqlValidator validator,
+  /**
+   * Validate the heading operands are in the form:
+   * (ROW, DESCRIPTOR, DESCRIPTOR ..., other params).
+   *
+   * @param callBinding The call binding
+   * @param descriptors The number of descriptors following the first operand (e.g. the table)
+   *
+   * @return true if validation passes
+   */
+  protected boolean validateTableWithFollowingDescriptors(
+      SqlCallBinding callBinding, int descriptors) {
+    final SqlNode operand0 = callBinding.operand(0);
+    final SqlValidator validator = callBinding.getValidator();
+    final RelDataType type = validator.getValidatedNodeType(operand0);
+    if (type.getSqlTypeName() != SqlTypeName.ROW) {
+      return false;
+    }
+    for (int i = 1; i < descriptors + 1; i++) {
+      final SqlNode operand = callBinding.operand(i);

Review comment:
       I think it is still straight-forward when it is the form `(ROW, DESCRIPTOR ..., other params)`, in this form, we always assume that the DESCRIPTOR has the column as the time column.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469610312



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -98,8 +173,8 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
     return typeFactory.builder()
         .kind(inputRowType.getStructKind())
         .addAll(inputRowType.getFieldList())
-        .add("window_start", timestampType)
-        .add("window_end", timestampType)
+        .add("wstart", timestampType)
+        .add("wend", timestampType)

Review comment:
       Note that I asked one of the authors of the "one sql to rule them all" paper. The only reason that they use wstart and wend was because of the layout for a paper. 
   
   I also believe in a JIRA or an email (sorry I cannot recall which one it is) we agreed to switch to window_start and window_end.
   
   So I will suggest we keep window_start and window_end, which can be understood easier.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469642630



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -98,8 +173,8 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
     return typeFactory.builder()
         .kind(inputRowType.getStructKind())
         .addAll(inputRowType.getFieldList())
-        .add("window_start", timestampType)
-        .add("window_end", timestampType)
+        .add("wstart", timestampType)
+        .add("wend", timestampType)

Review comment:
       I will spend some time to search for that JIRA.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r470248449



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -66,7 +86,53 @@ protected boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding call
     }
   }
 
-  protected void validateColumnNames(SqlValidator validator,
+  /**
+   * Validate the heading operands are in the form:
+   * (ROW, DESCRIPTOR, DESCRIPTOR ..., other params).
+   *
+   * @param callBinding The call binding
+   * @param descriptors The number of descriptors following the first operand (e.g. the table)
+   *
+   * @return true if validation passes
+   */
+  protected boolean validateTableWithFollowingDescriptors(
+      SqlCallBinding callBinding, int descriptors) {
+    final SqlNode operand0 = callBinding.operand(0);
+    final SqlValidator validator = callBinding.getValidator();
+    final RelDataType type = validator.getValidatedNodeType(operand0);
+    if (type.getSqlTypeName() != SqlTypeName.ROW) {
+      return false;
+    }
+    for (int i = 1; i < descriptors + 1; i++) {
+      final SqlNode operand = callBinding.operand(i);

Review comment:
       Ok. I will take a look what is the best way when authoring that PR.

##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -98,8 +173,8 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
     return typeFactory.builder()
         .kind(inputRowType.getStructKind())
         .addAll(inputRowType.getFieldList())
-        .add("window_start", timestampType)
-        .add("window_end", timestampType)
+        .add("wstart", timestampType)
+        .add("wend", timestampType)

Review comment:
       THank you!

##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -98,8 +173,8 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
     return typeFactory.builder()
         .kind(inputRowType.getStructKind())
         .addAll(inputRowType.getFieldList())
-        .add("window_start", timestampType)
-        .add("window_end", timestampType)
+        .add("wstart", timestampType)
+        .add("wend", timestampType)

Review comment:
       Thank you!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469610312



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -98,8 +173,8 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
     return typeFactory.builder()
         .kind(inputRowType.getStructKind())
         .addAll(inputRowType.getFieldList())
-        .add("window_start", timestampType)
-        .add("window_end", timestampType)
+        .add("wstart", timestampType)
+        .add("wend", timestampType)

Review comment:
       Note that I asked one of the authors of the "one sql to rule them all" paper. The only reason that they use wstart and wend was because of the layout for a paper. I got suggestion to use window_start and window_end.
   
   I also believe in a JIRA or an email (sorry I cannot recall which one it is) we agreed to switch to window_start and window_end.
   
   So I will suggest we keep window_start and window_end, which can be understood easier than wstart/wend.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] chunweilei commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
chunweilei commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r468966224



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -66,7 +86,53 @@ protected boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding call
     }
   }
 
-  protected void validateColumnNames(SqlValidator validator,
+  /**
+   * Validate the heading operands are in the form:
+   * (ROW, DESCRIPTOR, DESCRIPTOR ..., other params).
+   *
+   * @param callBinding The call binding
+   * @param descriptors The number of descriptors following the first operand (e.g. the table)
+   *
+   * @return true if validation passes
+   */
+  protected boolean validateTableWithFollowingDescriptors(
+      SqlCallBinding callBinding, int descriptors) {
+    final SqlNode operand0 = callBinding.operand(0);
+    final SqlValidator validator = callBinding.getValidator();
+    final RelDataType type = validator.getValidatedNodeType(operand0);
+    if (type.getSqlTypeName() != SqlTypeName.ROW) {
+      return false;
+    }
+    for (int i = 1; i < descriptors + 1; i++) {
+      final SqlNode operand = callBinding.operand(i);

Review comment:
       Why use `i < descriptors + 1` rathen than `i <= descriptors`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] danny0405 merged pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
danny0405 merged pull request #2103:
URL: https://github.com/apache/calcite/pull/2103


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] danny0405 commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469812746



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -98,8 +173,8 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
     return typeFactory.builder()
         .kind(inputRowType.getStructKind())
         .addAll(inputRowType.getFieldList())
-        .add("window_start", timestampType)
-        .add("window_end", timestampType)
+        .add("wstart", timestampType)
+        .add("wend", timestampType)

Review comment:
       Thanks for the share, i have reverted the rename back.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] chunweilei commented on pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
chunweilei commented on pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#issuecomment-672634130


   FYI, master branch is still locked~~


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469619227



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -66,7 +86,53 @@ protected boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding call
     }
   }
 
-  protected void validateColumnNames(SqlValidator validator,
+  /**
+   * Validate the heading operands are in the form:
+   * (ROW, DESCRIPTOR, DESCRIPTOR ..., other params).
+   *
+   * @param callBinding The call binding
+   * @param descriptors The number of descriptors following the first operand (e.g. the table)
+   *
+   * @return true if validation passes
+   */
+  protected boolean validateTableWithFollowingDescriptors(
+      SqlCallBinding callBinding, int descriptors) {
+    final SqlNode operand0 = callBinding.operand(0);
+    final SqlValidator validator = callBinding.getValidator();
+    final RelDataType type = validator.getValidatedNodeType(operand0);
+    if (type.getSqlTypeName() != SqlTypeName.ROW) {
+      return false;
+    }
+    for (int i = 1; i < descriptors + 1; i++) {
+      final SqlNode operand = callBinding.operand(i);

Review comment:
       Hi @danny0405  
   
   any suggestions on https://issues.apache.org/jira/browse/CALCITE-3780?focusedCommentId=17169620&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17169620?
   
   Your change will assume that the structure of window table functions be `row, (ROW, DESCRIPTOR, DESCRIPTOR ..., other params)`, in the comment we are thinking to make key descriptor of SESSION as optional, which might mean it will be moved to the last of the arguments so it becomes `SESSION(data TABLE, ts DESCRIPTOR, gap interval, key DESCRIPTOR optional)`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] danny0405 commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469036354



##########
File path: core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
##########
@@ -10287,13 +10287,41 @@ private void checkCustomColumnResolving(String table) {
   }
 
   @Test public void testTumbleTableFunction() {
-    sql("select * from table(\n"
-        + "^tumble(table orders, descriptor(rowtime))^)")
-        .fails("Invalid number of arguments to function 'TUMBLE'. Was expecting 3 arguments");
     sql("select rowtime, productid, orderid, 'window_start', 'window_end' from table(\n"
         + "tumble(table orders, descriptor(rowtime), interval '2' hour))").ok();
     sql("select rowtime, productid, orderid, 'window_start', 'window_end' from table(\n"
         + "tumble(table orders, descriptor(rowtime), interval '2' hour, interval '1' hour))").ok();
+    // test named params.
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "data => table orders,\n"
+        + "timecol => descriptor(rowtime),\n"
+        + "size => interval '2' hour))").ok();
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "data => table orders,\n"
+        + "timecol => descriptor(rowtime),\n"
+        + "size => interval '2' hour,\n"
+        + "\"OFFSET\" => interval '1' hour))").ok();
+    // negative tests.
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "^\"data\"^ => table orders,\n"
+        + "TIMECOL => descriptor(rowtime),\n"
+        + "SIZE => interval '2' hour))")
+        .fails("Param 'data' not found in function 'TUMBLE'; did you mean 'DATA'\\?");

Review comment:
       No, but only the first `data` param, because for a table function, only the first param can be a query.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] chunweilei commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
chunweilei commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r468966224



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -66,7 +86,53 @@ protected boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding call
     }
   }
 
-  protected void validateColumnNames(SqlValidator validator,
+  /**
+   * Validate the heading operands are in the form:
+   * (ROW, DESCRIPTOR, DESCRIPTOR ..., other params).
+   *
+   * @param callBinding The call binding
+   * @param descriptors The number of descriptors following the first operand (e.g. the table)
+   *
+   * @return true if validation passes
+   */
+  protected boolean validateTableWithFollowingDescriptors(
+      SqlCallBinding callBinding, int descriptors) {
+    final SqlNode operand0 = callBinding.operand(0);
+    final SqlValidator validator = callBinding.getValidator();
+    final RelDataType type = validator.getValidatedNodeType(operand0);
+    if (type.getSqlTypeName() != SqlTypeName.ROW) {
+      return false;
+    }
+    for (int i = 1; i < descriptors + 1; i++) {
+      final SqlNode operand = callBinding.operand(i);

Review comment:
       Why use `i < descriptors + 1` rather than `i <= descriptors`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] chunweilei commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
chunweilei commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469059281



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -66,7 +86,53 @@ protected boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding call
     }
   }
 
-  protected void validateColumnNames(SqlValidator validator,
+  /**
+   * Validate the heading operands are in the form:
+   * (ROW, DESCRIPTOR, DESCRIPTOR ..., other params).
+   *
+   * @param callBinding The call binding
+   * @param descriptors The number of descriptors following the first operand (e.g. the table)
+   *
+   * @return true if validation passes
+   */
+  protected boolean validateTableWithFollowingDescriptors(
+      SqlCallBinding callBinding, int descriptors) {
+    final SqlNode operand0 = callBinding.operand(0);
+    final SqlValidator validator = callBinding.getValidator();
+    final RelDataType type = validator.getValidatedNodeType(operand0);
+    if (type.getSqlTypeName() != SqlTypeName.ROW) {
+      return false;
+    }
+    for (int i = 1; i < descriptors + 1; i++) {
+      final SqlNode operand = callBinding.operand(i);

Review comment:
       `i <= descriptors` is better since it has fewer operations.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] danny0405 commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469036912



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -66,7 +86,53 @@ protected boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding call
     }
   }
 
-  protected void validateColumnNames(SqlValidator validator,
+  /**
+   * Validate the heading operands are in the form:
+   * (ROW, DESCRIPTOR, DESCRIPTOR ..., other params).
+   *
+   * @param callBinding The call binding
+   * @param descriptors The number of descriptors following the first operand (e.g. the table)
+   *
+   * @return true if validation passes
+   */
+  protected boolean validateTableWithFollowingDescriptors(
+      SqlCallBinding callBinding, int descriptors) {
+    final SqlNode operand0 = callBinding.operand(0);
+    final SqlValidator validator = callBinding.getValidator();
+    final RelDataType type = validator.getValidatedNodeType(operand0);
+    if (type.getSqlTypeName() != SqlTypeName.ROW) {
+      return false;
+    }
+    for (int i = 1; i < descriptors + 1; i++) {
+      final SqlNode operand = callBinding.operand(i);

Review comment:
       Because the `descriptors ` represent the number of the descriptors.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r468072910



##########
File path: core/src/main/java/org/apache/calcite/sql/fun/SqlArgumentAssignmentOperator.java
##########
@@ -47,4 +47,8 @@
     writer.keyword(getName());
     call.operand(0).unparse(writer, getRightPrec(), rightPrec);
   }
+
+  @Override public boolean argumentMustBeScalar(int ordinal) {
+    return false;

Review comment:
       This is really nice. I actually tried to add named parameter support but failed on making non-scalar case (i.e. table function) work.
   
   I wasn't aware that this can be as simple as override this parameter in SqlArgumentAssignmentOperator.java




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] danny0405 commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469132103



##########
File path: core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
##########
@@ -10287,13 +10287,41 @@ private void checkCustomColumnResolving(String table) {
   }
 
   @Test public void testTumbleTableFunction() {
-    sql("select * from table(\n"
-        + "^tumble(table orders, descriptor(rowtime))^)")
-        .fails("Invalid number of arguments to function 'TUMBLE'. Was expecting 3 arguments");
     sql("select rowtime, productid, orderid, 'window_start', 'window_end' from table(\n"
         + "tumble(table orders, descriptor(rowtime), interval '2' hour))").ok();
     sql("select rowtime, productid, orderid, 'window_start', 'window_end' from table(\n"
         + "tumble(table orders, descriptor(rowtime), interval '2' hour, interval '1' hour))").ok();
+    // test named params.
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "data => table orders,\n"
+        + "timecol => descriptor(rowtime),\n"
+        + "size => interval '2' hour))").ok();
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "data => table orders,\n"
+        + "timecol => descriptor(rowtime),\n"
+        + "size => interval '2' hour,\n"
+        + "\"OFFSET\" => interval '1' hour))").ok();
+    // negative tests.
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "^\"data\"^ => table orders,\n"
+        + "TIMECOL => descriptor(rowtime),\n"
+        + "SIZE => interval '2' hour))")
+        .fails("Param 'data' not found in function 'TUMBLE'; did you mean 'DATA'\\?");

Review comment:
       Already added in the `SqlToRelConverterTest`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] chunweilei commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
chunweilei commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469059693



##########
File path: core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
##########
@@ -10287,13 +10287,41 @@ private void checkCustomColumnResolving(String table) {
   }
 
   @Test public void testTumbleTableFunction() {
-    sql("select * from table(\n"
-        + "^tumble(table orders, descriptor(rowtime))^)")
-        .fails("Invalid number of arguments to function 'TUMBLE'. Was expecting 3 arguments");
     sql("select rowtime, productid, orderid, 'window_start', 'window_end' from table(\n"
         + "tumble(table orders, descriptor(rowtime), interval '2' hour))").ok();
     sql("select rowtime, productid, orderid, 'window_start', 'window_end' from table(\n"
         + "tumble(table orders, descriptor(rowtime), interval '2' hour, interval '1' hour))").ok();
+    // test named params.
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "data => table orders,\n"
+        + "timecol => descriptor(rowtime),\n"
+        + "size => interval '2' hour))").ok();
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "data => table orders,\n"
+        + "timecol => descriptor(rowtime),\n"
+        + "size => interval '2' hour,\n"
+        + "\"OFFSET\" => interval '1' hour))").ok();
+    // negative tests.
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "^\"data\"^ => table orders,\n"
+        + "TIMECOL => descriptor(rowtime),\n"
+        + "SIZE => interval '2' hour))")
+        .fails("Param 'data' not found in function 'TUMBLE'; did you mean 'DATA'\\?");

Review comment:
       Should we add some tests about it? (Ignore it if it already has)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] chunweilei commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
chunweilei commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469029610



##########
File path: core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
##########
@@ -10287,13 +10287,41 @@ private void checkCustomColumnResolving(String table) {
   }
 
   @Test public void testTumbleTableFunction() {
-    sql("select * from table(\n"
-        + "^tumble(table orders, descriptor(rowtime))^)")
-        .fails("Invalid number of arguments to function 'TUMBLE'. Was expecting 3 arguments");
     sql("select rowtime, productid, orderid, 'window_start', 'window_end' from table(\n"
         + "tumble(table orders, descriptor(rowtime), interval '2' hour))").ok();
     sql("select rowtime, productid, orderid, 'window_start', 'window_end' from table(\n"
         + "tumble(table orders, descriptor(rowtime), interval '2' hour, interval '1' hour))").ok();
+    // test named params.
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "data => table orders,\n"
+        + "timecol => descriptor(rowtime),\n"
+        + "size => interval '2' hour))").ok();
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "data => table orders,\n"
+        + "timecol => descriptor(rowtime),\n"
+        + "size => interval '2' hour,\n"
+        + "\"OFFSET\" => interval '1' hour))").ok();
+    // negative tests.
+    sql("select rowtime, productid, orderid, 'window_start', 'window_end'\n"
+        + "from table(\n"
+        + "tumble(\n"
+        + "^\"data\"^ => table orders,\n"
+        + "TIMECOL => descriptor(rowtime),\n"
+        + "SIZE => interval '2' hour))")
+        .fails("Param 'data' not found in function 'TUMBLE'; did you mean 'DATA'\\?");

Review comment:
       does the order of parameters matter?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [calcite] amaliujia commented on a change in pull request #2103: [CALCITE-4171] Support named parameters for table window functions

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2103:
URL: https://github.com/apache/calcite/pull/2103#discussion_r469610312



##########
File path: core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
##########
@@ -98,8 +173,8 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
     return typeFactory.builder()
         .kind(inputRowType.getStructKind())
         .addAll(inputRowType.getFieldList())
-        .add("window_start", timestampType)
-        .add("window_end", timestampType)
+        .add("wstart", timestampType)
+        .add("wend", timestampType)

Review comment:
       Note that I asked one of the authors of the "one sql to rule them all" paper. The only reason that they use wstart and wend was because of the layout for a paper. I got suggestion to use window_start and window_end.
   
   I also believe in a JIRA or an email (sorry I cannot recall which one it is) we agreed to switch to window_start and window_end.
   
   So I will suggest we keep window_start and window_end, which can be understood easier.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org