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

[GitHub] [pinot] gortiz opened a new pull request, #10371: add the ability to change FilterOperatorUtils code

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

   This PR is related to https://github.com/apache/pinot/pull/10269. While https://github.com/apache/pinot/pull/10269 let us change the PlanMaker, let us change the `FilterOperatorUtils` code.
   
   It would be great to have a proper Query SPI system that let us modify different places in the code, but while we don't have it, this let us change the actual implementation of leaf nodes, which is very useful in a proprietary extension StarTree is creating.


-- 
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 #10371: add the ability to change FilterOperatorUtils code

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/FilterOperatorUtils.java:
##########
@@ -28,9 +28,218 @@
 
 
 public class FilterOperatorUtils {
+
+  private static Implementation _instance = new DefaultImplementation();
+
   private FilterOperatorUtils() {
   }
 
+  public static void setImplementation(Implementation newImplementation) {
+    _instance = newImplementation;
+  }
+
+  public interface Implementation {
+
+    /**
+     * Returns the leaf filter operator (i.e. not {@link AndFilterOperator} or {@link OrFilterOperator}).
+     */
+    BaseFilterOperator getLeafFilterOperator(PredicateEvaluator predicateEvaluator, DataSource dataSource,
+        int numDocs, boolean nullHandlingEnabled);
+
+    /**
+     * Returns the AND filter operator or equivalent filter operator.
+     */
+    BaseFilterOperator getAndFilterOperator(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators, int numDocs);
+
+    /**
+     * Returns the OR filter operator or equivalent filter operator.
+     */
+    BaseFilterOperator getOrFilterOperator(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators, int numDocs);
+
+    /**
+     * Returns the NOT filter operator or equivalent filter operator.
+     */
+    BaseFilterOperator getNotFilterOperator(QueryContext queryContext, BaseFilterOperator filterOperator,
+        int numDocs);
+
+    /**
+     * For AND filter operator, reorders its child filter operators based on the their cost and puts the ones with
+     * inverted index first in order to reduce the number of documents to be processed.
+     * <p>Special filter operators such as {@link MatchAllFilterOperator} and {@link EmptyFilterOperator} should be
+     * removed from the list before calling this method.
+     */
+    void reorderAndFilterChildOperators(QueryContext queryContext,

Review Comment:
   This method should not be part of the interface because it is implementation detail for `getAndFilterOperator()`



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/FilterOperatorUtils.java:
##########
@@ -28,9 +28,218 @@
 
 
 public class FilterOperatorUtils {
+
+  private static Implementation _instance = new DefaultImplementation();
+
   private FilterOperatorUtils() {
   }
 
+  public static void setImplementation(Implementation newImplementation) {
+    _instance = newImplementation;
+  }
+
+  public interface Implementation {
+
+    /**
+     * Returns the leaf filter operator (i.e. not {@link AndFilterOperator} or {@link OrFilterOperator}).
+     */
+    BaseFilterOperator getLeafFilterOperator(PredicateEvaluator predicateEvaluator, DataSource dataSource,
+        int numDocs, boolean nullHandlingEnabled);
+
+    /**
+     * Returns the AND filter operator or equivalent filter operator.
+     */
+    BaseFilterOperator getAndFilterOperator(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators, int numDocs);
+
+    /**
+     * Returns the OR filter operator or equivalent filter operator.
+     */
+    BaseFilterOperator getOrFilterOperator(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators, int numDocs);
+
+    /**
+     * Returns the NOT filter operator or equivalent filter operator.
+     */
+    BaseFilterOperator getNotFilterOperator(QueryContext queryContext, BaseFilterOperator filterOperator,
+        int numDocs);
+
+    /**
+     * For AND filter operator, reorders its child filter operators based on the their cost and puts the ones with
+     * inverted index first in order to reduce the number of documents to be processed.
+     * <p>Special filter operators such as {@link MatchAllFilterOperator} and {@link EmptyFilterOperator} should be
+     * removed from the list before calling this method.
+     */
+    void reorderAndFilterChildOperators(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators);
+  }
+
+  public static class DefaultImplementation implements Implementation {
+    @Override
+    public BaseFilterOperator getLeafFilterOperator(PredicateEvaluator predicateEvaluator, DataSource dataSource,
+        int numDocs, boolean nullHandlingEnabled) {
+      if (predicateEvaluator.isAlwaysFalse()) {
+        return EmptyFilterOperator.getInstance();
+      } else if (predicateEvaluator.isAlwaysTrue()) {
+        return new MatchAllFilterOperator(numDocs);
+      }
+
+      // Currently sorted index based filtering is supported only for
+      // dictionary encoded columns. The on-disk segment metadata
+      // will indicate if the column is sorted or not regardless of
+      // whether it is raw or dictionary encoded. Here when creating
+      // the filter operator, we need to make sure that sort filter
+      // operator is used only if the column is sorted and has dictionary.
+      Predicate.Type predicateType = predicateEvaluator.getPredicateType();
+      if (predicateType == Predicate.Type.RANGE) {
+        if (dataSource.getDataSourceMetadata().isSorted() && dataSource.getDictionary() != null) {
+          return new SortedIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        if (RangeIndexBasedFilterOperator.canEvaluate(predicateEvaluator, dataSource)) {
+          return new RangeIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        return new ScanBasedFilterOperator(predicateEvaluator, dataSource, numDocs, nullHandlingEnabled);
+      } else if (predicateType == Predicate.Type.REGEXP_LIKE) {
+        if (dataSource.getFSTIndex() != null && dataSource.getDataSourceMetadata().isSorted()) {
+          return new SortedIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        if (dataSource.getFSTIndex() != null && dataSource.getInvertedIndex() != null) {
+          return new BitmapBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        return new ScanBasedFilterOperator(predicateEvaluator, dataSource, numDocs, nullHandlingEnabled);
+      } else {
+        if (dataSource.getDataSourceMetadata().isSorted() && dataSource.getDictionary() != null) {
+          return new SortedIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        if (dataSource.getInvertedIndex() != null) {
+          return new BitmapBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        if (RangeIndexBasedFilterOperator.canEvaluate(predicateEvaluator, dataSource)) {
+          return new RangeIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        return new ScanBasedFilterOperator(predicateEvaluator, dataSource, numDocs, nullHandlingEnabled);
+      }
+    }
+
+    @Override
+    public BaseFilterOperator getAndFilterOperator(QueryContext queryContext, List<BaseFilterOperator> filterOperators,
+        int numDocs) {
+      List<BaseFilterOperator> childFilterOperators = new ArrayList<>(filterOperators.size());
+      for (BaseFilterOperator filterOperator : filterOperators) {
+        if (filterOperator.isResultEmpty()) {
+          return EmptyFilterOperator.getInstance();
+        } else if (!filterOperator.isResultMatchingAll()) {
+          childFilterOperators.add(filterOperator);
+        }
+      }
+      int numChildFilterOperators = childFilterOperators.size();
+      if (numChildFilterOperators == 0) {
+        // Return match all filter operator if all child filter operators match all records
+        return new MatchAllFilterOperator(numDocs);
+      } else if (numChildFilterOperators == 1) {
+        // Return the child filter operator if only one left
+        return childFilterOperators.get(0);
+      } else {
+        // Return the AND filter operator with re-ordered child filter operators
+        reorderAndFilterChildOperators(queryContext, childFilterOperators);
+        return new AndFilterOperator(childFilterOperators, queryContext.getQueryOptions());
+      }
+    }
+
+    @Override
+    public BaseFilterOperator getOrFilterOperator(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators, int numDocs) {
+      List<BaseFilterOperator> childFilterOperators = new ArrayList<>(filterOperators.size());
+      for (BaseFilterOperator filterOperator : filterOperators) {
+        if (filterOperator.isResultMatchingAll()) {
+          return new MatchAllFilterOperator(numDocs);
+        } else if (!filterOperator.isResultEmpty()) {
+          childFilterOperators.add(filterOperator);
+        }
+      }
+      int numChildFilterOperators = childFilterOperators.size();
+      if (numChildFilterOperators == 0) {
+        // Return empty filter operator if all child filter operators's result is empty
+        return EmptyFilterOperator.getInstance();
+      } else if (numChildFilterOperators == 1) {
+        // Return the child filter operator if only one left
+        return childFilterOperators.get(0);
+      } else {
+        // Return the OR filter operator with child filter operators
+        return new OrFilterOperator(childFilterOperators, numDocs);
+      }
+    }
+
+    @Override
+    public BaseFilterOperator getNotFilterOperator(QueryContext queryContext, BaseFilterOperator filterOperator,
+        int numDocs) {
+      if (filterOperator.isResultMatchingAll()) {
+        return EmptyFilterOperator.getInstance();
+      } else if (filterOperator.isResultEmpty()) {
+        return new MatchAllFilterOperator(numDocs);
+      }
+
+      return new NotFilterOperator(filterOperator, numDocs);
+    }
+
+    @Override
+    public void reorderAndFilterChildOperators(QueryContext queryContext, List<BaseFilterOperator> filterOperators) {
+      filterOperators.sort(new Comparator<BaseFilterOperator>() {
+        @Override
+        public int compare(BaseFilterOperator o1, BaseFilterOperator o2) {
+          return getPriority(o1) - getPriority(o2);
+        }
+
+        int getPriority(BaseFilterOperator filterOperator) {
+          if (filterOperator instanceof SortedIndexBasedFilterOperator) {
+            return 0;
+          }
+          if (filterOperator instanceof BitmapBasedFilterOperator) {
+            return 1;
+          }
+          if (filterOperator instanceof RangeIndexBasedFilterOperator
+              || filterOperator instanceof TextContainsFilterOperator
+              || filterOperator instanceof TextMatchFilterOperator || filterOperator instanceof JsonMatchFilterOperator
+              || filterOperator instanceof H3IndexFilterOperator
+              || filterOperator instanceof H3InclusionIndexFilterOperator) {
+            return 2;
+          }
+          if (filterOperator instanceof AndFilterOperator) {
+            return 3;
+          }
+          if (filterOperator instanceof OrFilterOperator) {
+            return 4;
+          }
+          if (filterOperator instanceof NotFilterOperator) {
+            return getPriority(((NotFilterOperator) filterOperator).getChildFilterOperator());
+          }
+          if (filterOperator instanceof ScanBasedFilterOperator) {
+            return getScanBasedFilterPriority(queryContext, (ScanBasedFilterOperator) filterOperator, 5);
+          }
+          if (filterOperator instanceof ExpressionFilterOperator) {
+            return 10;
+          }
+          throw new IllegalStateException(filterOperator.getClass().getSimpleName()
+              + " should not be reordered, remove it from the list before calling this method");
+        }
+      });
+    }
+
+    public static int getScanBasedFilterPriority(QueryContext queryContext,

Review Comment:
   I don't get why we cannot keep this method as `protected`. IMO both `reorderAndFilterChildOperators()` and `getScanBasedFilterPriority()` should be protected



-- 
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] gortiz commented on a diff in pull request #10371: add the ability to change FilterOperatorUtils code

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/FilterOperatorUtils.java:
##########
@@ -28,9 +28,207 @@
 
 
 public class FilterOperatorUtils {
+
+  private static volatile Implementation _instance = new Implementation();

Review Comment:
   It doesn't. Volatile is expensive in x86 if and only if there are writes and reads. In a scenario like this one, where the attribute will be changed once at startup. Also, this code is not in the hotpath (row execution) but in the planning phase. There are quite more expensive costs like our heavily usage of allocation in the planning phase.
   
   Anyway, we can remove it if you prefer that.
   
   References: 
   * https://stackoverflow.com/questions/4633866/is-volatile-expensive



-- 
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 #10371: add the ability to change FilterOperatorUtils code

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/FilterOperatorUtils.java:
##########
@@ -28,9 +28,207 @@
 
 
 public class FilterOperatorUtils {
+
+  private static volatile Implementation _instance = new Implementation();

Review Comment:
   This is good learning for me. I thought reading volatile is always from main memory, but seems it can be read from L1 cache as long as the value is not written to for most modern hardware



-- 
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 #10371: add the ability to change FilterOperatorUtils code

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

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/10371?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 [#10371](https://codecov.io/gh/apache/pinot/pull/10371?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ab8b19f) into [master](https://codecov.io/gh/apache/pinot/commit/9c7e771d22a0cbd3391593e9976669fff4d4b442?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9c7e771) will **decrease** coverage by `0.01%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #10371      +/-   ##
   ============================================
   - Coverage     13.76%   13.76%   -0.01%     
     Complexity      231      231              
   ============================================
     Files          1979     1979              
     Lines        107729   107737       +8     
     Branches      16457    16460       +3     
   ============================================
   - Hits          14831    14825       -6     
   - Misses        91717    91731      +14     
     Partials       1181     1181              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests2 | `13.76% <0.00%> (-0.01%)` | :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/10371?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...inot/core/operator/filter/FilterOperatorUtils.java](https://codecov.io/gh/apache/pinot/pull/10371?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9maWx0ZXIvRmlsdGVyT3BlcmF0b3JVdGlscy5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...lix/core/realtime/PinotRealtimeSegmentManager.java](https://codecov.io/gh/apache/pinot/pull/10371?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL3JlYWx0aW1lL1Bpbm90UmVhbHRpbWVTZWdtZW50TWFuYWdlci5qYXZh) | `72.50% <0.00%> (-2.00%)` | :arrow_down: |
   | [...ntroller/helix/core/minion/TaskMetricsEmitter.java](https://codecov.io/gh/apache/pinot/pull/10371?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9UYXNrTWV0cmljc0VtaXR0ZXIuamF2YQ==) | `86.53% <0.00%> (-1.93%)` | :arrow_down: |
   | [...lix/core/minion/PinotHelixTaskResourceManager.java](https://codecov.io/gh/apache/pinot/pull/10371?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9QaW5vdEhlbGl4VGFza1Jlc291cmNlTWFuYWdlci5qYXZh) | `28.63% <0.00%> (-1.12%)` | :arrow_down: |
   | [...ces/PinotSegmentUploadDownloadRestletResource.java](https://codecov.io/gh/apache/pinot/pull/10371?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90U2VnbWVudFVwbG9hZERvd25sb2FkUmVzdGxldFJlc291cmNlLmphdmE=) | `41.07% <0.00%> (+2.07%)` | :arrow_up: |
   
   :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] gortiz commented on pull request #10371: add the ability to change FilterOperatorUtils code

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

   > To minimize the change, we can still keep the FilterOperatorUtils and all the static util function, but also make an interface (basically add an interface for the methods, and make this class the default implementation for the interface). I'd imagine you want to plug in some other implementation via the setImplementation(), instead of extending this class, you may implement the interface
   
   Isn't that almost the same as I did? The only difference is whether the _thing_ is a class or an interface. I can change it (and I guess I will), but TBH it doesn't seem something that important. The only difference is whether implementation can extend other classes, which I don't think that will be that common (and in that case it could be changed when needed!)


-- 
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] gortiz commented on a diff in pull request #10371: add the ability to change FilterOperatorUtils code

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/FilterOperatorUtils.java:
##########
@@ -28,9 +28,218 @@
 
 
 public class FilterOperatorUtils {
+
+  private static Implementation _instance = new DefaultImplementation();
+
   private FilterOperatorUtils() {
   }
 
+  public static void setImplementation(Implementation newImplementation) {
+    _instance = newImplementation;
+  }
+
+  public interface Implementation {
+
+    /**
+     * Returns the leaf filter operator (i.e. not {@link AndFilterOperator} or {@link OrFilterOperator}).
+     */
+    BaseFilterOperator getLeafFilterOperator(PredicateEvaluator predicateEvaluator, DataSource dataSource,
+        int numDocs, boolean nullHandlingEnabled);
+
+    /**
+     * Returns the AND filter operator or equivalent filter operator.
+     */
+    BaseFilterOperator getAndFilterOperator(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators, int numDocs);
+
+    /**
+     * Returns the OR filter operator or equivalent filter operator.
+     */
+    BaseFilterOperator getOrFilterOperator(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators, int numDocs);
+
+    /**
+     * Returns the NOT filter operator or equivalent filter operator.
+     */
+    BaseFilterOperator getNotFilterOperator(QueryContext queryContext, BaseFilterOperator filterOperator,
+        int numDocs);
+
+    /**
+     * For AND filter operator, reorders its child filter operators based on the their cost and puts the ones with
+     * inverted index first in order to reduce the number of documents to be processed.
+     * <p>Special filter operators such as {@link MatchAllFilterOperator} and {@link EmptyFilterOperator} should be
+     * removed from the list before calling this method.
+     */
+    void reorderAndFilterChildOperators(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators);
+  }
+
+  public static class DefaultImplementation implements Implementation {
+    @Override
+    public BaseFilterOperator getLeafFilterOperator(PredicateEvaluator predicateEvaluator, DataSource dataSource,
+        int numDocs, boolean nullHandlingEnabled) {
+      if (predicateEvaluator.isAlwaysFalse()) {
+        return EmptyFilterOperator.getInstance();
+      } else if (predicateEvaluator.isAlwaysTrue()) {
+        return new MatchAllFilterOperator(numDocs);
+      }
+
+      // Currently sorted index based filtering is supported only for
+      // dictionary encoded columns. The on-disk segment metadata
+      // will indicate if the column is sorted or not regardless of
+      // whether it is raw or dictionary encoded. Here when creating
+      // the filter operator, we need to make sure that sort filter
+      // operator is used only if the column is sorted and has dictionary.
+      Predicate.Type predicateType = predicateEvaluator.getPredicateType();
+      if (predicateType == Predicate.Type.RANGE) {
+        if (dataSource.getDataSourceMetadata().isSorted() && dataSource.getDictionary() != null) {
+          return new SortedIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        if (RangeIndexBasedFilterOperator.canEvaluate(predicateEvaluator, dataSource)) {
+          return new RangeIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        return new ScanBasedFilterOperator(predicateEvaluator, dataSource, numDocs, nullHandlingEnabled);
+      } else if (predicateType == Predicate.Type.REGEXP_LIKE) {
+        if (dataSource.getFSTIndex() != null && dataSource.getDataSourceMetadata().isSorted()) {
+          return new SortedIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        if (dataSource.getFSTIndex() != null && dataSource.getInvertedIndex() != null) {
+          return new BitmapBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        return new ScanBasedFilterOperator(predicateEvaluator, dataSource, numDocs, nullHandlingEnabled);
+      } else {
+        if (dataSource.getDataSourceMetadata().isSorted() && dataSource.getDictionary() != null) {
+          return new SortedIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        if (dataSource.getInvertedIndex() != null) {
+          return new BitmapBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        if (RangeIndexBasedFilterOperator.canEvaluate(predicateEvaluator, dataSource)) {
+          return new RangeIndexBasedFilterOperator(predicateEvaluator, dataSource, numDocs);
+        }
+        return new ScanBasedFilterOperator(predicateEvaluator, dataSource, numDocs, nullHandlingEnabled);
+      }
+    }
+
+    @Override
+    public BaseFilterOperator getAndFilterOperator(QueryContext queryContext, List<BaseFilterOperator> filterOperators,
+        int numDocs) {
+      List<BaseFilterOperator> childFilterOperators = new ArrayList<>(filterOperators.size());
+      for (BaseFilterOperator filterOperator : filterOperators) {
+        if (filterOperator.isResultEmpty()) {
+          return EmptyFilterOperator.getInstance();
+        } else if (!filterOperator.isResultMatchingAll()) {
+          childFilterOperators.add(filterOperator);
+        }
+      }
+      int numChildFilterOperators = childFilterOperators.size();
+      if (numChildFilterOperators == 0) {
+        // Return match all filter operator if all child filter operators match all records
+        return new MatchAllFilterOperator(numDocs);
+      } else if (numChildFilterOperators == 1) {
+        // Return the child filter operator if only one left
+        return childFilterOperators.get(0);
+      } else {
+        // Return the AND filter operator with re-ordered child filter operators
+        reorderAndFilterChildOperators(queryContext, childFilterOperators);
+        return new AndFilterOperator(childFilterOperators, queryContext.getQueryOptions());
+      }
+    }
+
+    @Override
+    public BaseFilterOperator getOrFilterOperator(QueryContext queryContext,
+        List<BaseFilterOperator> filterOperators, int numDocs) {
+      List<BaseFilterOperator> childFilterOperators = new ArrayList<>(filterOperators.size());
+      for (BaseFilterOperator filterOperator : filterOperators) {
+        if (filterOperator.isResultMatchingAll()) {
+          return new MatchAllFilterOperator(numDocs);
+        } else if (!filterOperator.isResultEmpty()) {
+          childFilterOperators.add(filterOperator);
+        }
+      }
+      int numChildFilterOperators = childFilterOperators.size();
+      if (numChildFilterOperators == 0) {
+        // Return empty filter operator if all child filter operators's result is empty
+        return EmptyFilterOperator.getInstance();
+      } else if (numChildFilterOperators == 1) {
+        // Return the child filter operator if only one left
+        return childFilterOperators.get(0);
+      } else {
+        // Return the OR filter operator with child filter operators
+        return new OrFilterOperator(childFilterOperators, numDocs);
+      }
+    }
+
+    @Override
+    public BaseFilterOperator getNotFilterOperator(QueryContext queryContext, BaseFilterOperator filterOperator,
+        int numDocs) {
+      if (filterOperator.isResultMatchingAll()) {
+        return EmptyFilterOperator.getInstance();
+      } else if (filterOperator.isResultEmpty()) {
+        return new MatchAllFilterOperator(numDocs);
+      }
+
+      return new NotFilterOperator(filterOperator, numDocs);
+    }
+
+    @Override
+    public void reorderAndFilterChildOperators(QueryContext queryContext, List<BaseFilterOperator> filterOperators) {
+      filterOperators.sort(new Comparator<BaseFilterOperator>() {
+        @Override
+        public int compare(BaseFilterOperator o1, BaseFilterOperator o2) {
+          return getPriority(o1) - getPriority(o2);
+        }
+
+        int getPriority(BaseFilterOperator filterOperator) {
+          if (filterOperator instanceof SortedIndexBasedFilterOperator) {
+            return 0;
+          }
+          if (filterOperator instanceof BitmapBasedFilterOperator) {
+            return 1;
+          }
+          if (filterOperator instanceof RangeIndexBasedFilterOperator
+              || filterOperator instanceof TextContainsFilterOperator
+              || filterOperator instanceof TextMatchFilterOperator || filterOperator instanceof JsonMatchFilterOperator
+              || filterOperator instanceof H3IndexFilterOperator
+              || filterOperator instanceof H3InclusionIndexFilterOperator) {
+            return 2;
+          }
+          if (filterOperator instanceof AndFilterOperator) {
+            return 3;
+          }
+          if (filterOperator instanceof OrFilterOperator) {
+            return 4;
+          }
+          if (filterOperator instanceof NotFilterOperator) {
+            return getPriority(((NotFilterOperator) filterOperator).getChildFilterOperator());
+          }
+          if (filterOperator instanceof ScanBasedFilterOperator) {
+            return getScanBasedFilterPriority(queryContext, (ScanBasedFilterOperator) filterOperator, 5);
+          }
+          if (filterOperator instanceof ExpressionFilterOperator) {
+            return 10;
+          }
+          throw new IllegalStateException(filterOperator.getClass().getSimpleName()
+              + " should not be reordered, remove it from the list before calling this method");
+        }
+      });
+    }
+
+    public static int getScanBasedFilterPriority(QueryContext queryContext,

Review Comment:
   Yes, we can do it given that it is a class.



-- 
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] gortiz commented on pull request #10371: add the ability to change FilterOperatorUtils code

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

   @saurabhd336 @Jackie-Jiang please review


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

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 #10371: add the ability to change FilterOperatorUtils code

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/FilterOperatorUtils.java:
##########
@@ -28,9 +28,207 @@
 
 
 public class FilterOperatorUtils {
+
+  private static volatile Implementation _instance = new Implementation();
+
   private FilterOperatorUtils() {
   }
 
+  public static void setImplementation(Implementation newImplementation) {
+    _instance = newImplementation;
+  }
+
+  public static class Implementation {

Review Comment:
   Let's introduce an interface, and make this the default implementation



-- 
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] gortiz commented on a diff in pull request #10371: add the ability to change FilterOperatorUtils code

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/FilterOperatorUtils.java:
##########
@@ -28,9 +28,207 @@
 
 
 public class FilterOperatorUtils {
+
+  private static volatile Implementation _instance = new Implementation();
+
   private FilterOperatorUtils() {
   }
 
+  public static void setImplementation(Implementation newImplementation) {
+    _instance = newImplementation;
+  }
+
+  public static class Implementation {

Review Comment:
   I would love to do that, but... how would we consume the interface from other classes? Right now we call the a static method, so we don't need any reference. If we use an interface here, we would need to supply the instance to all callers (like `FilterPlanNode`).
   
   Do you have an idea on how to do that without adding a new argument in the constructor? As said in other PRs, this is not a new problem. We may need to start thinking about introducing a dependency injection system like Guice in Pinot, as it would be less aggressive in the code than adding so many changes on the constructors.



-- 
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] gortiz commented on pull request #10371: add the ability to change FilterOperatorUtils code

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

   Even I'm don't like the interface + default implementation change at all, I've change it in order to be able to merge this.


-- 
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 #10371: add the ability to change FilterOperatorUtils code

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/FilterOperatorUtils.java:
##########
@@ -28,9 +28,207 @@
 
 
 public class FilterOperatorUtils {
+
+  private static volatile Implementation _instance = new Implementation();
+
   private FilterOperatorUtils() {
   }
 
+  public static void setImplementation(Implementation newImplementation) {
+    _instance = newImplementation;
+  }
+
+  public static class Implementation {

Review Comment:
   To minimize the change, we can still keep the `FilterOperatorUtils` and all the static util function, but also make an interface (basically add an interface for the methods, and make this class the default implementation for the interface). I'd imagine you want to plug in some other implementation via the `setImplementation()`, instead of extending this class, you may implement the interface



##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/FilterOperatorUtils.java:
##########
@@ -28,9 +28,207 @@
 
 
 public class FilterOperatorUtils {
+
+  private static volatile Implementation _instance = new Implementation();

Review Comment:
   `volatile` might introduce extra overhead since the util will be accessed by multiple threads. We will always plug-in the implementation at service start time, so it should be okay to remove the `volatile`



-- 
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 #10371: add the ability to change FilterOperatorUtils code

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


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