You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by "macroguo-ghy (via GitHub)" <gi...@apache.org> on 2023/11/04 07:03:33 UTC

[PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

macroguo-ghy opened a new pull request, #3502:
URL: https://github.com/apache/calcite/pull/3502

   This PR is about three things:
   
   1. Implementing parsing, validating, and converting SqlNode to RelNode, and RelNode to SqlNode for lambda expressions.
   2. Implementing the conversion of lambda expression RexNode to Linq4j Expression to execute higher order functions.
   3. Adding the EXISTS function, which is enabled in the Spark library.


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1853383672

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [15 New issues](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&inNewCodePeriod=true)  
   [71.2% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "macroguo-ghy (via GitHub)" <gi...@apache.org>.
macroguo-ghy commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1807928275

   Hi @mihaibudiu, thanks for your review. I have made some commits to address your feedback:
   - Add docs about lambda expressions and higher-order functions. 
   - Add some negative tests in `SqlParserTest`.
   - Log a new jira case [CALCITE-6116](https://issues.apache.org/jira/browse/CALCITE-6116) to track the EXISTS function.


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1794430920

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL)
   
   [![83.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.5%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list) [83.5% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "julianhyde (via GitHub)" <gi...@apache.org>.
julianhyde commented on code in PR #3502:
URL: https://github.com/apache/calcite/pull/3502#discussion_r1393156745


##########
core/src/main/java/org/apache/calcite/rex/RexLambdaRef.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.sql.SqlKind;
+
+/**
+ * Variable which references a field of a lambda expression.

Review Comment:
   s/which/that/



##########
core/src/main/java/org/apache/calcite/sql/fun/SqlLambdaExpressionOperator.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.SqlKind;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.validate.SqlLambdaExpressionScope;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+import java.util.List;
+
+import static com.google.common.collect.ImmutableList.toImmutableList;
+
+/**
+ * The {@code SqlLambdaExpressionOperator} represents a lambda expression.
+ * The syntax :
+ * {@code IDENTIFIER -> EXPRESSION} or {@code (IDENTIFIER, IDENTIFIER, ...) -> EXPRESSION}.
+ */
+public class SqlLambdaExpressionOperator extends SqlSpecialOperator {

Review Comment:
   Make this a private inner class of `SqlLambdaExpression`.



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionOperandTypeChecker.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlLambdaExpressionScope;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Operand type-checking strategy where the type of the operand is a lambda
+ * expression with a given return type and argument types.
+ */
+public class LambdaExpressionOperandTypeChecker implements SqlSingleOperandTypeChecker {

Review Comment:
   I don't think this should be public. That will prevent refactoring. Outside the package people should just use a method in `OperandTypes`.



##########
core/src/main/codegen/templates/Parser.jj:
##########
@@ -3912,6 +3919,29 @@ SqlNode Expression3(ExprContext exprContext) :
     }
 }
 
+/**
+ * Parses a lambda expression.
+ */
+SqlNode LambdaExpression() :
+{
+    final SqlNodeList parameters;
+    final SqlNode expression;
+    final Span s;
+}
+{
+    (
+        LOOKAHEAD(2)
+        <LPAREN> <RPAREN> { parameters = SqlNodeList.EMPTY; }
+    |
+        parameters = SimpleIdentifierOrList()

Review Comment:
   should there be a rule that allows `()`, `x` and `(x [, y]*)`?



##########
core/src/main/codegen/templates/Parser.jj:
##########
@@ -8787,6 +8817,7 @@ void NonReservedKeyWord2of3() :
 |   < NE2: "!=" >
 |   < PLUS: "+" >
 |   < MINUS: "-" >
+|   < LAMBDA_OPERATOR: "->" >

Review Comment:
   just `LAMBDA`



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaExpression.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.sql.SqlKind;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Represents a lambda expression.
+ */
+public class RexLambdaExpression extends RexNode {
+  //~ Instance fields --------------------------------------------------------
+
+  private final List<RexLambdaRef> parameters;
+  private final RexNode expression;
+
+  //~ Constructors -----------------------------------------------------------
+
+  RexLambdaExpression(List<RexLambdaRef> parameters, RexNode expression) {
+    this.expression = expression;
+    this.parameters = parameters;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public RelDataType getType() {
+    return expression.getType();
+  }
+
+  @Override public SqlKind getKind() {
+    return SqlKind.LAMBDA;
+  }
+
+  @Override public <R> R accept(RexVisitor<R> visitor) {
+    return visitor.visitLambda(this);
+  }
+
+  @Override public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitLambda(this, arg);
+  }
+
+  public RexNode getExpression() {
+    return expression;
+  }
+
+  public List<RexLambdaRef> getParameters() {
+    return parameters;
+  }
+
+  @Override public boolean equals(@Nullable Object o) {
+    if (this == o) {

Review Comment:
   can you use the compact form 'this == o || o instanceof RexLambdaExpression & ...'?
   
   no need for Objects.equals. use expression.equals because it's not-null.



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionOperandTypeChecker.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlLambdaExpressionScope;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Operand type-checking strategy where the type of the operand is a lambda
+ * expression with a given return type and argument types.
+ */
+public class LambdaExpressionOperandTypeChecker implements SqlSingleOperandTypeChecker {
+
+  private final SqlTypeFamily returnTypeFamily;
+  private final List<SqlTypeFamily> argFamilies;
+
+  public LambdaExpressionOperandTypeChecker(
+      SqlTypeFamily returnTypeFamily,
+      List<SqlTypeFamily> argFamilies) {
+    this.returnTypeFamily = returnTypeFamily;
+    this.argFamilies = argFamilies;
+  }
+
+  @Override public String getAllowedSignatures(SqlOperator op, String opName) {
+    ImmutableList.Builder<SqlTypeFamily> builder = ImmutableList.builder();
+    builder.addAll(argFamilies);
+    builder.add(returnTypeFamily);
+
+    return SqlUtil.getAliasedSignature(op, opName, builder.build());
+  }
+
+  @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+      boolean throwOnFailure) {
+    return false;
+  }
+
+  @Override public boolean checkSingleOperandType(
+      SqlCallBinding callBinding, SqlNode operand, int iFormalOperand, boolean throwOnFailure) {
+    if (!(operand instanceof SqlLambdaExpression)
+        || ((SqlLambdaExpression) operand).getParameters().size() != argFamilies.size()) {
+      if (throwOnFailure) {
+        throw callBinding.newValidationSignatureError();
+      }
+      return false;
+    }
+    SqlLambdaExpression lambdaExpr = (SqlLambdaExpression) operand;
+
+    if (lambdaExpr.getParameters().isEmpty()
+        || argFamilies.stream().allMatch(f -> f == SqlTypeFamily.ANY)
+        || returnTypeFamily == SqlTypeFamily.ANY) {
+      return true;
+    }
+
+    if (SqlUtil.isNullLiteral(lambdaExpr.getExpression(), false)) {
+      if (callBinding.isTypeCoercionEnabled()) {
+        return true;
+      } else if (throwOnFailure) {
+        throw callBinding.getValidator().newValidationError(lambdaExpr.getExpression(),
+            RESOURCE.nullIllegal());
+      } else {
+        return false;
+      }
+    }
+
+    // Replace the parameter types in the lambda expression.
+    final SqlValidator validator = callBinding.getValidator();
+    SqlLambdaExpressionScope scope =
+        (SqlLambdaExpressionScope) validator.getLambdaExpressionScope(lambdaExpr);
+    for (int i = 0; i < argFamilies.size(); i++) {
+      SqlNode param = lambdaExpr.getParameters().get(i);
+      RelDataType type = argFamilies.get(i).getDefaultConcreteType(callBinding.getTypeFactory());
+      if (type != null) {
+        scope.getParameterTypes().put(param.toString(), type);
+      }
+    }
+    lambdaExpr.accept(new TypeRemover(validator));
+
+    // Given the new relDataType, re-validate the lambda expression.
+    validator.validateLambdaExpression(lambdaExpr);
+    final RelDataType newType = validator.getValidatedNodeType(lambdaExpr);
+    assert newType instanceof LambdaExpressionSqlType;
+    final SqlTypeName returnTypeName =
+        ((LambdaExpressionSqlType) newType).getReturnType().getSqlTypeName();
+    if (returnTypeName == SqlTypeName.ANY
+        || returnTypeFamily.getTypeNames().contains(returnTypeName)) {
+      return true;
+    }
+
+    if (throwOnFailure) {
+      throw callBinding.newValidationSignatureError();
+    }
+    return false;
+  }
+
+  /**

Review Comment:
   can you explain why this is needed?



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionOperandTypeChecker.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlLambdaExpressionScope;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Operand type-checking strategy where the type of the operand is a lambda
+ * expression with a given return type and argument types.
+ */
+public class LambdaExpressionOperandTypeChecker implements SqlSingleOperandTypeChecker {
+
+  private final SqlTypeFamily returnTypeFamily;
+  private final List<SqlTypeFamily> argFamilies;
+
+  public LambdaExpressionOperandTypeChecker(
+      SqlTypeFamily returnTypeFamily,
+      List<SqlTypeFamily> argFamilies) {
+    this.returnTypeFamily = returnTypeFamily;

Review Comment:
   check nulls and copy to immutable list



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaExpression.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.sql.SqlKind;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Represents a lambda expression.
+ */
+public class RexLambdaExpression extends RexNode {
+  //~ Instance fields --------------------------------------------------------
+
+  private final List<RexLambdaRef> parameters;
+  private final RexNode expression;
+
+  //~ Constructors -----------------------------------------------------------
+
+  RexLambdaExpression(List<RexLambdaRef> parameters, RexNode expression) {
+    this.expression = expression;

Review Comment:
   ImmutbleList.copyOf(parameters)
   requireNonNull(expression)



##########
core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java:
##########
@@ -1176,6 +1176,15 @@ private static RelDataType arrayAppendPrependReturnType(SqlOperatorBinding opBin
           SqlLibraryOperators::arrayAppendPrependReturnType,
           OperandTypes.ARRAY_ELEMENT);
 
+  /** The "EXISTS(array, function1)" function (Spark); Returns whether a predicate holds

Review Comment:
   the parameter isn't a `function1`, it's a lambda; there is no function type in SQL
   
   grammar: shouldn't have capital 'R' after ';'.



##########
core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java:
##########
@@ -5371,6 +5372,28 @@ public static List sortArray(List list, boolean ascending) {
     return list;
   }
 
+  /** Support the EXISTS(list, function1) function. */

Review Comment:
   Are you aware of the `nullableExists` function?



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaRef.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.sql.SqlKind;
+
+/**
+ * Variable which references a field of a lambda expression.
+ */
+public class RexLambdaRef extends RexInputRef {
+
+  private final String paramName;

Review Comment:
   could this be an ordinal?



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaRef.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.sql.SqlKind;
+
+/**
+ * Variable which references a field of a lambda expression.
+ */
+public class RexLambdaRef extends RexInputRef {

Review Comment:
   Extend RexSlot instead? Use the same strategy for names as RexLocalRef does.



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaExpression.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.sql.SqlKind;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Represents a lambda expression.
+ */
+public class RexLambdaExpression extends RexNode {

Review Comment:
   rename RexLambdaExpression to RexLambda. give example.



##########
core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java:
##########
@@ -77,6 +77,7 @@ public enum SqlTypeFamily implements RelDataTypeFamily {
   CURSOR,
   COLUMN_LIST,
   GEO,
+  LAMBDA_EXPRESSION,

Review Comment:
   change to FUNCTION



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionSqlType.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataTypeField;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * SQL lambda expression type.
+ */
+public class LambdaExpressionSqlType extends AbstractSqlType {
+  private final @Nullable RelDataType parameterType;
+  private final RelDataType returnType;
+
+  public LambdaExpressionSqlType(@Nullable RelDataType parameterType, RelDataType returnType) {
+    super(SqlTypeName.LAMBDA_EXPRESSION, true, null);
+    assert parameterType == null || parameterType.isStruct();

Review Comment:
   why allow null parameter type? a `PairList<String, RelDataType>` (which may be empty) seems about right.



##########
site/_docs/reference.md:
##########
@@ -1205,6 +1205,7 @@ Note:
 | MULTISET | Unordered collection that may contain duplicates | Example: int multiset
 | ARRAY    | Ordered, contiguous collection that may contain duplicates | Example: varchar(10) array
 | CURSOR   | Cursor over the result of executing a query |
+| LAMBDA_EXPRESSION| A function definition that is not bound to an identifier.| Example FUNCTION(INTEGER, VARCHAR(30)) -> INTEGER

Review Comment:
   Let's just call this a FUNCTION type. Add spaces around `|`. Remove '.'
   
   You should note that it is not fully supported e.g. in CAST or DDL



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionSqlType.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataTypeField;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * SQL lambda expression type.
+ */

Review Comment:
   can this be private?
   
   I would call this a function type, not a lambda type. In future we may allow other kinds of function value.



##########
core/src/main/java/org/apache/calcite/sql/validate/LambdaExpressionNamespace.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Namespace for {@code lambda expression}.
+ */
+public class LambdaExpressionNamespace extends AbstractNamespace {

Review Comment:
   rename to LambdaNamespace



##########
core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java:
##########
@@ -101,6 +101,25 @@ public static FamilyOperandTypeChecker family(List<SqlTypeFamily> families) {
     return family(families, i -> false);
   }
 
+  /**
+   * Creates a checker that passes if the operand is a lambda expression with

Review Comment:
   change 'lambda expression' to 'function' but give lambda expression as an example.



##########
core/src/main/java/org/apache/calcite/sql/validate/SqlLambdaExpressionScope.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Litmus;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Scope for a {@link SqlLambdaExpression LAMBDA EXPRESSION}.
+ */
+public class SqlLambdaExpressionScope extends ListScope {
+  private final SqlLambdaExpression lambdaExpr;
+  private final Map<String, RelDataType> parameterTypes;
+
+  public SqlLambdaExpressionScope(
+      SqlValidatorScope parent, SqlLambdaExpression lambdaExpr) {
+    super(parent);
+    this.lambdaExpr = lambdaExpr;
+
+    // default parameter type is ANY

Review Comment:
   default type is ANY? Sounds dubious.
   
   Can you justify this with an example? SQL is a strongly typed language.



##########
core/src/main/java/org/apache/calcite/sql/validate/SqlLambdaExpressionScope.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Litmus;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Scope for a {@link SqlLambdaExpression LAMBDA EXPRESSION}.
+ */
+public class SqlLambdaExpressionScope extends ListScope {
+  private final SqlLambdaExpression lambdaExpr;
+  private final Map<String, RelDataType> parameterTypes;

Review Comment:
   why cache the parameter types? caches go out of date. better to have a method that computes them when needed.



##########
testkit/src/main/java/org/apache/calcite/test/MockSqlOperatorTable.java:
##########
@@ -623,4 +625,24 @@ public CompositeFunction() {
       return typeFactory.createSqlType(SqlTypeName.BIGINT);
     }
   }
+
+  private static final SqlFunction HIGHER_ORDER_FUNCTION =
+      new SqlFunction("HIGHER_ORDER_FUNCTION",

Review Comment:
   Use `SqlBasicFunction.create`.



##########
core/src/main/codegen/templates/Parser.jj:
##########
@@ -1060,6 +1064,9 @@ void AddArg(List<SqlNode> list, ExprContext exprContext) :
     )
     (
         e = Default()
+    |
+        LOOKAHEAD((SimpleIdentifierOrList() | <LPAREN> <RPAREN>) <LAMBDA_OPERATOR>)
+        e = LambdaExpression()

Review Comment:
   How expensive is this LOOKAHEAD?



##########
site/_docs/reference.md:
##########
@@ -2614,6 +2615,22 @@ Note:
 | jsonValue IS JSON ARRAY           | Whether *jsonValue* is a JSON array
 | jsonValue IS NOT JSON ARRAY       | Whether *jsonValue* is not a JSON array
 
+### Higher-order Functions
+
+A higher-order function takes one or more lambda expressions as arguments.
+
+Lambda Expression Syntax:
+{% highlight sql %}
+lambdaExpression:
+      parameters '->' expression
+
+parameters:
+      '(' [ identifier [, identifier ] ] ')'
+  |   identifier
+{% endhighlight %}
+
+Higher-order functions are not included in the SQL standard, so all the functions will be listed in the next chapter.
+

Review Comment:
   Is it true that higher-order functions must be built-in?
   
   I think it would be helpful to list the higher-order functions here.



##########
core/src/main/codegen/templates/Parser.jj:
##########
@@ -1033,6 +1034,9 @@ void AddArg0(List<SqlNode> list, ExprContext exprContext) :
     )
     (
         e = Default()
+    |
+        LOOKAHEAD((SimpleIdentifierOrList() | <LPAREN> <RPAREN>) <LAMBDA_OPERATOR>)
+        e = LambdaExpression()

Review Comment:
   How expensive is this LOOKAHEAD?



##########
core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java:
##########
@@ -779,6 +782,17 @@ public SqlNode toSql(@Nullable RexProgram program, RexNode rex) {
           return SqlStdOperatorTable.NOT.createCall(POS, node);
         }
 
+      case LAMBDA:
+        final RexLambdaExpression lambda = (RexLambdaExpression) rex;
+        final SqlNodeList parameters = new SqlNodeList(POS);
+        for (RexLambdaRef parameter : lambda.getParameters()) {
+          parameters.add(toSql(program, parameter));
+        }
+        final SqlNode expression = toSql(program, lambda.getExpression());
+        return new SqlLambdaExpression(POS, parameters, expression);
+      case LAMBDA_REF:

Review Comment:
   add blank line before



##########
core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java:
##########
@@ -6683,6 +6683,31 @@ void testGroupExpressionEquivalenceParams() {
         .fails("'PERCENTILE_DISC' requires precisely one ORDER BY key");
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-3679">[CALCITE-3679]
+   * Allow lambda expressions in SQL queries</a>. */
+  @Test void testHigherOrderFunction() {
+    final SqlValidatorFixture s = fixture()
+        .withOperatorTable(MockSqlOperatorTable.standard().extend());
+    s.withSql("select HIGHER_ORDER_FUNCTION(1, (x, y) -> x + 1)").ok();
+    s.withSql("select HIGHER_ORDER_FUNCTION(1, (x, y) -> y)").ok();
+    s.withSql("select HIGHER_ORDER_FUNCTION(1, (x, y) -> char_length(x) + 1)").ok();
+    s.withSql("select HIGHER_ORDER_FUNCTION(1, (x, y) -> null)").ok();
+    s.withSql("select HIGHER_ORDER_FUNCTION(1, (x, y) -> char_length(x) + 1)")
+        .type("RecordType(INTEGER NOT NULL EXPR$0) NOT NULL");
+
+    s.withSql("select HIGHER_ORDER_FUNCTION2(1, () -> 0.1)").ok();
+    s.withSql("select HIGHER_ORDER_FUNCTION2(1, () -> 0.1)")
+        .type("RecordType(INTEGER NOT NULL EXPR$0) NOT NULL");
+
+    s.withSql("select ^HIGHER_ORDER_FUNCTION(1, null)^")
+        .fails("Cannot apply '(?s).*HIGHER_ORDER_FUNCTION' to arguments of type "
+            + "'HIGHER_ORDER_FUNCTION\\(<INTEGER>, <NULL>\\)'.*");
+    s.withSql("select ^HIGHER_ORDER_FUNCTION(1, (x, y, z) -> x + 1)^")
+        .fails("Cannot apply '(?s).*HIGHER_ORDER_FUNCTION' to arguments of type "
+            + "'HIGHER_ORDER_FUNCTION\\(<INTEGER>, <FUNCTION\\(ANY, ANY, ANY\\) -> ANY>\\)'.*");
+  }
+

Review Comment:
   need to see more tests for EXISTS. including negative tests. wrong type of arguments, wrong number of arguments, arguments of wrong name, arguments of wrong case.
   
   check type derivation (if you didn't do it in SqlOperatorTest)



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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "julianhyde (via GitHub)" <gi...@apache.org>.
julianhyde commented on code in PR #3502:
URL: https://github.com/apache/calcite/pull/3502#discussion_r1393219264


##########
core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java:
##########
@@ -134,6 +134,18 @@ RelDataType createMapType(
       RelDataType keyType,
       RelDataType valueType);
 
+  /**
+   * Create a lambda expression type. Lambda expressions are functions that

Review Comment:
   s/Create/Creates/



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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1818176101

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL)
   
   [![84.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '84.6%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list) [84.6% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "julianhyde (via GitHub)" <gi...@apache.org>.
julianhyde commented on code in PR #3502:
URL: https://github.com/apache/calcite/pull/3502#discussion_r1393228926


##########
core/src/main/java/org/apache/calcite/rex/RexLambdaExpression.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.sql.SqlKind;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Represents a lambda expression.
+ */
+public class RexLambdaExpression extends RexNode {
+  //~ Instance fields --------------------------------------------------------
+
+  private final List<RexLambdaRef> parameters;
+  private final RexNode expression;
+
+  //~ Constructors -----------------------------------------------------------
+
+  RexLambdaExpression(List<RexLambdaRef> parameters, RexNode expression) {
+    this.expression = expression;
+    this.parameters = parameters;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public RelDataType getType() {
+    return expression.getType();
+  }
+
+  @Override public SqlKind getKind() {
+    return SqlKind.LAMBDA;
+  }
+
+  @Override public <R> R accept(RexVisitor<R> visitor) {
+    return visitor.visitLambda(this);
+  }
+
+  @Override public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitLambda(this, arg);
+  }
+
+  public RexNode getExpression() {
+    return expression;
+  }
+
+  public List<RexLambdaRef> getParameters() {
+    return parameters;
+  }
+
+  @Override public boolean equals(@Nullable Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    RexLambdaExpression rexLambdaExpression = (RexLambdaExpression) o;
+    return Objects.equals(expression, rexLambdaExpression.expression)
+        && Objects.equals(parameters, rexLambdaExpression.parameters);
+  }
+
+  @Override public int hashCode() {
+    return Objects.hash(expression, parameters);
+  }
+
+  @Override public String toString() {

Review Comment:
   Best thing is to do what `RexMRAggCall` does - make the class final and initialize `digest` in the constructor



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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1807978150

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [19 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL)
   
   [![83.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list) [83.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1853576821

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [12 New issues](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&inNewCodePeriod=true)  
   [72.1% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1797082714

   The real tests of whether the design works properly is to allow a lambda expression wherever a function call is allowed.
   So you should be allowed to say `SELECT (x -> x + 1)(age) FROM Person` or
   `SELECT  ":" || ((x -> x || x)(SUBSTRING(name, 3, 3))) FROM Person WHERE (x -> x > 5)(age)`.
   I think there is limited value to having a lambda expression that only works in some very restricted contexts.


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "macroguo-ghy (via GitHub)" <gi...@apache.org>.
macroguo-ghy commented on code in PR #3502:
URL: https://github.com/apache/calcite/pull/3502#discussion_r1393712543


##########
core/src/main/java/org/apache/calcite/rex/RexLambdaExpression.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.sql.SqlKind;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Represents a lambda expression.
+ */
+public class RexLambdaExpression extends RexNode {
+  //~ Instance fields --------------------------------------------------------
+
+  private final List<RexLambdaRef> parameters;
+  private final RexNode expression;
+
+  //~ Constructors -----------------------------------------------------------
+
+  RexLambdaExpression(List<RexLambdaRef> parameters, RexNode expression) {
+    this.expression = expression;
+    this.parameters = parameters;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public RelDataType getType() {
+    return expression.getType();
+  }
+
+  @Override public SqlKind getKind() {
+    return SqlKind.LAMBDA;
+  }
+
+  @Override public <R> R accept(RexVisitor<R> visitor) {
+    return visitor.visitLambda(this);
+  }
+
+  @Override public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitLambda(this, arg);
+  }
+
+  public RexNode getExpression() {
+    return expression;
+  }
+
+  public List<RexLambdaRef> getParameters() {
+    return parameters;
+  }
+
+  @Override public boolean equals(@Nullable Object o) {
+    if (this == o) {

Review Comment:
   Fixed



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaExpression.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.sql.SqlKind;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Represents a lambda expression.
+ */
+public class RexLambdaExpression extends RexNode {
+  //~ Instance fields --------------------------------------------------------
+
+  private final List<RexLambdaRef> parameters;
+  private final RexNode expression;
+
+  //~ Constructors -----------------------------------------------------------
+
+  RexLambdaExpression(List<RexLambdaRef> parameters, RexNode expression) {
+    this.expression = expression;

Review Comment:
   Fixed



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaRef.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.sql.SqlKind;
+
+/**
+ * Variable which references a field of a lambda expression.
+ */
+public class RexLambdaRef extends RexInputRef {
+
+  private final String paramName;

Review Comment:
   After making `RexLambdaRef` extend `RexSlot`, we can now use the `name`, so there is no need for the `paramName` anymore.



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaRef.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.sql.SqlKind;
+
+/**
+ * Variable which references a field of a lambda expression.
+ */
+public class RexLambdaRef extends RexInputRef {

Review Comment:
   Fixed



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaRef.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.sql.SqlKind;
+
+/**
+ * Variable which references a field of a lambda expression.

Review Comment:
   Fixed



##########
core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java:
##########
@@ -5371,6 +5372,28 @@ public static List sortArray(List list, boolean ascending) {
     return list;
   }
 
+  /** Support the EXISTS(list, function1) function. */

Review Comment:
   It wasn't until you told me that I noticed this function.
   I modified `boolean exists(List list, Function1<Object, Boolean> function1)`.



##########
core/src/main/java/org/apache/calcite/sql/fun/SqlLambdaExpressionOperator.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.SqlKind;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.validate.SqlLambdaExpressionScope;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+import java.util.List;
+
+import static com.google.common.collect.ImmutableList.toImmutableList;
+
+/**
+ * The {@code SqlLambdaExpressionOperator} represents a lambda expression.
+ * The syntax :
+ * {@code IDENTIFIER -> EXPRESSION} or {@code (IDENTIFIER, IDENTIFIER, ...) -> EXPRESSION}.
+ */
+public class SqlLambdaExpressionOperator extends SqlSpecialOperator {

Review Comment:
   Fixed



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionOperandTypeChecker.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlLambdaExpressionScope;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Operand type-checking strategy where the type of the operand is a lambda
+ * expression with a given return type and argument types.
+ */
+public class LambdaExpressionOperandTypeChecker implements SqlSingleOperandTypeChecker {
+
+  private final SqlTypeFamily returnTypeFamily;
+  private final List<SqlTypeFamily> argFamilies;
+
+  public LambdaExpressionOperandTypeChecker(
+      SqlTypeFamily returnTypeFamily,
+      List<SqlTypeFamily> argFamilies) {
+    this.returnTypeFamily = returnTypeFamily;

Review Comment:
   Fixed.



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionOperandTypeChecker.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlLambdaExpressionScope;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Operand type-checking strategy where the type of the operand is a lambda
+ * expression with a given return type and argument types.
+ */
+public class LambdaExpressionOperandTypeChecker implements SqlSingleOperandTypeChecker {
+
+  private final SqlTypeFamily returnTypeFamily;
+  private final List<SqlTypeFamily> argFamilies;
+
+  public LambdaExpressionOperandTypeChecker(
+      SqlTypeFamily returnTypeFamily,
+      List<SqlTypeFamily> argFamilies) {
+    this.returnTypeFamily = returnTypeFamily;
+    this.argFamilies = argFamilies;
+  }
+
+  @Override public String getAllowedSignatures(SqlOperator op, String opName) {
+    ImmutableList.Builder<SqlTypeFamily> builder = ImmutableList.builder();
+    builder.addAll(argFamilies);
+    builder.add(returnTypeFamily);
+
+    return SqlUtil.getAliasedSignature(op, opName, builder.build());
+  }
+
+  @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+      boolean throwOnFailure) {
+    return false;
+  }
+
+  @Override public boolean checkSingleOperandType(
+      SqlCallBinding callBinding, SqlNode operand, int iFormalOperand, boolean throwOnFailure) {
+    if (!(operand instanceof SqlLambdaExpression)
+        || ((SqlLambdaExpression) operand).getParameters().size() != argFamilies.size()) {
+      if (throwOnFailure) {
+        throw callBinding.newValidationSignatureError();
+      }
+      return false;
+    }
+    SqlLambdaExpression lambdaExpr = (SqlLambdaExpression) operand;
+
+    if (lambdaExpr.getParameters().isEmpty()
+        || argFamilies.stream().allMatch(f -> f == SqlTypeFamily.ANY)
+        || returnTypeFamily == SqlTypeFamily.ANY) {
+      return true;
+    }
+
+    if (SqlUtil.isNullLiteral(lambdaExpr.getExpression(), false)) {
+      if (callBinding.isTypeCoercionEnabled()) {
+        return true;
+      } else if (throwOnFailure) {
+        throw callBinding.getValidator().newValidationError(lambdaExpr.getExpression(),
+            RESOURCE.nullIllegal());
+      } else {
+        return false;
+      }
+    }
+
+    // Replace the parameter types in the lambda expression.
+    final SqlValidator validator = callBinding.getValidator();
+    SqlLambdaExpressionScope scope =
+        (SqlLambdaExpressionScope) validator.getLambdaExpressionScope(lambdaExpr);
+    for (int i = 0; i < argFamilies.size(); i++) {
+      SqlNode param = lambdaExpr.getParameters().get(i);
+      RelDataType type = argFamilies.get(i).getDefaultConcreteType(callBinding.getTypeFactory());
+      if (type != null) {
+        scope.getParameterTypes().put(param.toString(), type);
+      }
+    }
+    lambdaExpr.accept(new TypeRemover(validator));
+
+    // Given the new relDataType, re-validate the lambda expression.
+    validator.validateLambdaExpression(lambdaExpr);
+    final RelDataType newType = validator.getValidatedNodeType(lambdaExpr);
+    assert newType instanceof LambdaExpressionSqlType;
+    final SqlTypeName returnTypeName =
+        ((LambdaExpressionSqlType) newType).getReturnType().getSqlTypeName();
+    if (returnTypeName == SqlTypeName.ANY
+        || returnTypeFamily.getTypeNames().contains(returnTypeName)) {
+      return true;
+    }
+
+    if (throwOnFailure) {
+      throw callBinding.newValidationSignatureError();
+    }
+    return false;
+  }
+
+  /**

Review Comment:
   Since lambda expressions will be validated for the second time based on the given parameter type, the type cached during the first validation must be cleared.
   I will write the reason in Javadoc.



##########
core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java:
##########
@@ -77,6 +77,7 @@ public enum SqlTypeFamily implements RelDataTypeFamily {
   CURSOR,
   COLUMN_LIST,
   GEO,
+  LAMBDA_EXPRESSION,

Review Comment:
   Fixed.



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionOperandTypeChecker.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlLambdaExpressionScope;
+import org.apache.calcite.sql.validate.SqlValidator;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Operand type-checking strategy where the type of the operand is a lambda
+ * expression with a given return type and argument types.
+ */
+public class LambdaExpressionOperandTypeChecker implements SqlSingleOperandTypeChecker {

Review Comment:
   Fixed. I made this class as inner class of `OperandTypes`.



##########
core/src/main/java/org/apache/calcite/sql/validate/SqlLambdaExpressionScope.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Litmus;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Scope for a {@link SqlLambdaExpression LAMBDA EXPRESSION}.
+ */
+public class SqlLambdaExpressionScope extends ListScope {
+  private final SqlLambdaExpression lambdaExpr;
+  private final Map<String, RelDataType> parameterTypes;

Review Comment:
   We need to register the type of parameter to `SqlLambdaExpressionScope` when validate lambda expression second time, so I use the map as cache.



##########
testkit/src/main/java/org/apache/calcite/test/MockSqlOperatorTable.java:
##########
@@ -623,4 +625,24 @@ public CompositeFunction() {
       return typeFactory.createSqlType(SqlTypeName.BIGINT);
     }
   }
+
+  private static final SqlFunction HIGHER_ORDER_FUNCTION =
+      new SqlFunction("HIGHER_ORDER_FUNCTION",

Review Comment:
   Fixed



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionSqlType.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataTypeField;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * SQL lambda expression type.
+ */

Review Comment:
   I have renamed this class to `FunctionSqlType`. But I think all the `RelDataType` classes are public, `FunctionSqlType` should be consistent with them.



##########
core/src/main/java/org/apache/calcite/sql/validate/SqlLambdaExpressionScope.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLambdaExpression;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Litmus;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Scope for a {@link SqlLambdaExpression LAMBDA EXPRESSION}.
+ */
+public class SqlLambdaExpressionScope extends ListScope {
+  private final SqlLambdaExpression lambdaExpr;
+  private final Map<String, RelDataType> parameterTypes;
+
+  public SqlLambdaExpressionScope(
+      SqlValidatorScope parent, SqlLambdaExpression lambdaExpr) {
+    super(parent);
+    this.lambdaExpr = lambdaExpr;
+
+    // default parameter type is ANY

Review Comment:
   In fact, the lambda expression will be validated twice.
   
   A simple lambda expression `a -> a`, we can not infer the type of lambda if we just know the lambda expression, this is why I say "default parameter type is ANY".
   
   But if we see the lambda expression in a function, we will validate the lambda expression second time in `LambdaExpressionOperandTypeChecker`.
   
   For example, give a function `test_func1`, whose type checker is 
   ```java
   OperandTypes.sequence("TEST_FUNC1(INTEGER, FUNCTION(STRING) -> NUMERIC)",
           OperandTypes.family(SqlTypeFamily.INTEGER),
           OperandTypes.function(SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING))
   ```
   
   Assuming there is an SQL statement, 
   ```sql
   select test_func1(1, x -> x / 2);
   ```
   The first validation: 
   we assume the type of `x` is `ANY`. We can know the `x -> x / 2` is legal.
   The second validation:
   Based on the type checker, the type of `x` is string, so `x / 2` is not a valid expression.
   
   
   



##########
core/src/main/codegen/templates/Parser.jj:
##########
@@ -8787,6 +8817,7 @@ void NonReservedKeyWord2of3() :
 |   < NE2: "!=" >
 |   < PLUS: "+" >
 |   < MINUS: "-" >
+|   < LAMBDA_OPERATOR: "->" >

Review Comment:
   Fixed.



##########
core/src/main/codegen/templates/Parser.jj:
##########
@@ -3912,6 +3919,29 @@ SqlNode Expression3(ExprContext exprContext) :
     }
 }
 
+/**
+ * Parses a lambda expression.
+ */
+SqlNode LambdaExpression() :
+{
+    final SqlNodeList parameters;
+    final SqlNode expression;
+    final Span s;
+}
+{
+    (
+        LOOKAHEAD(2)
+        <LPAREN> <RPAREN> { parameters = SqlNodeList.EMPTY; }
+    |
+        parameters = SimpleIdentifierOrList()

Review Comment:
   I extract this rule to `SimpleIdentifierOrListOrEmpty`.



##########
core/src/main/codegen/templates/Parser.jj:
##########
@@ -1033,6 +1034,9 @@ void AddArg0(List<SqlNode> list, ExprContext exprContext) :
     )
     (
         e = Default()
+    |
+        LOOKAHEAD((SimpleIdentifierOrList() | <LPAREN> <RPAREN>) <LAMBDA_OPERATOR>)
+        e = LambdaExpression()

Review Comment:
   The most important token is `->`, so the expense depends on how many tokens come before `->`.
   For example:
   ```
   () -> true  // equals LOOKAHEAD(3)
   (a, b) -> a + b // equals LOOKAHED(6)
   ```



##########
core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java:
##########
@@ -779,6 +782,17 @@ public SqlNode toSql(@Nullable RexProgram program, RexNode rex) {
           return SqlStdOperatorTable.NOT.createCall(POS, node);
         }
 
+      case LAMBDA:
+        final RexLambdaExpression lambda = (RexLambdaExpression) rex;
+        final SqlNodeList parameters = new SqlNodeList(POS);
+        for (RexLambdaRef parameter : lambda.getParameters()) {
+          parameters.add(toSql(program, parameter));
+        }
+        final SqlNode expression = toSql(program, lambda.getExpression());
+        return new SqlLambdaExpression(POS, parameters, expression);
+      case LAMBDA_REF:

Review Comment:
   Fixed.



##########
core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java:
##########
@@ -134,6 +134,18 @@ RelDataType createMapType(
       RelDataType keyType,
       RelDataType valueType);
 
+  /**
+   * Create a lambda expression type. Lambda expressions are functions that

Review Comment:
   Fixed.



##########
core/src/main/java/org/apache/calcite/rex/RexLambdaExpression.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.sql.SqlKind;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Represents a lambda expression.
+ */
+public class RexLambdaExpression extends RexNode {
+  //~ Instance fields --------------------------------------------------------
+
+  private final List<RexLambdaRef> parameters;
+  private final RexNode expression;
+
+  //~ Constructors -----------------------------------------------------------
+
+  RexLambdaExpression(List<RexLambdaRef> parameters, RexNode expression) {
+    this.expression = expression;
+    this.parameters = parameters;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public RelDataType getType() {
+    return expression.getType();
+  }
+
+  @Override public SqlKind getKind() {
+    return SqlKind.LAMBDA;
+  }
+
+  @Override public <R> R accept(RexVisitor<R> visitor) {
+    return visitor.visitLambda(this);
+  }
+
+  @Override public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitLambda(this, arg);
+  }
+
+  public RexNode getExpression() {
+    return expression;
+  }
+
+  public List<RexLambdaRef> getParameters() {
+    return parameters;
+  }
+
+  @Override public boolean equals(@Nullable Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    RexLambdaExpression rexLambdaExpression = (RexLambdaExpression) o;
+    return Objects.equals(expression, rexLambdaExpression.expression)
+        && Objects.equals(parameters, rexLambdaExpression.parameters);
+  }
+
+  @Override public int hashCode() {
+    return Objects.hash(expression, parameters);
+  }
+
+  @Override public String toString() {

Review Comment:
   Fixed, I use `for` loop instead.
   `RexLambdaExpression` extends `RexNode`, the `computeDigest` is defined in `RexCall`.



##########
core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java:
##########
@@ -101,6 +101,25 @@ public static FamilyOperandTypeChecker family(List<SqlTypeFamily> families) {
     return family(families, i -> false);
   }
 
+  /**
+   * Creates a checker that passes if the operand is a lambda expression with

Review Comment:
   Fixed.



##########
core/src/main/java/org/apache/calcite/sql/type/LambdaExpressionSqlType.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.rel.type.RelDataTypeField;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * SQL lambda expression type.
+ */
+public class LambdaExpressionSqlType extends AbstractSqlType {
+  private final @Nullable RelDataType parameterType;
+  private final RelDataType returnType;
+
+  public LambdaExpressionSqlType(@Nullable RelDataType parameterType, RelDataType returnType) {
+    super(SqlTypeName.LAMBDA_EXPRESSION, true, null);
+    assert parameterType == null || parameterType.isStruct();

Review Comment:
   The `parameterType` is never `null`, I will remove `@Nullable` and `assert parameterType == null`.



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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1821413409

   Do the lambdas allow capturing values?
   `SELECT EXISTS(T.a,  x -> x < T.VALUE) FROM T`


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1794261612

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL)
   
   [![83.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.5%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list) [83.5% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "julianhyde (via GitHub)" <gi...@apache.org>.
julianhyde commented on code in PR #3502:
URL: https://github.com/apache/calcite/pull/3502#discussion_r1393223537


##########
core/src/main/java/org/apache/calcite/rex/RexLambdaExpression.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.sql.SqlKind;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Represents a lambda expression.
+ */
+public class RexLambdaExpression extends RexNode {
+  //~ Instance fields --------------------------------------------------------
+
+  private final List<RexLambdaRef> parameters;
+  private final RexNode expression;
+
+  //~ Constructors -----------------------------------------------------------
+
+  RexLambdaExpression(List<RexLambdaRef> parameters, RexNode expression) {
+    this.expression = expression;
+    this.parameters = parameters;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public RelDataType getType() {
+    return expression.getType();
+  }
+
+  @Override public SqlKind getKind() {
+    return SqlKind.LAMBDA;
+  }
+
+  @Override public <R> R accept(RexVisitor<R> visitor) {
+    return visitor.visitLambda(this);
+  }
+
+  @Override public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+    return visitor.visitLambda(this, arg);
+  }
+
+  public RexNode getExpression() {
+    return expression;
+  }
+
+  public List<RexLambdaRef> getParameters() {
+    return parameters;
+  }
+
+  @Override public boolean equals(@Nullable Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    RexLambdaExpression rexLambdaExpression = (RexLambdaExpression) o;
+    return Objects.equals(expression, rexLambdaExpression.expression)
+        && Objects.equals(parameters, rexLambdaExpression.parameters);
+  }
+
+  @Override public int hashCode() {
+    return Objects.hash(expression, parameters);
+  }
+
+  @Override public String toString() {

Review Comment:
   why override `toString` rather than `computeDigest`?
   
   I think it would be clearer and more concise if you use a `for` loop rather than functional style.



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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "macroguo-ghy (via GitHub)" <gi...@apache.org>.
macroguo-ghy commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1851889246

   Thank you for your review. I apologize for the delayed response; I have been quite busy over the past few weeks.
   
   Regarding this PR, I have made modifications to almost all of the reviews. And then, I will write more tests. Once that is done, I will split this PR into two parts: one for lambda and another for the exists function


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1793651320

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL)
   
   [![83.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.5%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list) [83.5% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1793829383

   Is there a design document for this feature?
   This looks extremely useful, but before I review it I would like to have a big picture understanding of how it's supposed to work.
   For example, is there type inference for lambdas?
   
   Moreover, if EXISTS is independent on the lambdas perhaps it should be in a separate 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.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1794429043

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL)
   
   [![83.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.5%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list) [83.5% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1816368270

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL)
   
   [![84.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '84.5%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list) [84.5% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1793373875

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&types=CODE_SMELL)
   
   [![83.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.5%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list) [83.5% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1793829650

   Leaving aside the design document, I don't see any changes to the documentation files either.


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "macroguo-ghy (via GitHub)" <gi...@apache.org>.
macroguo-ghy merged PR #3502:
URL: https://github.com/apache/calcite/pull/3502


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1862112357

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [12 New issues](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&inNewCodePeriod=true)  
   [72.1% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


Re: [PR] [CALCITE-3679] Allow lambda expressions in SQL queries [calcite]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3502:
URL: https://github.com/apache/calcite/pull/3502#issuecomment-1853548508

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [11 New issues](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3502&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3502&resolved=false&inNewCodePeriod=true)  
   [72.0% Coverage on New Code](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_coverage&view=list)  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3502&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3502)
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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