You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2018/10/09 15:50:39 UTC

[37/45] carbondata git commit: [CARBONDATA-2983][BloomDataMap] Change bloom query model to proceed multiple filter values

[CARBONDATA-2983][BloomDataMap] Change bloom query model to proceed multiple filter values

This PR is supposed to optimize the pruning for InExpression.
For a expression like colA in (1, 2, 3).
Previously it will create 3 bloom query model and for each query model it will iterate all the bloomfilters; But now we only need to generate 1 bloom query model.

Also this PR does an intersection for the pruned result generated by each expression. This optimization is important if we have multiple index columns and used them in one query.

This closes #2781


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/396c26f5
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/396c26f5
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/396c26f5

Branch: refs/heads/branch-1.5
Commit: 396c26f531192e3da8a233ca14024dd8899da52a
Parents: 3edea12
Author: Manhua <ke...@qq.com>
Authored: Fri Sep 28 11:39:39 2018 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Oct 4 18:13:12 2018 +0530

----------------------------------------------------------------------
 .../datamap/bloom/BloomCoarseGrainDataMap.java  | 98 ++++++++++++--------
 1 file changed, 59 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/396c26f5/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
index ee71142..a5376be 100644
--- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
+++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
@@ -21,15 +21,7 @@ import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TimeZone;
+import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
@@ -47,7 +39,6 @@ import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -171,7 +162,7 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
   @Override
   public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
       List<PartitionSpec> partitions) throws IOException {
-    Set<Blocklet> hitBlocklets = new HashSet<>();
+    Set<Blocklet> hitBlocklets = null;
     if (filterExp == null) {
       // null is different from empty here. Empty means after pruning, no blocklet need to scan.
       return null;
@@ -185,6 +176,7 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
       throw new RuntimeException(e);
     }
     for (BloomQueryModel bloomQueryModel : bloomQueryModels) {
+      Set<Blocklet> tempHitBlockletsResult = new HashSet<>();
       LOGGER.debug("prune blocklet for query: " + bloomQueryModel);
       BloomCacheKeyValue.CacheKey cacheKey = new BloomCacheKeyValue.CacheKey(
           this.indexPath.toString(), bloomQueryModel.columnName);
@@ -195,17 +187,32 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
           // skip shard which has been pruned in Main datamap
           continue;
         }
-        boolean scanRequired = bloomFilter.membershipTest(new Key(bloomQueryModel.filterValue));
+        boolean scanRequired = false;
+        for (byte[] value: bloomQueryModel.filterValues) {
+          scanRequired = bloomFilter.membershipTest(new Key(value));
+          if (scanRequired) {
+            // if any filter value hit this bloomfilter
+            // no need to check other filter values
+            break;
+          }
+        }
         if (scanRequired) {
           LOGGER.debug(String.format("BloomCoarseGrainDataMap: Need to scan -> blocklet#%s",
               String.valueOf(bloomFilter.getBlockletNo())));
           Blocklet blocklet = new Blocklet(bloomFilter.getShardName(),
-                  String.valueOf(bloomFilter.getBlockletNo()));
-          hitBlocklets.add(blocklet);
+              String.valueOf(bloomFilter.getBlockletNo()));
+          tempHitBlockletsResult.add(blocklet);
         } else {
           LOGGER.debug(String.format("BloomCoarseGrainDataMap: Skip scan -> blocklet#%s",
               String.valueOf(bloomFilter.getBlockletNo())));
         }
+        // get intersect result between query models
+        // pre-condition: only And/In/EqualTo expression exists in single bloom datamap
+        if (null == hitBlocklets) {
+          hitBlocklets = tempHitBlockletsResult;
+        } else {
+          hitBlocklets.retainAll(tempHitBlockletsResult);
+        }
       }
     }
     return new ArrayList<>(hitBlocklets);
@@ -247,17 +254,17 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
       if (left instanceof ColumnExpression && right instanceof ListExpression) {
         column = ((ColumnExpression) left).getColumnName();
         if (this.name2Col.containsKey(column)) {
-          List<BloomQueryModel> models =
+          BloomQueryModel bloomQueryModel =
               buildQueryModelForIn((ColumnExpression) left, (ListExpression) right);
-          queryModels.addAll(models);
+          queryModels.add(bloomQueryModel);
         }
         return queryModels;
       } else if (left instanceof ListExpression && right instanceof ColumnExpression) {
         column = ((ColumnExpression) right).getColumnName();
         if (this.name2Col.containsKey(column)) {
-          List<BloomQueryModel> models =
+          BloomQueryModel bloomQueryModel =
               buildQueryModelForIn((ColumnExpression) right, (ListExpression) left);
-          queryModels.addAll(models);
+          queryModels.add(bloomQueryModel);
         }
         return queryModels;
       } else {
@@ -274,13 +281,15 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
     return queryModels;
   }
 
