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/01/08 02:22:40 UTC

[GitHub] [calcite] ritesh-kapoor opened a new pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

ritesh-kapoor opened a new pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733
 
 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r391718119
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
 ##########
 @@ -271,7 +271,7 @@ protected static Connection getRemoteConnection() throws SQLException {
   @Test public void testRemoteTypeInfo() throws Exception {
     CalciteAssert.hr().with(REMOTE_CONNECTION_FACTORY)
         .metaData(GET_TYPEINFO)
-        .returns(CalciteAssert.checkResultCount(is(45)));
+        .returns(CalciteAssert.checkResultCount(is(46)));
   }
 
 Review comment:
   As LAMBDA Type is added, count is increased by 1 :)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r396054101
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/sql/fun/SqlLambdaOperator.java
 ##########
 @@ -0,0 +1,115 @@
+/*
+ * 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.calcite.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLambda;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+public class SqlLambdaOperator extends SqlOperator {
+
+  public static final SqlLambdaOperator INSTANCE = new SqlLambdaOperator();
+
+  //~ Instance fields --------------------------------------------------------
+
+  //~ Constructors -----------------------------------------------------------
+
+  /**
+   * Creates a SqlLambdaOperator.
+   */
+  SqlLambdaOperator() {
+    super(
+        "->",
+        SqlKind.LAMBDA,
+        32,
+        false,
+        ReturnTypes.ARG0_NULLABLE,
+        InferTypes.RETURN_TYPE,
+        OperandTypes.VARIADIC);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  public SqlOperandCountRange getOperandCountRange() {
+    return SqlOperandCountRanges.any();
+  }
+
+  @Override public SqlSyntax getSyntax() {
+    return SqlSyntax.SPECIAL;
+  }
+
+  public boolean checkOperandTypes(
+      SqlCallBinding callBinding,
+      boolean throwOnFailure) {
+    return true;
+  }
+
+  public void unparse(
+      SqlWriter writer,
+      SqlCall call,
+      int leftPrec,
+      int rightPrec) {
+    final SqlWriter.Frame frame = writer.startList("", "");
+    call.operand(0).unparse(writer, getLeftPrec(), getRightPrec());
+    writer.sep(getName());
+
+    call.operand(1).unparse(writer, getLeftPrec(), getRightPrec());
+    writer.endList(frame);
+  }
+
+  @Override public void validateCall(SqlCall call, SqlValidator validator,
+      SqlValidatorScope scope,
+      SqlValidatorScope operandScope) {
+    validateOperands(validator, scope, call);
+  }
+
+  public SqlCall createCall(
+      SqlLiteral functionQualifier,
+      SqlParserPos pos,
+      SqlNode... operands) {
+    assert functionQualifier == null;
+    assert operands.length == 2;
+    return new SqlLambda(pos, (SqlNodeList) operands[0], operands[1]);
+  }
+
+  @Override public RelDataType deriveType(
+      SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
+    return validateOperands(validator, scope, call);
 
 Review comment:
   Should implement the `checkOperandTypes` correctly to deduce the type, you can take `SqlCaseOperator` for a reference.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r401332555
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/sql/validate/LambdaScope.java
 ##########
 @@ -0,0 +1,131 @@
+/*
+ * 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.calcite.sql.validate;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.StructKind;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambda;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The name-resolution context for expression inside a LAMBDA clause. The objects visible are the
+ * lambda paramters, and those inherited from the parent scope.
+ *
+ * <p>Consider "SELECT lambdaExp((x)-&gt;A.a+x) FROM A
+ * resolved in the lambda scope for "Lambda Parameters and Table columns of A"</p>
+ */
+public class LambdaScope extends ListScope {
+
 
 Review comment:
   ParameterScope has a limited scope where as LambdaScope can access parent scope as well. It didn't seem right to me.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r401331638
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
 ##########
 @@ -304,6 +305,17 @@ private SqlLibraryOperators() {
           OperandTypes.CHARACTER,
           SqlFunctionCategory.STRING);
 
+  @LibraryOperator(libraries = {MYSQL})
+  public static final SqlFunction MAP_FILTER =
+      new SqlFunction("MAP_FILTER",
+          SqlKind.OTHER_FUNCTION,
+          ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NULLABLE),
+          null,
+          OperandTypes.sequence("MAP_FILTER(<ANY>, <LAMBDA(BOOLEAN, ANY, ANY)>)",
+              OperandTypes.family(SqlTypeFamily.MAP),
+              OperandTypes.lambda(SqlTypeFamily.BOOLEAN, SqlTypeFamily.ANY, SqlTypeFamily.ANY)),
+          SqlFunctionCategory.SYSTEM);
 
 Review comment:
   The Map Function accepts lambda which returns boolean value and takes two arguments of any type. This seems to be correct.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r364040906
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/sql/fun/SqlLambda.java
 ##########
 @@ -0,0 +1,114 @@
+/*
+ * 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.calcite.sql.fun;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlWith;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.UnmodifiableArrayList;
+
+import java.util.List;
+
+/**
+ * A <code>SqlLambda</code> is a node of a parse tree which represents a lambda statement. It warrants
+ * its own node type just because we have a lot of methods to put somewhere.
+ */
+public class SqlLambda extends SqlCall {
+
+  SqlNodeList parameters;
+  SqlNode expression;
+
+  //~ Constructors -----------------------------------------------------------
+
+
+  public SqlLambda(SqlParserPos pos, SqlNodeList parameters,
+      SqlNode expression) {
+    super(pos);
+    this.parameters = parameters;
+    this.expression = expression;
+  }
+
+  /**
+   * Creates a call to the lambda operator
+   *
+   * <blockquote><code>LAMBDA<br>
+   */
+  public static SqlLambda createLambda(SqlParserPos pos, List<SqlNode> parameters,
+      SqlNode expression) {
+    SqlNodeList nodes = new SqlNodeList(parameters, pos);
+    return new SqlLambda(pos, nodes, expression);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override
+  public SqlKind getKind() {
+    return SqlKind.LAMBDA;
+  }
+
+  public SqlOperator getOperator() {
+    return SqlStdOperatorTable.LAMBDA;
+  }
+
+  public List<SqlNode> getOperandList() {
+    return UnmodifiableArrayList.of(parameters, expression);
+  }
+
+  @Override
+  public void setOperand(int i, SqlNode operand) {
+    switch (i) {
+    case 0:
+      parameters = (SqlNodeList) operand;
+      break;
+    case 1:
+      expression = operand;
+      break;
+    default:
+      throw new AssertionError(i);
+    }
+  }
+
+  public SqlNodeList getParameters() {
+    return parameters;
+  }
+
+  public SqlNode getExpression() {
+    return expression;
+  }
+
+  @Override
+  public void validate(SqlValidator validator,
+      SqlValidatorScope scope) {
+    final RelDataTypeFactory typeFactory = validator.getTypeFactory();
+    final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
 
 Review comment:
   Type information resolution is still pending

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#issuecomment-602002920
 
 
   > > @danny0405 Need you favour for PR review, I know its hard for you to review so many PR's. It would be really nice if you could help me with this one. :)
   > 
   > Would review this weekend ~ There are some conflicts, you can resolve them first.
   
   I have resolved conflicts, Thanks for taking out time. :)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 edited a comment on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 edited a comment on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#issuecomment-599427677
 
 
   > @danny0405 Need you favour for PR review, I know its hard for you to review so many PR's. It would be really nice if you could help me with this one. :)
   
   Would review this weekend ~ There are some conflicts, you can resolve them first.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r401334294
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
 ##########
 @@ -687,6 +690,10 @@ private Expression translate0(RexNode expr, RexImpTable.NullAs nullAs,
       Expression input = list.append("inp" + index + "_", x); // safe to share
       return handleNullUnboxingIfNecessary(input, nullAs, storageType);
     }
+    case LAMBDA_REF: {
+      final String name = ((RexLambdaRef) expr).getName();
+      return Expressions.parameter(typeFactory.getJavaClass(expr.getType()), name);
 
 Review comment:
   This is important and there are test failures 
   `FAILURE  49.5sec,  277 completed,   1 failed,   1 skipped, org.apache.calcite.test.CalciteSqlOperatorTest
   FAILURE  83.9sec, 5925 completed,   3 failed, 104 skipped, Gradle Test Run :core:test`

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#issuecomment-599427677
 
 
   > @danny0405 Need you favour for PR review, I know its hard for you to review so many PR's. It would be really nice if you could help me with this one. :)
   
   Would review this weekend ~

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r391785810
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
 ##########
 @@ -4559,6 +4559,46 @@ private void checkNullOperand(SqlTester tester, String op) {
         });
   }
 
+  @Test public void testLambda() {
+    tester.checkType("(a)->2+2*a", "LAMBDA NOT NULL");
+    tester.checkType("(a,b)->a+b", "LAMBDA NOT NULL");
+    tester.checkType("(a,b,c)->2*a+b+c", "LAMBDA NOT NULL");
+  }
+
+  @Test public void testMapFilterFunc() {
+    final SqlTester tester = tester(SqlLibrary.MYSQL);
 
 Review comment:
   Added few more test cases

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] chunweilei commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
chunweilei commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r391384551
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
 ##########
 @@ -4559,6 +4559,46 @@ private void checkNullOperand(SqlTester tester, String op) {
         });
   }
 
+  @Test public void testLambda() {
+    tester.checkType("(a)->2+2*a", "LAMBDA NOT NULL");
+    tester.checkType("(a,b)->a+b", "LAMBDA NOT NULL");
+    tester.checkType("(a,b,c)->2*a+b+c", "LAMBDA NOT NULL");
+  }
+
+  @Test public void testMapFilterFunc() {
+    final SqlTester tester = tester(SqlLibrary.MYSQL);
 
 Review comment:
   I think we should also add some negative tests. For instance, `tester.checkType("(b)->2+2*a")`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#issuecomment-571862529
 
 
   This PR is incomplete in terms of functionality and requires alot of refactoring and test cases addition.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r369161604
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
 ##########
 @@ -4559,7 +4559,17 @@ private void checkNullOperand(SqlTester tester, String op) {
         });
   }
 
