Repository: carbondata
Updated Branches:
  refs/heads/master 83f9f12a8 -> 22958d941


[CARBONDATA-2929][DataMap] Add block skipped info for explain command

This pr will add block skipped info by counting distinct file path from hit 
blocklet

This closes #2711


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

Branch: refs/heads/master
Commit: 22958d94148ca44c5ce52f0f658e12425c603eb2
Parents: 83f9f12
Author: Manhua <[email protected]>
Authored: Wed Sep 12 11:29:46 2018 +0800
Committer: Jacky Li <[email protected]>
Committed: Wed Sep 12 15:03:03 2018 +0800

----------------------------------------------------------------------
 .../indexstore/blockletindex/BlockDataMap.java  | 11 ++++++
 .../blockletindex/BlockletDataMap.java          | 12 ++++++
 .../core/profiler/ExplainCollector.java         | 22 +++++++++--
 .../core/profiler/TablePruningInfo.java         | 39 +++++++++++++++++---
 .../hadoop/api/CarbonInputFormat.java           | 23 +++++++++---
 .../lucene/LuceneFineGrainDataMapSuite.scala    |  6 +--
 .../TestPreAggregateTableSelection.scala        | 20 +++++-----
 .../bloom/BloomCoarseGrainDataMapSuite.scala    | 24 +++++++++++-
 8 files changed, 126 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/22958d94/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
index 57c92c6..d6e5784 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
@@ -566,6 +566,16 @@ public class BlockDataMap extends CoarseGrainDataMap
     }
   }
 