-  private BloomQueryModel buildQueryModelForEqual(ColumnExpression ce,
-      LiteralExpression le) throws DictionaryGenerationException, UnsupportedEncodingException {
-    String columnName = ce.getColumnName();
-    DataType dataType = ce.getDataType();
+  /**
+   * Here preprocessed NULL and date/timestamp data type.
+   *
+   * Note that if the datatype is date/timestamp, the expressionValue is long type.
+   */
+  private Object getLiteralExpValue(LiteralExpression le) {
     Object expressionValue = le.getLiteralExpValue();
     Object literalValue;
-    // note that if the datatype is date/timestamp, the expressionValue is long type.
+
     if (null == expressionValue) {
       literalValue = null;
     } else if (le.getLiteralExpDataType() == DataTypes.DATE) {
@@ -288,7 +297,6 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
       // the below settings are set statically according to DateDirectDirectionaryGenerator
       format.setLenient(false);
       format.setTimeZone(TimeZone.getTimeZone("GMT"));
-
       literalValue = format.format(new Date((long) expressionValue / 1000));
     } else if (le.getLiteralExpDataType() == DataTypes.TIMESTAMP) {
       DateFormat format =
@@ -299,28 +307,37 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
     } else {
       literalValue = expressionValue;
     }
+    return literalValue;
+  }
+
 
-    return buildQueryModelInternal(this.name2Col.get(columnName), literalValue, dataType);
+  private BloomQueryModel buildQueryModelForEqual(ColumnExpression ce,
+      LiteralExpression le) throws DictionaryGenerationException, UnsupportedEncodingException {
+    List<byte[]> filterValues = new ArrayList<>();
+    byte[] internalFilterValue = getInternalFilterValue(this.name2Col.get(ce.getColumnName()), le);
+    filterValues.add(internalFilterValue);
+    return new BloomQueryModel(ce.getColumnName(), filterValues);
   }
 
   /**
-   * for `in` expressions, we use `equal` to handle it.
    * Note that `in` operator needs at least one match not exactly match. since while doing pruning,
    * we collect all the blocklets that will match the querymodel, this will not be a problem.
    */
-  private List<BloomQueryModel> buildQueryModelForIn(ColumnExpression ce, ListExpression le)
+  private BloomQueryModel buildQueryModelForIn(ColumnExpression ce, ListExpression le)
       throws DictionaryGenerationException, UnsupportedEncodingException {
-    List<BloomQueryModel> queryModels = new ArrayList<>();
+    List<byte[]> filterValues = new ArrayList<>();
     for (Expression child : le.getChildren()) {
-      queryModels.add(buildQueryModelForEqual(ce, (LiteralExpression) child));
+      byte[] internalFilterValue = getInternalFilterValue(
+          this.name2Col.get(ce.getColumnName()), (LiteralExpression) child);
+      filterValues.add(internalFilterValue);
     }
-    return queryModels;
+    return new BloomQueryModel(ce.getColumnName(), filterValues);
   }
 
-  private BloomQueryModel buildQueryModelInternal(CarbonColumn carbonColumn,
-      Object filterLiteralValue, DataType filterValueDataType) throws
+  private byte[] getInternalFilterValue(CarbonColumn carbonColumn, LiteralExpression le) throws
       DictionaryGenerationException, UnsupportedEncodingException {
-    // convert the filter value to string and apply convertes on it to get carbon internal value
+    Object filterLiteralValue = getLiteralExpValue(le);
+    // convert the filter value to string and apply converters on it to get carbon internal value
     String strFilterValue = null;
     if (null != filterLiteralValue) {
       strFilterValue = String.valueOf(filterLiteralValue);
@@ -362,7 +379,7 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
     if (internalFilterValue.length == 0) {
       internalFilterValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
     }
-    return new BloomQueryModel(carbonColumn.getColName(), internalFilterValue);
+    return internalFilterValue;
   }
 
   @Override
@@ -376,25 +393,28 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
 
   static class BloomQueryModel {
     private String columnName;
-    private byte[] filterValue;
+    private List<byte[]> filterValues;
 
     /**
      * represent an query model will be applyied on bloom index
      *
      * @param columnName bloom index column
-     * @param filterValue key for the bloom index,
+     * @param filterValues key for the bloom index,
      *                   this value is converted from user specified filter value in query
      */
-    private BloomQueryModel(String columnName, byte[] filterValue) {
+    private BloomQueryModel(String columnName, List<byte[]> filterValues) {
       this.columnName = columnName;
-      this.filterValue = filterValue;
+      this.filterValues = filterValues;
     }
 
     @Override
     public String toString() {
       final StringBuilder sb = new StringBuilder("BloomQueryModel{");
       sb.append("columnName='").append(columnName).append('\'');
-      sb.append(", filterValue=").append(Arrays.toString(filterValue));
+      sb.append(", filterValues=");
+      for (byte[] value: filterValues) {
+        sb.append(Arrays.toString(value));
+      }
       sb.append('}');
       return sb.toString();
     }