You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "jadami10 (via GitHub)" <gi...@apache.org> on 2023/03/19 19:37:55 UTC

[GitHub] [pinot] jadami10 opened a new pull request, #10444: optimize queries where lhs and rhs of predicate are equal

jadami10 opened a new pull request, #10444:
URL: https://github.com/apache/pinot/pull/10444

   This is a minor `performance` `bugfix`.
   
   1. this fixes NullPointerExceptions in existing optimizers when performing `WHERE 1=1` queries. These would fail because the filter expression had no function call
   2. I noticed that `WHERE 1=1` was no simplified, but `WHERE col1>0 AND 1=1` was actually being simplified in the `NumericalFilterOptimizer`. So I put that part in a separate class to be used more generally for future cases like this
       - it does a little more work than expected once it sees and AND/OR/NOT expression
       - something else is converting 1=1 to literal TRUE, but I'm not sure where that is
   3. This adds a `IdenticalPredicateFilterOptimizer` class that converts `WHERE 1=1` or `WHERE "colA"!="colA"` to TRUE/FALSE respectively
   
   I've added a bunch more test cases, and I've tested manually in the Quickstart app. This is my first contribution to the query parsing part of the code base, so I don't have a great sense what test coverage looks like. But I imagine between unit and integration tests, this should catch any glaring breaks?
   
   


-- 
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@pinot.apache.org

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


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


[GitHub] [pinot] jadami10 commented on a diff in pull request #10444: optimize queries where lhs and rhs of predicate are equal

Posted by "jadami10 (via GitHub)" <gi...@apache.org>.
jadami10 commented on code in PR #10444:
URL: https://github.com/apache/pinot/pull/10444#discussion_r1146164462


##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/FlattenAndOrFilterOptimizer.java:
##########
@@ -40,6 +40,9 @@ public Expression optimize(Expression filterExpression, @Nullable Schema schema)
 
   private Expression optimize(Expression filterExpression) {
     Function function = filterExpression.getFunctionCall();
+    if (function == null) {

Review Comment:
   oh right, yes! It's already `true` at this point



-- 
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@pinot.apache.org

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


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


[GitHub] [pinot] jadami10 commented on a diff in pull request #10444: optimize queries where lhs and rhs of predicate are equal

Posted by "jadami10 (via GitHub)" <gi...@apache.org>.
jadami10 commented on code in PR #10444:
URL: https://github.com/apache/pinot/pull/10444#discussion_r1149974687


##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+  protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);
+  protected static final Expression FALSE = RequestUtils.getLiteralExpression(false);
+
+  /**
+   * This recursively optimizes each part of the filter expression. For any AND/OR/NOT,
+   * we optimize each child, then we optimize the remaining statement. If there is only
+   * a child statement, we optimize that.
+   */
+  @Override
+  public Expression optimize(Expression filterExpression, @Nullable Schema schema) {
+    if (!canBeOptimized(filterExpression, schema)) {
+      return filterExpression;
+    }
+
+    Function function = filterExpression.getFunctionCall();
+    List<Expression> operands = function.getOperands();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case AND:
+      case OR:
+      case NOT:
+        // Recursively traverse the expression tree to find an operator node that can be rewritten.
+        operands.forEach(operand -> optimize(operand, schema));
+
+        // We have rewritten the child operands, so rewrite the parent if needed.
+        return optimizeCurrent(filterExpression);
+      default:
+        return optimizeChild(filterExpression, schema);
+    }
+  }
+
+  abstract boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema);
+
+  /**
+   * Optimize any cases that are not AND/OR/NOT. This should be done by converting any cases
+   * that are always true to TRUE or always false to FALSE.
+   */
+  abstract Expression optimizeChild(Expression filterExpression, @Nullable Schema schema);
+
+  /**
+   * If any of the operands of AND function is "false", then the AND function itself is false and can be replaced with
+   * "false" literal. Otherwise, remove all the "true" operands of the AND function. Similarly, if any of the operands
+   * of OR function is "true", then the OR function itself is true and can be replaced with "true" literal. Otherwise,
+   * remove all the "false" operands of the OR function.
+   */
+  protected Expression optimizeCurrent(Expression expression) {
+    Function function = expression.getFunctionCall();
+    String operator = function.getOperator();
+    List<Expression> operands = function.getOperands();
+    if (operator.equals(FilterKind.AND.name())) {
+      // If any of the literal operands are always false, then replace AND function with FALSE.
+      for (Expression operand : operands) {
+        if (isAlwaysFalse(operand)) {
+          return FALSE;
+        }
+      }
+
+      // Remove all Literal operands that are always true.
+      operands.removeIf(this::isAlwaysTrue);
+      if (operands.isEmpty()) {
+        return TRUE;
+      }
+    } else if (operator.equals(FilterKind.OR.name())) {
+      // If any of the literal operands are always true, then replace OR function with TRUE
+      for (Expression operand : operands) {
+        if (isAlwaysTrue(operand)) {
+          return TRUE;
+        }
+      }
+
+      // Remove all Literal operands that are always false.
+      operands.removeIf(this::isAlwaysFalse);
+      if (operands.isEmpty()) {
+        return FALSE;
+      }
+    } else if (operator.equals(FilterKind.NOT.name())) {
+      assert operands.size() == 1;
+      Expression operand = operands.get(0);
+      if (isAlwaysTrue(operand)) {
+        return FALSE;
+      }
+      if (isAlwaysFalse(operand)) {
+        return TRUE;
+      }
+    }
+    return expression;
+  }
+
+  private boolean isAlwaysFalse(Expression operand) {
+    return operand.equals(FALSE);
+  }
+
+  private boolean isAlwaysTrue(Expression operand) {
+    return operand.equals(TRUE);
+  }