-  @Test public void testRegexpReplaceFunc() {
+  @Test
+  public void testLambda() {
+//    tester.setFor(SqlStdOperatorTable.LAMBDA);
+    final SqlTester tester = tester(SqlLibrary.MYSQL);
+    tester.checkType("(a)->2+2*a", "LAMBDA NOT NULL");
+//    tester.checkString("map_filter((a)->a*2+2)","4","VARCHAR(2000)");
+    tester.checkString("map_filter((a)->2+2*a)", "8", "INTEGER");
 
 Review comment:
   Done

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#issuecomment-597737738
 
 
   @danny0405 Need you favour for PR review, I know its hard for you to review so many PR's. It would be really nice if you could help me with this one. :)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r364041136
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
 ##########
 @@ -4559,7 +4559,17 @@ private void checkNullOperand(SqlTester tester, String op) {
         });
   }
 
-  @Test public void testRegexpReplaceFunc() {
+  @Test
+  public void testLambda() {
+//    tester.setFor(SqlStdOperatorTable.LAMBDA);
+    final SqlTester tester = tester(SqlLibrary.MYSQL);
+    tester.checkType("(a)->2+2*a", "LAMBDA NOT NULL");
+//    tester.checkString("map_filter((a)->a*2+2)","4","VARCHAR(2000)");
+    tester.checkString("map_filter((a)->2+2*a)", "8", "INTEGER");
 
 Review comment:
   Was able to get map_filter(()->2+2) this working

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#issuecomment-572448182
 
 
   n interesting topic, would review if i have time ~

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r396054045
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/sql/SqlLambda.java
 ##########
 @@ -0,0 +1,88 @@
+/*
+ * 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.calcite.sql;
+
+import org.apache.calcite.sql.fun.SqlLambdaOperator;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.UnmodifiableArrayList;
+
+import java.util.List;
+
+/**
+ * A <code>SqlLambda</code> is a node of a parse tree which represents a lambda statement. It
+ * warrants * its own node type just because we have a lot of methods to put somewhere.
+ */
+public class SqlLambda extends SqlCall {
+
+  SqlNodeList parameters;
+  SqlNode expression;
+
+  //~ Constructors -----------------------------------------------------------
+
+
+  public SqlLambda(SqlParserPos pos, SqlNodeList parameters,
+      SqlNode expression) {
+    super(pos);
+    this.parameters = parameters;
+    this.expression = expression;
+  }
+
+  /**
+   * Creates a call to the lambda operator
+   *
+   * <br>LAMBDA<br>
+   */
+  public static SqlLambda createLambda(SqlParserPos pos, SqlNodeList parameters,
+      SqlNode expression) {
+    return new SqlLambda(pos, parameters, expression);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public SqlKind getKind() {
+    return SqlKind.LAMBDA;
+  }
+
+  public SqlOperator getOperator() {
+    return SqlLambdaOperator.INSTANCE;
+  }
+
+  public List<SqlNode> getOperandList() {
+    return UnmodifiableArrayList.of(parameters, expression);
+  }
+
+  @Override public void setOperand(int i, SqlNode operand) {
+    switch (i) {
+    case 0:
+      parameters = (SqlNodeList) operand;
+      break;
+    case 1:
+      expression = operand;
+      break;
+    default:
+      throw new AssertionError(i);
+    }
+  }
+
+  public SqlNodeList getParameters() {
+    return parameters;
+  }
+
 
 Review comment:
   Unparse the `SqlLambda` correctly.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r396053657
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/sql/validate/LambdaScope.java
 ##########
 @@ -0,0 +1,131 @@
+/*
+ * 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.calcite.sql.validate;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.StructKind;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambda;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The name-resolution context for expression inside a LAMBDA clause. The objects visible are the
+ * lambda paramters, and those inherited from the parent scope.
+ *
+ * <p>Consider "SELECT lambdaExp((x)-&gt;A.a+x) FROM A
+ * resolved in the lambda scope for "Lambda Parameters and Table columns of A"</p>
+ */
+public class LambdaScope extends ListScope {
+
 
 Review comment:
   Maybe we can extend the `ParameterScope` to support parent scope instead a new `LambdaScope`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#issuecomment-578039045
 
 
   Need help in PR review :)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r368268897
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
 ##########
 @@ -327,6 +328,7 @@
     defineMethod(SOUNDEX, BuiltInMethod.SOUNDEX.method, NullPolicy.STRICT);
     defineMethod(DIFFERENCE, BuiltInMethod.DIFFERENCE.method, NullPolicy.STRICT);
     defineMethod(REVERSE, BuiltInMethod.REVERSE.method, NullPolicy.STRICT);
+    defineMethod(MAP_FILTER, BuiltInMethod.MAP_FILTER.method, NullPolicy.STRICT);
 
 Review comment:
   Done

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 edited a comment on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 edited a comment on issue #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#issuecomment-572448182
 
 
   An interesting topic, would review if i have time ~

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r364041298
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
 ##########
 @@ -327,6 +328,7 @@
     defineMethod(SOUNDEX, BuiltInMethod.SOUNDEX.method, NullPolicy.STRICT);
     defineMethod(DIFFERENCE, BuiltInMethod.DIFFERENCE.method, NullPolicy.STRICT);
     defineMethod(REVERSE, BuiltInMethod.REVERSE.method, NullPolicy.STRICT);
+    defineMethod(MAP_FILTER, BuiltInMethod.MAP_FILTER.method, NullPolicy.STRICT);
 
 Review comment:
   Dummy function to test lambda functionality, actual implementation could be moved to other PR

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] chunweilei commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
chunweilei commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r392756969
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
 ##########
 @@ -271,7 +271,7 @@ protected static Connection getRemoteConnection() throws SQLException {
   @Test public void testRemoteTypeInfo() throws Exception {
     CalciteAssert.hr().with(REMOTE_CONNECTION_FACTORY)
         .metaData(GET_TYPEINFO)
-        .returns(CalciteAssert.checkResultCount(is(45)));
+        .returns(CalciteAssert.checkResultCount(is(46)));
   }
 
 Review comment:
   Got it. 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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r396053902
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/rex/RexLambdaRef.java
 ##########
 @@ -0,0 +1,120 @@
+/*
+ * 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.calcite.rex;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.Pair;
+
+import java.util.List;
+
+/**
+ * Variable which references a parameter in lambda expression.
+ *
+ * <p>Fields of the input are 0-based. If there is more than one input, they are
+ * numbered consecutively. For example</p>
+ *
+ * (a,b)-&gt;{expression}
+ *
+ * <p>then the fields are:</p>
+ *
+ * <ul>
+ * <li>Field #0: a</li>
+ * <li>Field #1: b</li>
+ * </ul>
+ *
+ * <p>So <code>RexLambdaRef(0, Integer)</code> is the correct reference for the
+ * field a.</p>
+ */
+public class RexLambdaRef extends RexSlot {
+  //~ Static fields/initializers ---------------------------------------------
 
 Review comment:
   Isn't is just a `RexLocalRef` ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r401335835
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/rex/RexLambdaRef.java
 ##########
 @@ -0,0 +1,120 @@
+/*
+ * 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.calcite.rex;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.Pair;
+
+import java.util.List;
+
+/**
+ * Variable which references a parameter in lambda expression.
+ *
+ * <p>Fields of the input are 0-based. If there is more than one input, they are
+ * numbered consecutively. For example</p>
+ *
+ * (a,b)-&gt;{expression}
+ *
+ * <p>then the fields are:</p>
+ *
+ * <ul>
+ * <li>Field #0: a</li>
+ * <li>Field #1: b</li>
+ * </ul>
+ *
+ * <p>So <code>RexLambdaRef(0, Integer)</code> is the correct reference for the
+ * field a.</p>
+ */
+public class RexLambdaRef extends RexSlot {
+  //~ Static fields/initializers ---------------------------------------------
 
 Review comment:
   This is used to refer lambda ref, appears to be similar but are different

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r396054141
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
 ##########
 @@ -304,6 +305,17 @@ private SqlLibraryOperators() {
           OperandTypes.CHARACTER,
           SqlFunctionCategory.STRING);
 
+  @LibraryOperator(libraries = {MYSQL})
+  public static final SqlFunction MAP_FILTER =
+      new SqlFunction("MAP_FILTER",
+          SqlKind.OTHER_FUNCTION,
+          ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NULLABLE),
+          null,
+          OperandTypes.sequence("MAP_FILTER(<ANY>, <LAMBDA(BOOLEAN, ANY, ANY)>)",
+              OperandTypes.family(SqlTypeFamily.MAP),
+              OperandTypes.lambda(SqlTypeFamily.BOOLEAN, SqlTypeFamily.ANY, SqlTypeFamily.ANY)),
+          SqlFunctionCategory.SYSTEM);
 
 Review comment:
   Shouldn't the operand type check be `OperandTypes.family(SqlTypeFamily.MAP, SqlTypeFamily.LAMBDA)` ?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r396053783
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
 ##########
 @@ -687,6 +690,10 @@ private Expression translate0(RexNode expr, RexImpTable.NullAs nullAs,
       Expression input = list.append("inp" + index + "_", x); // safe to share
       return handleNullUnboxingIfNecessary(input, nullAs, storageType);
     }
+    case LAMBDA_REF: {
+      final String name = ((RexLambdaRef) expr).getName();
+      return Expressions.parameter(typeFactory.getJavaClass(expr.getType()), name);
 
 Review comment:
   I remove this and the tests still passes, so, it seems not necessary.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
ritesh-kapoor commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r401334294
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
 ##########
 @@ -687,6 +690,10 @@ private Expression translate0(RexNode expr, RexImpTable.NullAs nullAs,
       Expression input = list.append("inp" + index + "_", x); // safe to share
       return handleNullUnboxingIfNecessary(input, nullAs, storageType);
     }
+    case LAMBDA_REF: {
+      final String name = ((RexLambdaRef) expr).getName();
+      return Expressions.parameter(typeFactory.getJavaClass(expr.getType()), name);
 
 Review comment:
   This is important and there are test failures `FAILURE  49.5sec,  277 completed,   1 failed,   1 skipped, org.apache.calcite.test.CalciteSqlOperatorTest
   FAILURE  83.9sec, 5925 completed,   3 failed, 104 skipped, Gradle Test Run :core:test`

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r396053715
 
 

 ##########
 File path: core/src/main/java/org/apache/calcite/sql/type/LambdaOperandTypeChecker.java
 ##########
 @@ -0,0 +1,104 @@
+/*
+ * 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.calcite.sql.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlLambda;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.validate.LambdaScope;
+
+import java.util.List;
+
+/**
+ * Lambda type-checking strategy where lambda return type is checked.
+ */
+public class LambdaOperandTypeChecker implements SqlSingleOperandTypeChecker {
+  //~ Instance fields --------------------------------------------------------
+
+  private List<SqlTypeFamily> argFamilies;
+
+  //~ Constructors -----------------------------------------------------------
+
+  /**
+   * Package private. Create using {@link OperandTypes#family}.
+   */
+  LambdaOperandTypeChecker(List<SqlTypeFamily> argFamilies) {
+    this.argFamilies = argFamilies;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+      boolean throwOnFailure) {
+    return checkSingleOperandType(callBinding, callBinding.getCall(), 0, throwOnFailure);
+  }
+
+  @Override public SqlOperandCountRange getOperandCountRange() {
+    return SqlOperandCountRanges.of(1);
+  }
+
+  @Override public String getAllowedSignatures(SqlOperator op, String opName) {
+    return SqlUtil.getAliasedSignature(op, opName, argFamilies);
+  }
+
+  @Override public Consistency getConsistency() {
+    return Consistency.NONE;
+  }
+
+  @Override public boolean isOptional(int i) {
+    return false;
+  }
+
+  @Override public boolean checkSingleOperandType(SqlCallBinding callBinding,
+      SqlNode node, int iFormalOperand, boolean throwOnFailure) {
+
+    assert node instanceof SqlLambda;
+    SqlLambda sqlLambda = (SqlLambda) node;
+    LambdaScope lambdaScope = (LambdaScope) callBinding.getValidator().getLambdaScope(node);
+
+    if (sqlLambda.getParameters().size() != argFamilies.size() - 1) {
+      if (throwOnFailure) {
+        throw callBinding.newValidationSignatureError();
+      }
+      return false;
+    }
+
+    for (int i = 1; i < argFamilies.size(); i++) {
+      lambdaScope.setParameterType(i - 1, argFamilies.get(i));
+      RelDataType type = callBinding.getValidator().getTypeFactory().createSqlType(SqlTypeName.ANY);
+      if (argFamilies.get(i) != SqlTypeFamily.ANY) {
+        type = argFamilies.get(i)
+            .getDefaultConcreteType(callBinding.getTypeFactory());
+      }
 
 Review comment:
   The type inference should happen in `SqlLambdaOperator`, not the type checker(only check validated node types).

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [calcite] chunweilei commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries

Posted by GitBox <gi...@apache.org>.
chunweilei commented on a change in pull request #1733: [CALCITE-3679] Allow lambda expressions in SQL queries
URL: https://github.com/apache/calcite/pull/1733#discussion_r391386026
 
 

 ##########
 File path: core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
 ##########
 @@ -271,7 +271,7 @@ protected static Connection getRemoteConnection() throws SQLException {
   @Test public void testRemoteTypeInfo() throws Exception {
     CalciteAssert.hr().with(REMOTE_CONNECTION_FACTORY)
         .metaData(GET_TYPEINFO)
-        .returns(CalciteAssert.checkResultCount(is(45)));
+        .returns(CalciteAssert.checkResultCount(is(46)));
   }
 
 Review comment:
   Why change this number?

----------------------------------------------------------------
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


With regards,
Apache Git Services