+  // get total block number in this datamap
+  protected int getTotalBlocks() {
+    if (isLegacyStore) {
+      // dummy value
+      return 0;
+    } else {
+      return memoryDMStore.getRowCount();
+    }
+  }
+
   // get total blocklet number in this datamap
   protected int getTotalBlocklets() {
     if (isLegacyStore) {
@@ -627,6 +637,7 @@ public class BlockDataMap extends CoarseGrainDataMap
     } else {
       ExplainCollector.setShowPruningInfo(true);
       ExplainCollector.addTotalBlocklets(totalBlocklets);
+      ExplainCollector.addTotalBlocks(getTotalBlocks());
       ExplainCollector.addDefaultDataMapPruningHit(hitBlocklets);
     }
     return blocklets;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/22958d94/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
index 683657a..ed6ecdc 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -47,6 +47,8 @@ import org.apache.carbondata.core.util.BlockletDataMapUtil;
 public class BlockletDataMap extends BlockDataMap implements Serializable {
 
   private static final long serialVersionUID = -2170289352240810993L;
+  // total block number in this datamap
+  private int blockNum = 0;
 
   @Override
   public void init(DataMapModel dataMapModel) throws IOException, 
MemoryException {
@@ -128,6 +130,7 @@ public class BlockletDataMap extends BlockDataMap 
implements Serializable {
         if (null == tempFilePath || 
!tempFilePath.equals(blockInfo.getFilePath())) {
           tempFilePath = blockInfo.getFilePath();
           relativeBlockletId = 0;
+          blockNum++;
         }
         summaryRow = loadToUnsafe(schema, taskSummarySchema, fileFooter, 
segmentProperties,
             getMinMaxCacheColumns(), blockInfo.getFilePath(), summaryRow,
@@ -256,6 +259,15 @@ public class BlockletDataMap extends BlockDataMap 
implements Serializable {
   }
 
   @Override
+  protected int getTotalBlocks() {
+    if (isLegacyStore) {
+      return super.getTotalBlocklets();
+    } else {
+      return blockNum;
+    }
+  }
+
+  @Override
   protected int getTotalBlocklets() {
     if (isLegacyStore) {
       return super.getTotalBlocklets();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/22958d94/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java 
b/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
index a86937c..755c56a 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
@@ -108,19 +108,26 @@ public class ExplainCollector {
     }
   }
 
+  public static void setDefaultDataMapPruningBlockHit(int numBlocks) {
+    if (enabled()) {
+      TablePruningInfo scan = getCurrentTablePruningInfo();
+      scan.setNumBlocksAfterDefaultPruning(numBlocks);
+    }
+  }
+
   public static void recordCGDataMapPruning(DataMapWrapperSimpleInfo 
dataMapWrapperSimpleInfo,
-      int numBlocklets) {
+      int numBlocklets, int numBlocks) {
     if (enabled()) {
       TablePruningInfo scan = getCurrentTablePruningInfo();
-      scan.setNumBlockletsAfterCGPruning(dataMapWrapperSimpleInfo, 
numBlocklets);
+      scan.setNumBlockletsAfterCGPruning(dataMapWrapperSimpleInfo, 
numBlocklets, numBlocks);
     }
   }
 
   public static void recordFGDataMapPruning(DataMapWrapperSimpleInfo 
dataMapWrapperSimpleInfo,
-      int numBlocklets) {
+      int numBlocklets, int numBlocks) {
     if (enabled()) {
       TablePruningInfo scan = getCurrentTablePruningInfo();
-      scan.setNumBlockletsAfterFGPruning(dataMapWrapperSimpleInfo, 
numBlocklets);
+      scan.setNumBlockletsAfterFGPruning(dataMapWrapperSimpleInfo, 
numBlocklets, numBlocks);
     }
   }
 
@@ -131,6 +138,13 @@ public class ExplainCollector {
     }
   }
 
+  public static void addTotalBlocks(int numBlocks) {
+    if (enabled()) {
+      TablePruningInfo scan = getCurrentTablePruningInfo();
+      scan.addTotalBlocks(numBlocks);
+    }
+  }
+
   /**
    * Return the current TablePruningInfo (It is the last one in the map, since 
it is in
    * single thread)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/22958d94/core/src/main/java/org/apache/carbondata/core/profiler/TablePruningInfo.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/profiler/TablePruningInfo.java 
b/core/src/main/java/org/apache/carbondata/core/profiler/TablePruningInfo.java
index 187540c..f7d6af5 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/profiler/TablePruningInfo.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/profiler/TablePruningInfo.java
@@ -26,18 +26,26 @@ import 
org.apache.carbondata.core.datamap.dev.expr.DataMapWrapperSimpleInfo;
 @InterfaceAudience.Internal
 public class TablePruningInfo {
 
+  private int totalBlocks;
   private int totalBlocklets;
   private String filterStatement;
   private boolean showPruningInfo;
 
+  private int numBlocksAfterDefaultPruning;
   private int numBlockletsAfterDefaultPruning = 0;
 
   private DataMapWrapperSimpleInfo cgDataMap;
+  private int numBlocksAfterCGPruning;
   private int numBlockletsAfterCGPruning;
 
   private DataMapWrapperSimpleInfo fgDataMap;
+  private int numBlocksAfterFGPruning;
   private int numBlockletsAfterFGPruning;
 
+  void addTotalBlocks(int numBlocks) {
+    this.totalBlocks += numBlocks;
+  }
+
   void addTotalBlocklets(int numBlocklets) {
     this.totalBlocklets += numBlocklets;
   }
@@ -50,19 +58,30 @@ public class TablePruningInfo {
     this.showPruningInfo = showPruningInfo;
   }
 
+  void setNumBlocksAfterDefaultPruning(int numBlocks) {
+    this.numBlocksAfterDefaultPruning = numBlocks;
+  }
+
+  /**
+   * To get blocklet number no matter what cache level(block/blocklet) it is,
+   * we accumulate blocklet number in default datamap instead of setting it
+   * in CarbonInputFormat
+   */
   void addNumBlockletsAfterDefaultPruning(int numBlocklets) {
     this.numBlockletsAfterDefaultPruning += numBlocklets;
   }
 
   void setNumBlockletsAfterCGPruning(DataMapWrapperSimpleInfo 
dataMapWrapperSimpleInfo,
-      int numBlocklets) {
+      int numBlocklets, int numBlocks) {
     this.cgDataMap = dataMapWrapperSimpleInfo;
+    this.numBlocksAfterCGPruning = numBlocks;
     this.numBlockletsAfterCGPruning = numBlocklets;
   }
 
   void setNumBlockletsAfterFGPruning(DataMapWrapperSimpleInfo 
dataMapWrapperSimpleInfo,
-      int numBlocklets) {
+      int numBlocklets, int numBlocks) {
     this.fgDataMap = dataMapWrapperSimpleInfo;
+    this.numBlocksAfterFGPruning = numBlocks;
     this.numBlockletsAfterFGPruning = numBlocklets;
   }
 
@@ -71,31 +90,39 @@ public class TablePruningInfo {
     if (showPruningInfo) {
       StringBuilder builder = new StringBuilder();
       builder
-          .append(" - total blocklets: ").append(totalBlocklets).append("\n")
+          .append(" - total: ").append(totalBlocks).append(" blocks, ")
+          .append(totalBlocklets).append(" blocklets").append("\n")
           .append(" - filter: ").append(filterStatement).append("\n");
+      int skipBlocks = totalBlocks - numBlocksAfterDefaultPruning;
       int skipBlocklets = totalBlocklets - numBlockletsAfterDefaultPruning;
       builder
           .append(" - pruned by Main DataMap").append("\n")
-          .append("    - skipped blocklets: 
").append(skipBlocklets).append("\n");
+          .append("    - skipped: ").append(skipBlocks).append(" blocks, ")
+          .append(skipBlocklets).append(" blocklets").append("\n");
       if (cgDataMap != null) {
+        skipBlocks = numBlocksAfterDefaultPruning - numBlocksAfterCGPruning;
         skipBlocklets = numBlockletsAfterDefaultPruning - 
numBlockletsAfterCGPruning;
         builder
             .append(" - pruned by CG DataMap").append("\n")
             .append("    - name: 
").append(cgDataMap.getDataMapWrapperName()).append("\n")
             .append("    - provider: 
").append(cgDataMap.getDataMapWrapperProvider()).append("\n")
-            .append("    - skipped blocklets: 
").append(skipBlocklets).append("\n");
+            .append("    - skipped: ").append(skipBlocks).append(" blocks, ")
+            .append(skipBlocklets).append(" blocklets").append("\n");;
       }
       if (fgDataMap != null) {
         if (numBlockletsAfterCGPruning != 0) {
+          skipBlocks = numBlocksAfterCGPruning - numBlocksAfterFGPruning;
           skipBlocklets = numBlockletsAfterCGPruning - 
numBlockletsAfterFGPruning;
         } else {
+          skipBlocks = numBlocksAfterDefaultPruning - numBlocksAfterFGPruning;
           skipBlocklets = numBlockletsAfterDefaultPruning - 
numBlockletsAfterFGPruning;
         }
         builder
             .append(" - pruned by FG DataMap").append("\n")
             .append("    - name: 
").append(fgDataMap.getDataMapWrapperName()).append("\n")
             .append("    - provider: 
").append(fgDataMap.getDataMapWrapperProvider()).append("\n")
-            .append("    - skipped blocklets: 
").append(skipBlocklets).append("\n");
+            .append("    - skipped: ").append(skipBlocks).append(" blocks, ")
+            .append(skipBlocklets).append(" blocklets").append("\n");;
       }
       return builder.toString();
     } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/22958d94/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
index d0dff5a..8183335 100644
--- 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
+++ 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
@@ -21,10 +21,7 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-import java.util.Objects;
+import java.util.*;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonCommonConstantsInternal;
@@ -455,6 +452,18 @@ m filterExpression
   }
 
   /**
+   * for explain command
+   * get number of block by counting distinct file path of blocklets
+   */
+  private int getBlockCount(List<ExtendedBlocklet> blocklets) {
+    Set<String> filepaths = new HashSet<>();
+    for (ExtendedBlocklet blocklet: blocklets) {
+      filepaths.add(blocklet.getPath());
+    }
+    return filepaths.size();
+  }
+
+  /**
    * Prune the blocklets using the filter expression with available datamaps.
    * First pruned with default blocklet datamap, then pruned with CG and FG 
datamaps
    */
@@ -484,6 +493,8 @@ m filterExpression
       prunedBlocklets = defaultDataMap.prune(segmentIds, expression, 
partitionsToPrune);
     }
 
+    
ExplainCollector.setDefaultDataMapPruningBlockHit(getBlockCount(prunedBlocklets));
+
     if (prunedBlocklets.size() == 0) {
       return prunedBlocklets;
     }
@@ -508,7 +519,7 @@ m filterExpression
       prunedBlocklets = intersectFilteredBlocklets(carbonTable, 
prunedBlocklets, cgPrunedBlocklets);
       ExplainCollector.recordCGDataMapPruning(
           DataMapWrapperSimpleInfo.fromDataMapWrapper(cgDataMapExprWrapper),
-          prunedBlocklets.size());
+          prunedBlocklets.size(), getBlockCount(prunedBlocklets));
     }
 
     if (prunedBlocklets.size() == 0) {
@@ -528,7 +539,7 @@ m filterExpression
             fgPrunedBlocklets);
         ExplainCollector.recordFGDataMapPruning(
             DataMapWrapperSimpleInfo.fromDataMapWrapper(fgDataMapExprWrapper),
-            prunedBlocklets.size());
+            prunedBlocklets.size(), getBlockCount(prunedBlocklets));
       }
     }
     return prunedBlocklets;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/22958d94/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
index e7bd366..0c6134b 100644
--- 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
+++ 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
@@ -621,14 +621,14 @@ class LuceneFineGrainDataMapSuite extends QueryTest with 
BeforeAndAfterAll {
       assertResult(
         """== CarbonData Profiler ==
           |Table Scan on main
-          | - total blocklets: 1
+          | - total: 1 blocks, 1 blocklets
           | - filter: TEXT_MATCH('name:bob')
           | - pruned by Main DataMap
-          |    - skipped blocklets: 0
+          |    - skipped: 0 blocks, 0 blocklets
           | - pruned by FG DataMap
           |    - name: dm
           |    - provider: lucene
-          |    - skipped blocklets: 1
+          |    - skipped: 1 blocks, 1 blocklets
           |""".stripMargin)(rows(0).getString(0))
     } finally {
       LuceneFineGrainDataMapSuite.deleteFile(file1)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/22958d94/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
index ebb2491..0dc52ea 100644
--- 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
+++ 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
@@ -386,10 +386,10 @@ class TestPreAggregateTableSelection extends 
SparkQueryTest with BeforeAndAfterA
     assertResult(
       """== CarbonData Profiler ==
         |Table Scan on maintable
-        | - total blocklets: 1
+        | - total: 1 blocks, 1 blocklets
         | - filter: none
         | - pruned by Main DataMap
-        |    - skipped blocklets: 0
+        |    - skipped: 0 blocks, 0 blocklets
         |""".stripMargin)(rows(0).getString(0))
   }
 
@@ -400,10 +400,10 @@ class TestPreAggregateTableSelection extends 
SparkQueryTest with BeforeAndAfterA
         |Query rewrite based on DataMap:
         | - agg1 (preaggregate)
         |Table Scan on maintable_agg1
-        | - total blocklets: 1
+        | - total: 1 blocks, 1 blocklets
         | - filter: none
         | - pruned by Main DataMap
-        |    - skipped blocklets: 0
+        |    - skipped: 0 blocks, 0 blocklets
         |""".stripMargin)(rows(0).getString(0))
   }
 
@@ -415,10 +415,10 @@ class TestPreAggregateTableSelection extends 
SparkQueryTest with BeforeAndAfterA
         |Query rewrite based on DataMap:
         | - agg1 (preaggregate)
         |Table Scan on maintable_agg1
-        | - total blocklets: 1
+        | - total: 1 blocks, 1 blocklets
         | - filter: (maintable_name <> null and maintable_name = a)
         | - pruned by Main DataMap
-        |    - skipped blocklets: 1
+        |    - skipped: 1 blocks, 1 blocklets
         |""".stripMargin)(rows(0).getString(0))
 
   }
@@ -431,17 +431,17 @@ class TestPreAggregateTableSelection extends 
SparkQueryTest with BeforeAndAfterA
     assert(rows(0).getString(0).contains(
       """
         |Table Scan on maintable
-        | - total blocklets: 1
+        | - total: 1 blocks, 1 blocklets
         | - filter: ((id <> null and id < 3) and name <> null)
         | - pruned by Main DataMap
-        |    - skipped blocklets: 0""".stripMargin))
+        |    - skipped: 0 blocks, 0 blocklets""".stripMargin))
     assert(rows(0).getString(0).contains(
       """
         |Table Scan on maintableavg
-        | - total blocklets: 1
+        | - total: 1 blocks, 1 blocklets
         | - filter: name <> null
         | - pruned by Main DataMap
-        |    - skipped blocklets: 0""".stripMargin))
+        |    - skipped: 0 blocks, 0 blocklets""".stripMargin))
 
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/22958d94/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
 
b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
index 7cb841f..0a22937 100644
--- 
a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
+++ 
b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
@@ -719,8 +719,18 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with 
BeforeAndAfterAll with
          | FROM $bloomDMSampleTable
          | WHERE num1 = 1
            """.stripMargin).collect()
+
     assert(explainString(0).getString(0).contains(
-      "- name: datamap2\n    - provider: bloomfilter\n    - skipped blocklets: 
1"))
+      """
+        |Table Scan on carbon_bloom
+        | - total: 3 blocks, 3 blocklets
+        | - filter: (num1 <> null and num1 = 1)
+        | - pruned by Main DataMap
+        |    - skipped: 1 blocks, 1 blocklets
+        | - pruned by CG DataMap
+        |    - name: datamap2
+        |    - provider: bloomfilter
+        |    - skipped: 1 blocks, 1 blocklets""".stripMargin))
 
     explainString = sql(
       s"""
@@ -728,8 +738,18 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with 
BeforeAndAfterAll with
          | FROM $bloomDMSampleTable
          | WHERE dictString = 'S21'
            """.stripMargin).collect()
+
     assert(explainString(0).getString(0).contains(
-      "- name: datamap2\n    - provider: bloomfilter\n    - skipped blocklets: 
0"))
+      """
+        |Table Scan on carbon_bloom
+        | - total: 3 blocks, 3 blocklets
+        | - filter: (dictstring <> null and dictstring = S21)
+        | - pruned by Main DataMap
+        |    - skipped: 1 blocks, 1 blocklets
+        | - pruned by CG DataMap
+        |    - name: datamap2
+        |    - provider: bloomfilter
+        |    - skipped: 0 blocks, 0 blocklets""".stripMargin))
 
   }
 

Reply via email to