You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2021/11/01 03:18:51 UTC

[pinot] branch master updated: During segment pruning, release the bloom filter after each segment is processed (#7668)

This is an automated email from the ASF dual-hosted git repository.

jackie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new b2add14  During segment pruning, release the bloom filter after each segment is processed (#7668)
b2add14 is described below

commit b2add149688d28150561c6933df6f753db5173a5
Author: Xiaotian (Jackie) Jiang <17...@users.noreply.github.com>
AuthorDate: Sun Oct 31 20:18:27 2021 -0700

    During segment pruning, release the bloom filter after each segment is processed (#7668)
---
 .../query/pruner/ColumnValueSegmentPruner.java     | 42 +++++++++++-----------
 1 file changed, 22 insertions(+), 20 deletions(-)

diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPruner.java b/pinot-core/src/main/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPruner.java
index cf0aefc..898b157 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPruner.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPruner.java
@@ -26,7 +26,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import javax.annotation.Nullable;
 import org.apache.pinot.common.request.context.ExpressionContext;
 import org.apache.pinot.common.request.context.FilterContext;
 import org.apache.pinot.common.request.context.predicate.EqPredicate;
@@ -130,8 +129,20 @@ public class ColumnValueSegmentPruner implements SegmentPruner {
         // Prune segments
         for (int i = 0; i < numSegments; i++) {
           IndexSegment segment = segments.get(i);
-          if (!pruneSegment(segment, filter, dataSourceCaches[i], fetchContexts[i])) {
-            selectedSegments.add(segment);
+          FetchContext fetchContext = fetchContexts[i];
+          if (fetchContext != null) {
+            segment.acquire(fetchContext);
+            try {
+              if (!pruneSegment(segment, filter, dataSourceCaches[i])) {
+                selectedSegments.add(segment);
+              }
+            } finally {
+              segment.release(fetchContext);
+            }
+          } else {
+            if (!pruneSegment(segment, filter, dataSourceCaches[i])) {
+              selectedSegments.add(segment);
+            }
           }
         }
       } finally {
@@ -146,7 +157,7 @@ public class ColumnValueSegmentPruner implements SegmentPruner {
     } else {
       for (IndexSegment segment : segments) {
         Map<String, DataSource> dataSourceCache = new HashMap<>();
-        if (!pruneSegment(segment, filter, dataSourceCache, null)) {
+        if (!pruneSegment(segment, filter, dataSourceCache)) {
           selectedSegments.add(segment);
         }
       }
@@ -188,19 +199,18 @@ public class ColumnValueSegmentPruner implements SegmentPruner {
     }
   }
 
-  private boolean pruneSegment(IndexSegment segment, FilterContext filter, Map<String, DataSource> dataSourceCache,
-      @Nullable FetchContext fetchContext) {
+  private boolean pruneSegment(IndexSegment segment, FilterContext filter, Map<String, DataSource> dataSourceCache) {
     switch (filter.getType()) {
       case AND:
         for (FilterContext child : filter.getChildren()) {
-          if (pruneSegment(segment, child, dataSourceCache, fetchContext)) {
+          if (pruneSegment(segment, child, dataSourceCache)) {
             return true;
           }
         }
         return false;
       case OR:
         for (FilterContext child : filter.getChildren()) {
-          if (!pruneSegment(segment, child, dataSourceCache, fetchContext)) {
+          if (!pruneSegment(segment, child, dataSourceCache)) {
             return false;
           }
         }
@@ -213,9 +223,9 @@ public class ColumnValueSegmentPruner implements SegmentPruner {
         }
         Predicate.Type predicateType = predicate.getType();
         if (predicateType == Predicate.Type.EQ) {
-          return pruneEqPredicate(segment, (EqPredicate) predicate, dataSourceCache, fetchContext);
+          return pruneEqPredicate(segment, (EqPredicate) predicate, dataSourceCache);
         } else if (predicateType == Predicate.Type.IN) {
-          return pruneInPredicate(segment, (InPredicate) predicate, dataSourceCache, fetchContext);
+          return pruneInPredicate(segment, (InPredicate) predicate, dataSourceCache);
         } else if (predicateType == Predicate.Type.RANGE) {
           return pruneRangePredicate(segment, (RangePredicate) predicate, dataSourceCache);
         } else {
@@ -235,7 +245,7 @@ public class ColumnValueSegmentPruner implements SegmentPruner {
    * </ul>
    */
   private boolean pruneEqPredicate(IndexSegment segment, EqPredicate eqPredicate,
-      Map<String, DataSource> dataSourceCache, @Nullable FetchContext fetchContext) {
+      Map<String, DataSource> dataSourceCache) {
     String column = eqPredicate.getLhs().getIdentifier();
     DataSource dataSource = dataSourceCache.computeIfAbsent(column, segment::getDataSource);
     // NOTE: Column must exist after DataSchemaSegmentPruner
@@ -261,10 +271,6 @@ public class ColumnValueSegmentPruner implements SegmentPruner {
     // Check bloom filter
     BloomFilterReader bloomFilter = dataSource.getBloomFilter();
     if (bloomFilter != null) {
-      // Acquire the bloom filter if it needs to be fetched
-      if (fetchContext != null) {
-        segment.acquire(fetchContext);
-      }
       if (!bloomFilter.mightContain(value.toString())) {
         return true;
       }
@@ -282,7 +288,7 @@ public class ColumnValueSegmentPruner implements SegmentPruner {
    * <p>NOTE: segments will not be pruned if the number of values is greater than the threshold.
    */
   private boolean pruneInPredicate(IndexSegment segment, InPredicate inPredicate,
-      Map<String, DataSource> dataSourceCache, @Nullable FetchContext fetchContext) {
+      Map<String, DataSource> dataSourceCache) {
     String column = inPredicate.getLhs().getIdentifier();
     DataSource dataSource = dataSourceCache.computeIfAbsent(column, segment::getDataSource);
     // NOTE: Column must exist after DataSchemaSegmentPruner
@@ -306,10 +312,6 @@ public class ColumnValueSegmentPruner implements SegmentPruner {
     // Check bloom filter
     BloomFilterReader bloomFilter = dataSource.getBloomFilter();
     if (bloomFilter != null) {
-      // Acquire the bloom filter if it needs to be fetched
-      if (fetchContext != null) {
-        segment.acquire(fetchContext);
-      }
       for (String value : values) {
         if (bloomFilter.mightContain(value)) {
           return false;

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