Review Comment:
   good point. it made more sense in the initial PR



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+  protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);
+  protected static final Expression FALSE = RequestUtils.getLiteralExpression(false);
+
+  /**
+   * This recursively optimizes each part of the filter expression. For any AND/OR/NOT,
+   * we optimize each child, then we optimize the remaining statement. If there is only
+   * a child statement, we optimize that.
+   */
+  @Override
+  public Expression optimize(Expression filterExpression, @Nullable Schema schema) {
+    if (!canBeOptimized(filterExpression, schema)) {
+      return filterExpression;
+    }
+
+    Function function = filterExpression.getFunctionCall();
+    List<Expression> operands = function.getOperands();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case AND:
+      case OR:
+      case NOT:
+        // Recursively traverse the expression tree to find an operator node that can be rewritten.
+        operands.forEach(operand -> optimize(operand, schema));

Review Comment:
   done



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/IdenticalPredicateFilterOptimizer.java:
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This optimizer converts all predicates where the left hand side == right hand side to
+ * a simple TRUE/FALSE literal value. While filters like, WHERE 1=1 OR "col1"="col1" are not
+ * typical, they end up expensive in Pinot because they are rewritten as A-A==0.
+ */
+public class IdenticalPredicateFilterOptimizer extends BaseAndOrBooleanFilterOptimizer {
+
+  @Override
+  boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema) {
+    // if there's no function call, there's no lhs or rhs
+    return filterExpression.getFunctionCall() != null;
+  }
+
+  @Override
+  Expression optimizeChild(Expression filterExpression, @Nullable Schema schema) {
+    Function function = filterExpression.getFunctionCall();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, true);
+        }
+        break;
+      case NOT_EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, false);
+        }
+        break;
+      default:
+        break;
+    }
+    return filterExpression;
+  }
+
+  /**
+   * Pinot queries of the WHERE 1 != 1 AND "col1" = "col2" variety are rewritten as
+   * 1-1 != 0 AND "col1"-"col2" = 0. Therefore, we check specifically for the case where
+   * the operand is set up in this fashion.
+   *
+   * We return false specifically after every check to ensure we're only continuing when
+   * the input looks as expected. Otherwise, it's easy to for one of the operand functions
+   * to return null and fail the query.
+   */
+  private boolean hasIdenticalLhsAndRhs(Expression operand) {
+    Function function = operand.getFunctionCall();
+    if (function == null) {
+      return false;
+    }
+    List<Expression> children = function.getOperands();
+    boolean hasTwoChildren = children.size() == 2;

Review Comment:
   a `precondition` would fail the query, no? even if it is possible, this function is really only supposed to optimize this one case.



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/IdenticalPredicateFilterOptimizer.java:
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This optimizer converts all predicates where the left hand side == right hand side to
+ * a simple TRUE/FALSE literal value. While filters like, WHERE 1=1 OR "col1"="col1" are not
+ * typical, they end up expensive in Pinot because they are rewritten as A-A==0.
+ */
+public class IdenticalPredicateFilterOptimizer extends BaseAndOrBooleanFilterOptimizer {
+
+  @Override
+  boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema) {
+    // if there's no function call, there's no lhs or rhs
+    return filterExpression.getFunctionCall() != null;
+  }
+
+  @Override
+  Expression optimizeChild(Expression filterExpression, @Nullable Schema schema) {
+    Function function = filterExpression.getFunctionCall();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, true);
+        }
+        break;
+      case NOT_EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, false);
+        }
+        break;
+      default:
+        break;
+    }
+    return filterExpression;
+  }
+
+  /**
+   * Pinot queries of the WHERE 1 != 1 AND "col1" = "col2" variety are rewritten as
+   * 1-1 != 0 AND "col1"-"col2" = 0. Therefore, we check specifically for the case where

Review Comment:
   I didn't see that class until now. But even then, I think I slightly prefer this more as an optimization than a rewrite. But it's probably easier to do there before it gets converted just for us to convert it back.



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/IdenticalPredicateFilterOptimizer.java:
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This optimizer converts all predicates where the left hand side == right hand side to
+ * a simple TRUE/FALSE literal value. While filters like, WHERE 1=1 OR "col1"="col1" are not
+ * typical, they end up expensive in Pinot because they are rewritten as A-A==0.
+ */
+public class IdenticalPredicateFilterOptimizer extends BaseAndOrBooleanFilterOptimizer {
+
+  @Override
+  boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema) {
+    // if there's no function call, there's no lhs or rhs
+    return filterExpression.getFunctionCall() != null;
+  }
+
+  @Override
+  Expression optimizeChild(Expression filterExpression, @Nullable Schema schema) {
+    Function function = filterExpression.getFunctionCall();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {

Review Comment:
   ty, no need to recompute it all



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/IdenticalPredicateFilterOptimizer.java:
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This optimizer converts all predicates where the left hand side == right hand side to
+ * a simple TRUE/FALSE literal value. While filters like, WHERE 1=1 OR "col1"="col1" are not
+ * typical, they end up expensive in Pinot because they are rewritten as A-A==0.
+ */
+public class IdenticalPredicateFilterOptimizer extends BaseAndOrBooleanFilterOptimizer {
+
+  @Override
+  boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema) {
+    // if there's no function call, there's no lhs or rhs
+    return filterExpression.getFunctionCall() != null;
+  }
+
+  @Override
+  Expression optimizeChild(Expression filterExpression, @Nullable Schema schema) {
+    Function function = filterExpression.getFunctionCall();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, true);

Review Comment:
   yup, even better



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+  protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);
+  protected static final Expression FALSE = RequestUtils.getLiteralExpression(false);
+
+  /**
+   * This recursively optimizes each part of the filter expression. For any AND/OR/NOT,
+   * we optimize each child, then we optimize the remaining statement. If there is only
+   * a child statement, we optimize that.
+   */
+  @Override
+  public Expression optimize(Expression filterExpression, @Nullable Schema schema) {
+    if (!canBeOptimized(filterExpression, schema)) {
+      return filterExpression;
+    }
+
+    Function function = filterExpression.getFunctionCall();
+    List<Expression> operands = function.getOperands();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case AND:
+      case OR:
+      case NOT:
+        // Recursively traverse the expression tree to find an operator node that can be rewritten.
+        operands.forEach(operand -> optimize(operand, schema));
+
+        // We have rewritten the child operands, so rewrite the parent if needed.
+        return optimizeCurrent(filterExpression);
+      default:
+        return optimizeChild(filterExpression, schema);
+    }
+  }
+
+  abstract boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema);
+
+  /**
+   * Optimize any cases that are not AND/OR/NOT. This should be done by converting any cases
+   * that are always true to TRUE or always false to FALSE.
+   */
+  abstract Expression optimizeChild(Expression filterExpression, @Nullable Schema schema);
+
+  /**
+   * If any of the operands of AND function is "false", then the AND function itself is false and can be replaced with
+   * "false" literal. Otherwise, remove all the "true" operands of the AND function. Similarly, if any of the operands
+   * of OR function is "true", then the OR function itself is true and can be replaced with "true" literal. Otherwise,
+   * remove all the "false" operands of the OR function.
+   */
+  protected Expression optimizeCurrent(Expression expression) {
+    Function function = expression.getFunctionCall();
+    String operator = function.getOperator();
+    List<Expression> operands = function.getOperands();
+    if (operator.equals(FilterKind.AND.name())) {
+      // If any of the literal operands are always false, then replace AND function with FALSE.
+      for (Expression operand : operands) {
+        if (isAlwaysFalse(operand)) {
+          return FALSE;
+        }
+      }
+
+      // Remove all Literal operands that are always true.
+      operands.removeIf(this::isAlwaysTrue);
+      if (operands.isEmpty()) {
+        return TRUE;
+      }
+    } else if (operator.equals(FilterKind.OR.name())) {
+      // If any of the literal operands are always true, then replace OR function with TRUE
+      for (Expression operand : operands) {
+        if (isAlwaysTrue(operand)) {
+          return TRUE;
+        }
+      }
+
+      // Remove all Literal operands that are always false.
+      operands.removeIf(this::isAlwaysFalse);
+      if (operands.isEmpty()) {
+        return FALSE;
+      }
+    } else if (operator.equals(FilterKind.NOT.name())) {
+      assert operands.size() == 1;
+      Expression operand = operands.get(0);
+      if (isAlwaysTrue(operand)) {
+        return FALSE;
+      }
+      if (isAlwaysFalse(operand)) {
+        return TRUE;
+      }
+    }
+    return expression;
+  }
+
+  private boolean isAlwaysFalse(Expression operand) {
+    return operand.equals(FALSE);
+  }
+
+  private boolean isAlwaysTrue(Expression operand) {
+    return operand.equals(TRUE);
+  }
+
+  /** Change the expression value to boolean literal with given value. */
+  protected static void setExpressionToBoolean(Expression expression, boolean value) {

Review Comment:
   done. There's still some more mutation in NumericalFilterOptimizer, but this gets rid of a big part



-- 
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@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #10444: optimize queries where lhs and rhs of predicate are equal

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #10444:
URL: https://github.com/apache/pinot/pull/10444#discussion_r1145666928


##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/FlattenAndOrFilterOptimizer.java:
##########
@@ -40,6 +40,9 @@ public Expression optimize(Expression filterExpression, @Nullable Schema schema)
 
   private Expression optimize(Expression filterExpression) {
     Function function = filterExpression.getFunctionCall();
+    if (function == null) {

Review Comment:
   `1=1` should be a function. It might fail here because `1=1` might already be evaluated to `true` (literal only expression)



-- 
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@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang merged pull request #10444: optimize queries where lhs and rhs of predicate are equal

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang merged PR #10444:
URL: https://github.com/apache/pinot/pull/10444


-- 
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@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #10444: optimize queries where lhs and rhs of predicate are equal

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #10444:
URL: https://github.com/apache/pinot/pull/10444#discussion_r1144217959


##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/FlattenAndOrFilterOptimizer.java:
##########
@@ -40,6 +40,9 @@ public Expression optimize(Expression filterExpression, @Nullable Schema schema)
 
   private Expression optimize(Expression filterExpression) {
     Function function = filterExpression.getFunctionCall();
+    if (function == null) {

Review Comment:
   Is this added to handle the `WHERE true` clause? I assume current code will cause NPE?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java:
##########
@@ -170,6 +170,8 @@ private void validateQueries() {
     for (String queryString : _queryWeightMap.keySet()) {
       try {
         PinotQuery pinotQuery = CalciteSqlParser.compileToPinotQuery(queryString);
+        // TODO: should we catch and ignore any errors here. If we error on query optimization,

Review Comment:
   Good point. Ignoring the error is more robust, while failing the query can help catch the bug in the optimizer and prevent certain unexpected performance degradation. Currently optimize logic is applied in-place (there is no return value), so I personally prefer directly failing the query since the query might already be modified and messed up



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+    protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);
+    protected static final Expression FALSE = RequestUtils.getLiteralExpression(false);
+
+    @Override
+    public abstract Expression optimize(Expression filterExpression, @Nullable Schema schema);

Review Comment:
   (minor) No need to override this API to an abstract method



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+    protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);

Review Comment:
   This file doesn't follow the [Pinot Style](https://docs.pinot.apache.org/developers/developers-and-contributors/code-setup#setup-ide)



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/IdenticalPredicateFilterOptimizer.java:
##########
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This optimizer converts all predicates where the left hand side == right hand side to
+ * a simple TRUE/FALSE literal value. While filters like, WHERE 1=1 OR "col1"="col1" are not
+ * typical, they end up expensive in Pinot because they are rewritten as A-A==0.
+ */
+public class IdenticalPredicateFilterOptimizer extends BaseAndOrBooleanFilterOptimizer {
+
+  @Override
+  public Expression optimize(Expression filterExpression, @Nullable Schema schema) {
+    Function function = filterExpression.getFunctionCall();
+    if (function == null) {
+      return filterExpression;
+    }
+
+    List<Expression> operands = function.getOperands();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case AND:
+      case OR:
+      case NOT:
+        // Recursively traverse the expression tree to find an operator node that can be rewritten.
+        operands.forEach(operand -> optimize(operand, schema));
+
+        // We have rewritten the child operands, so rewrite the parent if needed.
+        return optimizeCurrent(filterExpression);
+      case EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          return TRUE;
+        }
+        return filterExpression;
+      case NOT_EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          return FALSE;
+        }
+        return filterExpression;
+      default:
+        return filterExpression;
+    }
+  }
+
+  @Override
+  protected boolean isAlwaysFalse(Expression operand) {

Review Comment:
   Do we need to override this method? After the DFS, all the children should already be optimized



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/FlattenAndOrFilterOptimizer.java:
##########
@@ -50,7 +53,7 @@ private Expression optimize(Expression filterExpression) {
     for (Expression child : children) {
       Expression optimizedChild = optimize(child);
       Function childFunction = optimizedChild.getFunctionCall();
-      if (childFunction.getOperator().equals(operator)) {
+      if (childFunction != null && childFunction.getOperator().equals(operator)) {

Review Comment:
   We should be able to eliminate the `true` and `false` here. Probably out of the scope of this PR, and we can add a TODO to address later



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/QueryOptimizer.java:
##########
@@ -44,7 +45,7 @@ public class QueryOptimizer {
   //   values to the proper format so that they can be properly parsed
   private static final List<FilterOptimizer> FILTER_OPTIMIZERS =
       Arrays.asList(new FlattenAndOrFilterOptimizer(), new MergeEqInFilterOptimizer(), new NumericalFilterOptimizer(),
-          new TimePredicateFilterOptimizer(), new MergeRangeFilterOptimizer());
+          new TimePredicateFilterOptimizer(), new MergeRangeFilterOptimizer(), new IdenticalPredicateFilterOptimizer());

Review Comment:
   Should we apply this optimizer in the end? If it doesn't rely on other optimizers, we can put it next to the flatten optimizer to avoid other optimizer to optimize on identical predicate



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+    protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);
+    protected static final Expression FALSE = RequestUtils.getLiteralExpression(false);
+
+    @Override
+    public abstract Expression optimize(Expression filterExpression, @Nullable Schema schema);
+
+    /**
+     * If any of the operands of AND function is "false", then the AND function itself is false and can be replaced with
+     * "false" literal. Otherwise, remove all the "true" operands of the AND function. Similarly, if any of the operands
+     * of OR function is "true", then the OR function itself is true and can be replaced with "true" literal. Otherwise,
+     * remove all the "false" operands of the OR function.
+     */
+    protected Expression optimizeCurrent(Expression expression) {
+        Function function = expression.getFunctionCall();
+        String operator = function.getOperator();
+        List<Expression> operands = function.getOperands();
+        if (operator.equals(FilterKind.AND.name())) {
+            // If any of the literal operands are always false, then replace AND function with FALSE.
+            for (Expression operand : operands) {
+                if (isAlwaysFalse(operand)) {
+                    return FALSE;
+                }
+            }
+
+            // Remove all Literal operands that are always true.
+            operands.removeIf(this::isAlwaysTrue);
+            if (operands.isEmpty()) {
+                return TRUE;
+            }
+        } else if (operator.equals(FilterKind.OR.name())) {
+            // If any of the literal operands are always true, then replace OR function with TRUE
+            for (Expression operand : operands) {
+                if (isAlwaysTrue(operand)) {
+                    return TRUE;
+                }
+            }
+
+            // Remove all Literal operands that are always false.
+            operands.removeIf(this::isAlwaysFalse);
+            if (operands.isEmpty()) {
+                return FALSE;
+            }
+        } else if (operator.equals(FilterKind.NOT.name())) {
+            assert operands.size() == 1;
+            Expression operand = operands.get(0);
+            if (isAlwaysTrue(operand)) {
+                return FALSE;
+            }
+            if (isAlwaysFalse(operand)) {
+                return TRUE;
+            }
+        }
+        return expression;
+    }
+
+    protected boolean isAlwaysFalse(Expression operand) {

Review Comment:
   If we don't need to override this method (see comment below), we can change `optimizeCurrent` into a util method



-- 
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@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #10444: optimize queries where lhs and rhs of predicate are equal

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #10444:
URL: https://github.com/apache/pinot/pull/10444#discussion_r1149869941


##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+  protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);
+  protected static final Expression FALSE = RequestUtils.getLiteralExpression(false);
+
+  /**
+   * This recursively optimizes each part of the filter expression. For any AND/OR/NOT,
+   * we optimize each child, then we optimize the remaining statement. If there is only
+   * a child statement, we optimize that.
+   */
+  @Override
+  public Expression optimize(Expression filterExpression, @Nullable Schema schema) {
+    if (!canBeOptimized(filterExpression, schema)) {
+      return filterExpression;
+    }
+
+    Function function = filterExpression.getFunctionCall();
+    List<Expression> operands = function.getOperands();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case AND:
+      case OR:
+      case NOT:
+        // Recursively traverse the expression tree to find an operator node that can be rewritten.
+        operands.forEach(operand -> optimize(operand, schema));
+
+        // We have rewritten the child operands, so rewrite the parent if needed.
+        return optimizeCurrent(filterExpression);
+      default:
+        return optimizeChild(filterExpression, schema);
+    }
+  }
+
+  abstract boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema);
+
+  /**
+   * Optimize any cases that are not AND/OR/NOT. This should be done by converting any cases
+   * that are always true to TRUE or always false to FALSE.
+   */
+  abstract Expression optimizeChild(Expression filterExpression, @Nullable Schema schema);
+
+  /**
+   * If any of the operands of AND function is "false", then the AND function itself is false and can be replaced with
+   * "false" literal. Otherwise, remove all the "true" operands of the AND function. Similarly, if any of the operands
+   * of OR function is "true", then the OR function itself is true and can be replaced with "true" literal. Otherwise,
+   * remove all the "false" operands of the OR function.
+   */
+  protected Expression optimizeCurrent(Expression expression) {
+    Function function = expression.getFunctionCall();
+    String operator = function.getOperator();
+    List<Expression> operands = function.getOperands();
+    if (operator.equals(FilterKind.AND.name())) {
+      // If any of the literal operands are always false, then replace AND function with FALSE.
+      for (Expression operand : operands) {
+        if (isAlwaysFalse(operand)) {
+          return FALSE;
+        }
+      }
+
+      // Remove all Literal operands that are always true.
+      operands.removeIf(this::isAlwaysTrue);
+      if (operands.isEmpty()) {
+        return TRUE;
+      }
+    } else if (operator.equals(FilterKind.OR.name())) {
+      // If any of the literal operands are always true, then replace OR function with TRUE
+      for (Expression operand : operands) {
+        if (isAlwaysTrue(operand)) {
+          return TRUE;
+        }
+      }
+
+      // Remove all Literal operands that are always false.
+      operands.removeIf(this::isAlwaysFalse);
+      if (operands.isEmpty()) {
+        return FALSE;
+      }
+    } else if (operator.equals(FilterKind.NOT.name())) {
+      assert operands.size() == 1;
+      Expression operand = operands.get(0);
+      if (isAlwaysTrue(operand)) {
+        return FALSE;
+      }
+      if (isAlwaysFalse(operand)) {
+        return TRUE;
+      }
+    }
+    return expression;
+  }
+
+  private boolean isAlwaysFalse(Expression operand) {
+    return operand.equals(FALSE);
+  }
+
+  private boolean isAlwaysTrue(Expression operand) {
+    return operand.equals(TRUE);
+  }
+
+  /** Change the expression value to boolean literal with given value. */
+  protected static void setExpressionToBoolean(Expression expression, boolean value) {

Review Comment:
   Not introduced in this PR, but let's remove this method since we should avoid mutating an expression. We can use the constant `TRUE` and `FALSE` instead



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/IdenticalPredicateFilterOptimizer.java:
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This optimizer converts all predicates where the left hand side == right hand side to
+ * a simple TRUE/FALSE literal value. While filters like, WHERE 1=1 OR "col1"="col1" are not
+ * typical, they end up expensive in Pinot because they are rewritten as A-A==0.
+ */
+public class IdenticalPredicateFilterOptimizer extends BaseAndOrBooleanFilterOptimizer {
+
+  @Override
+  boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema) {
+    // if there's no function call, there's no lhs or rhs
+    return filterExpression.getFunctionCall() != null;
+  }
+
+  @Override
+  Expression optimizeChild(Expression filterExpression, @Nullable Schema schema) {
+    Function function = filterExpression.getFunctionCall();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {

Review Comment:
   (minor) Directly pass the operands



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/IdenticalPredicateFilterOptimizer.java:
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This optimizer converts all predicates where the left hand side == right hand side to
+ * a simple TRUE/FALSE literal value. While filters like, WHERE 1=1 OR "col1"="col1" are not
+ * typical, they end up expensive in Pinot because they are rewritten as A-A==0.
+ */
+public class IdenticalPredicateFilterOptimizer extends BaseAndOrBooleanFilterOptimizer {
+
+  @Override
+  boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema) {
+    // if there's no function call, there's no lhs or rhs
+    return filterExpression.getFunctionCall() != null;
+  }
+
+  @Override
+  Expression optimizeChild(Expression filterExpression, @Nullable Schema schema) {
+    Function function = filterExpression.getFunctionCall();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, true);
+        }
+        break;
+      case NOT_EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, false);
+        }
+        break;
+      default:
+        break;
+    }
+    return filterExpression;
+  }
+
+  /**
+   * Pinot queries of the WHERE 1 != 1 AND "col1" = "col2" variety are rewritten as
+   * 1-1 != 0 AND "col1"-"col2" = 0. Therefore, we check specifically for the case where

Review Comment:
   The rewrite is already happening in `PredicateComparisonRewriter.updateFunctionExpression()`, so we might just compare the lhs and rhs there.
   
   Since we already get this implementation, we can add a TODO here and revisit later



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+  protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);
+  protected static final Expression FALSE = RequestUtils.getLiteralExpression(false);
+
+  /**
+   * This recursively optimizes each part of the filter expression. For any AND/OR/NOT,
+   * we optimize each child, then we optimize the remaining statement. If there is only
+   * a child statement, we optimize that.
+   */
+  @Override
+  public Expression optimize(Expression filterExpression, @Nullable Schema schema) {
+    if (!canBeOptimized(filterExpression, schema)) {
+      return filterExpression;
+    }
+
+    Function function = filterExpression.getFunctionCall();
+    List<Expression> operands = function.getOperands();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case AND:
+      case OR:
+      case NOT:
+        // Recursively traverse the expression tree to find an operator node that can be rewritten.
+        operands.forEach(operand -> optimize(operand, schema));

Review Comment:
   Let's use `replaceAll()` here so that it still works when the optimize is not applied inplace



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/IdenticalPredicateFilterOptimizer.java:
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This optimizer converts all predicates where the left hand side == right hand side to
+ * a simple TRUE/FALSE literal value. While filters like, WHERE 1=1 OR "col1"="col1" are not
+ * typical, they end up expensive in Pinot because they are rewritten as A-A==0.
+ */
+public class IdenticalPredicateFilterOptimizer extends BaseAndOrBooleanFilterOptimizer {
+
+  @Override
+  boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema) {
+    // if there's no function call, there's no lhs or rhs
+    return filterExpression.getFunctionCall() != null;
+  }
+
+  @Override
+  Expression optimizeChild(Expression filterExpression, @Nullable Schema schema) {
+    Function function = filterExpression.getFunctionCall();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, true);
+        }
+        break;
+      case NOT_EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, false);
+        }
+        break;
+      default:
+        break;
+    }
+    return filterExpression;
+  }
+
+  /**
+   * Pinot queries of the WHERE 1 != 1 AND "col1" = "col2" variety are rewritten as
+   * 1-1 != 0 AND "col1"-"col2" = 0. Therefore, we check specifically for the case where
+   * the operand is set up in this fashion.
+   *
+   * We return false specifically after every check to ensure we're only continuing when
+   * the input looks as expected. Otherwise, it's easy to for one of the operand functions
+   * to return null and fail the query.
+   */
+  private boolean hasIdenticalLhsAndRhs(Expression operand) {
+    Function function = operand.getFunctionCall();
+    if (function == null) {
+      return false;
+    }
+    List<Expression> children = function.getOperands();
+    boolean hasTwoChildren = children.size() == 2;

Review Comment:
   (minor) Is there any case `EQ` or `NEQ` can have other than 2 children? We can probably make a precondition



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/IdenticalPredicateFilterOptimizer.java:
##########
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This optimizer converts all predicates where the left hand side == right hand side to
+ * a simple TRUE/FALSE literal value. While filters like, WHERE 1=1 OR "col1"="col1" are not
+ * typical, they end up expensive in Pinot because they are rewritten as A-A==0.
+ */
+public class IdenticalPredicateFilterOptimizer extends BaseAndOrBooleanFilterOptimizer {
+
+  @Override
+  boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema) {
+    // if there's no function call, there's no lhs or rhs
+    return filterExpression.getFunctionCall() != null;
+  }
+
+  @Override
+  Expression optimizeChild(Expression filterExpression, @Nullable Schema schema) {
+    Function function = filterExpression.getFunctionCall();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case EQUALS:
+        if (hasIdenticalLhsAndRhs(filterExpression)) {
+          setExpressionToBoolean(filterExpression, true);

Review Comment:
   Directly return `TRUE`, same for the following



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.ExpressionType;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.request.Literal;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+  protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);
+  protected static final Expression FALSE = RequestUtils.getLiteralExpression(false);
+
+  /**
+   * This recursively optimizes each part of the filter expression. For any AND/OR/NOT,
+   * we optimize each child, then we optimize the remaining statement. If there is only
+   * a child statement, we optimize that.
+   */
+  @Override
+  public Expression optimize(Expression filterExpression, @Nullable Schema schema) {
+    if (!canBeOptimized(filterExpression, schema)) {
+      return filterExpression;
+    }
+
+    Function function = filterExpression.getFunctionCall();
+    List<Expression> operands = function.getOperands();
+    FilterKind kind = FilterKind.valueOf(function.getOperator());
+    switch (kind) {
+      case AND:
+      case OR:
+      case NOT:
+        // Recursively traverse the expression tree to find an operator node that can be rewritten.
+        operands.forEach(operand -> optimize(operand, schema));
+
+        // We have rewritten the child operands, so rewrite the parent if needed.
+        return optimizeCurrent(filterExpression);
+      default:
+        return optimizeChild(filterExpression, schema);
+    }
+  }
+
+  abstract boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema);
+
+  /**
+   * Optimize any cases that are not AND/OR/NOT. This should be done by converting any cases
+   * that are always true to TRUE or always false to FALSE.
+   */
+  abstract Expression optimizeChild(Expression filterExpression, @Nullable Schema schema);
+
+  /**
+   * If any of the operands of AND function is "false", then the AND function itself is false and can be replaced with
+   * "false" literal. Otherwise, remove all the "true" operands of the AND function. Similarly, if any of the operands
+   * of OR function is "true", then the OR function itself is true and can be replaced with "true" literal. Otherwise,
+   * remove all the "false" operands of the OR function.
+   */
+  protected Expression optimizeCurrent(Expression expression) {
+    Function function = expression.getFunctionCall();
+    String operator = function.getOperator();
+    List<Expression> operands = function.getOperands();
+    if (operator.equals(FilterKind.AND.name())) {
+      // If any of the literal operands are always false, then replace AND function with FALSE.
+      for (Expression operand : operands) {
+        if (isAlwaysFalse(operand)) {
+          return FALSE;
+        }
+      }
+
+      // Remove all Literal operands that are always true.
+      operands.removeIf(this::isAlwaysTrue);
+      if (operands.isEmpty()) {
+        return TRUE;
+      }
+    } else if (operator.equals(FilterKind.OR.name())) {
+      // If any of the literal operands are always true, then replace OR function with TRUE
+      for (Expression operand : operands) {
+        if (isAlwaysTrue(operand)) {
+          return TRUE;
+        }
+      }
+
+      // Remove all Literal operands that are always false.
+      operands.removeIf(this::isAlwaysFalse);
+      if (operands.isEmpty()) {
+        return FALSE;
+      }
+    } else if (operator.equals(FilterKind.NOT.name())) {
+      assert operands.size() == 1;
+      Expression operand = operands.get(0);
+      if (isAlwaysTrue(operand)) {
+        return FALSE;
+      }
+      if (isAlwaysFalse(operand)) {
+        return TRUE;
+      }
+    }
+    return expression;
+  }
+
+  private boolean isAlwaysFalse(Expression operand) {
+    return operand.equals(FALSE);
+  }
+
+  private boolean isAlwaysTrue(Expression operand) {
+    return operand.equals(TRUE);
+  }

Review Comment:
   (nit) Slightly more readable if we just inline them or rename to `isTrue()` and `isFalse()`



-- 
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@pinot.apache.org

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


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


[GitHub] [pinot] codecov-commenter commented on pull request #10444: optimize queries where lhs and rhs of predicate are equal

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #10444:
URL: https://github.com/apache/pinot/pull/10444#issuecomment-1475547184

   ## [Codecov](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10444](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c7c578f) into [master](https://codecov.io/gh/apache/pinot/commit/d9c4315ca14997e6e0300d04f788e07723875159?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d9c4315) will **decrease** coverage by `50.31%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10444       +/-   ##
   =============================================
   - Coverage     64.21%   13.90%   -50.31%     
   + Complexity     6089      237     -5852     
   =============================================
     Files          2007     2009        +2     
     Lines        109281   109337       +56     
     Branches      16692    16708       +16     
   =============================================
   - Hits          70177    15208    -54969     
   - Misses        33993    92897    +58904     
   + Partials       5111     1232     -3879     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests1 | `?` | |
   | unittests2 | `13.90% <0.00%> (-0.02%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../pinot/controller/recommender/io/InputManager.java](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9yZWNvbW1lbmRlci9pby9JbnB1dE1hbmFnZXIuamF2YQ==) | `93.22% <ø> (ø)` | |
   | [...che/pinot/core/query/optimizer/QueryOptimizer.java](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9vcHRpbWl6ZXIvUXVlcnlPcHRpbWl6ZXIuamF2YQ==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [...imizer/filter/BaseAndOrBooleanFilterOptimizer.java](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9vcHRpbWl6ZXIvZmlsdGVyL0Jhc2VBbmRPckJvb2xlYW5GaWx0ZXJPcHRpbWl6ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../optimizer/filter/FlattenAndOrFilterOptimizer.java](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9vcHRpbWl6ZXIvZmlsdGVyL0ZsYXR0ZW5BbmRPckZpbHRlck9wdGltaXplci5qYXZh) | `0.00% <0.00%> (-77.78%)` | :arrow_down: |
   | [...izer/filter/IdenticalPredicateFilterOptimizer.java](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9vcHRpbWl6ZXIvZmlsdGVyL0lkZW50aWNhbFByZWRpY2F0ZUZpbHRlck9wdGltaXplci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ery/optimizer/filter/MergeEqInFilterOptimizer.java](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9vcHRpbWl6ZXIvZmlsdGVyL01lcmdlRXFJbkZpbHRlck9wdGltaXplci5qYXZh) | `0.00% <0.00%> (-92.60%)` | :arrow_down: |
   | [...ery/optimizer/filter/NumericalFilterOptimizer.java](https://codecov.io/gh/apache/pinot/pull/10444?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9vcHRpbWl6ZXIvZmlsdGVyL051bWVyaWNhbEZpbHRlck9wdGltaXplci5qYXZh) | `0.00% <0.00%> (-80.90%)` | :arrow_down: |
   
   ... and [1351 files with indirect coverage changes](https://codecov.io/gh/apache/pinot/pull/10444/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
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@pinot.apache.org

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


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


[GitHub] [pinot] jadami10 commented on a diff in pull request #10444: optimize queries where lhs and rhs of predicate are equal

Posted by "jadami10 (via GitHub)" <gi...@apache.org>.
jadami10 commented on code in PR #10444:
URL: https://github.com/apache/pinot/pull/10444#discussion_r1145467030


##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/FlattenAndOrFilterOptimizer.java:
##########
@@ -40,6 +40,9 @@ public Expression optimize(Expression filterExpression, @Nullable Schema schema)
 
   private Expression optimize(Expression filterExpression) {
     Function function = filterExpression.getFunctionCall();
+    if (function == null) {

Review Comment:
   both `where true` and `where 1=1` fail here with NPE.



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/QueryOptimizer.java:
##########
@@ -44,7 +45,7 @@ public class QueryOptimizer {
   //   values to the proper format so that they can be properly parsed
   private static final List<FilterOptimizer> FILTER_OPTIMIZERS =
       Arrays.asList(new FlattenAndOrFilterOptimizer(), new MergeEqInFilterOptimizer(), new NumericalFilterOptimizer(),
-          new TimePredicateFilterOptimizer(), new MergeRangeFilterOptimizer());
+          new TimePredicateFilterOptimizer(), new MergeRangeFilterOptimizer(), new IdenticalPredicateFilterOptimizer());

Review Comment:
   good call. it actually caught another null point exception moving it earlier



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+    protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);

Review Comment:
   good eye. I was working on a new laptop and hadn't set that up.



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/BaseAndOrBooleanFilterOptimizer.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.query.optimizer.filter;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.request.Expression;
+import org.apache.pinot.common.request.Function;
+import org.apache.pinot.common.utils.request.RequestUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.sql.FilterKind;
+
+/**
+ * This base class acts as a helper for any optimizer that is effectively removing filter conditions.
+ * It provides TRUE/FALSE literal classes that can be used to replace filter expressions that are always true/false.
+ * It provides an optimization implementation for AND/OR/NOT expressions.
+ */
+public abstract class BaseAndOrBooleanFilterOptimizer implements FilterOptimizer {
+
+    protected static final Expression TRUE = RequestUtils.getLiteralExpression(true);
+    protected static final Expression FALSE = RequestUtils.getLiteralExpression(false);
+
+    @Override
+    public abstract Expression optimize(Expression filterExpression, @Nullable Schema schema);
+
+    /**
+     * If any of the operands of AND function is "false", then the AND function itself is false and can be replaced with
+     * "false" literal. Otherwise, remove all the "true" operands of the AND function. Similarly, if any of the operands
+     * of OR function is "true", then the OR function itself is true and can be replaced with "true" literal. Otherwise,
+     * remove all the "false" operands of the OR function.
+     */
+    protected Expression optimizeCurrent(Expression expression) {
+        Function function = expression.getFunctionCall();
+        String operator = function.getOperator();
+        List<Expression> operands = function.getOperands();
+        if (operator.equals(FilterKind.AND.name())) {
+            // If any of the literal operands are always false, then replace AND function with FALSE.
+            for (Expression operand : operands) {
+                if (isAlwaysFalse(operand)) {
+                    return FALSE;
+                }
+            }
+
+            // Remove all Literal operands that are always true.
+            operands.removeIf(this::isAlwaysTrue);
+            if (operands.isEmpty()) {
+                return TRUE;
+            }
+        } else if (operator.equals(FilterKind.OR.name())) {
+            // If any of the literal operands are always true, then replace OR function with TRUE
+            for (Expression operand : operands) {
+                if (isAlwaysTrue(operand)) {
+                    return TRUE;
+                }
+            }
+
+            // Remove all Literal operands that are always false.
+            operands.removeIf(this::isAlwaysFalse);
+            if (operands.isEmpty()) {
+                return FALSE;
+            }
+        } else if (operator.equals(FilterKind.NOT.name())) {
+            assert operands.size() == 1;
+            Expression operand = operands.get(0);
+            if (isAlwaysTrue(operand)) {
+                return FALSE;
+            }
+            if (isAlwaysFalse(operand)) {
+                return TRUE;
+            }
+        }
+        return expression;
+    }
+
+    protected boolean isAlwaysFalse(Expression operand) {

Review Comment:
   I've reworked the interface to be a little clearer. The base class handles the DFS, and users just implement the base case. Let me know if this looks better.



##########
pinot-controller/src/main/java/org/apache/pinot/controller/recommender/io/InputManager.java:
##########
@@ -170,6 +170,8 @@ private void validateQueries() {
     for (String queryString : _queryWeightMap.keySet()) {
       try {
         PinotQuery pinotQuery = CalciteSqlParser.compileToPinotQuery(queryString);
+        // TODO: should we catch and ignore any errors here. If we error on query optimization,

Review Comment:
   ya, I was thinking the same about the fact that it might be only semi optimized when this fails. I've updated the comment to reflect this state



##########
pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/FlattenAndOrFilterOptimizer.java:
##########
@@ -50,7 +53,7 @@ private Expression optimize(Expression filterExpression) {
     for (Expression child : children) {
       Expression optimizedChild = optimize(child);
       Function childFunction = optimizedChild.getFunctionCall();
-      if (childFunction.getOperator().equals(operator)) {
+      if (childFunction != null && childFunction.getOperator().equals(operator)) {

Review Comment:
   It also fails on `where 1=1`, so the only way to eliminate those is to run some other optimizer first. that seems like it might be more complicated. Running this one first seems to simplify a lot of things



-- 
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@pinot.apache.org

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


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


[GitHub] [pinot] jadami10 commented on pull request #10444: optimize queries where lhs and rhs of predicate are equal

Posted by "jadami10 (via GitHub)" <gi...@apache.org>.
jadami10 commented on PR #10444:
URL: https://github.com/apache/pinot/pull/10444#issuecomment-1481156848

   > Looks good in general, great job!
   
   thank you! i see all checks passed. let me know if you have further comments, though


-- 
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@pinot.apache.org

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


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