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

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


The following commit(s) were added to refs/heads/master by this push:
     new f2fa4e5  [CARBONDATA-3593] Add BLOCKLET_SCANNED_NUM and fix 
TOTAL_BLOCKLET_NUM not right when blocklet filtered
f2fa4e5 is described below

commit f2fa4e5c57a1c4004f82a40995b29351b41a5950
Author: 沈洪 <yuling...@alipay.com>
AuthorDate: Fri Nov 22 21:00:16 2019 +0800

    [CARBONDATA-3593] Add BLOCKLET_SCANNED_NUM and fix TOTAL_BLOCKLET_NUM not 
right when blocklet filtered
    
    Add BLOCKLET_SCANNED_NUM and fix TOTAL_BLOCKLET_NUM not right when blocklet 
filtered.
    
    This closes #3476
---
 .../AbstractDetailQueryResultIterator.java         |  5 ++++
 .../scan/scanner/impl/BlockletFilterScanner.java   | 34 +++++++++++++++++-----
 .../core/stats/QueryStatisticsConstants.java       |  2 ++
 .../carbondata/core/stats/TaskStatistics.java      |  1 +
 4 files changed, 34 insertions(+), 8 deletions(-)

diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
index fa97d7a..25ba9a0 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
@@ -245,6 +245,11 @@ public abstract class AbstractDetailQueryResultIterator<E> 
extends CarbonIterato
         .put(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM, 
queryStatisticValidScanBlocklet);
     
queryStatisticsModel.getRecorder().recordStatistics(queryStatisticValidScanBlocklet);
 
+    QueryStatistic scannedBlocklets = new QueryStatistic();
+    queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .put(QueryStatisticsConstants.BLOCKLET_SCANNED_NUM, scannedBlocklets);
+    queryStatisticsModel.getRecorder().recordStatistics(scannedBlocklets);
+
     QueryStatistic totalNumberOfPages = new QueryStatistic();
     queryStatisticsModel.getStatisticsTypeAndObjMap()
         .put(QueryStatisticsConstants.TOTAL_PAGE_SCANNED, totalNumberOfPages);
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
index ca1dc15..347b9ce 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
@@ -113,6 +113,10 @@ public class BlockletFilterScanner extends 
BlockletFullScanner {
         .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
     
totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
         totalPagesScanned.getCount() + dataBlock.numberOfPages());
+    QueryStatistic totalBlockletStatistic = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
+    
totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
+        totalBlockletStatistic.getCount() + 1);
     // apply min max
     if (isMinMaxEnabled) {
       if (null == dataBlock.getColumnsMaxValue()
@@ -168,10 +172,6 @@ public class BlockletFilterScanner extends 
BlockletFullScanner {
   private BlockletScannedResult executeFilter(RawBlockletColumnChunks 
rawBlockletColumnChunks)
       throws FilterUnsupportedException, IOException {
     long startTime = System.currentTimeMillis();
-    QueryStatistic totalBlockletStatistic = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
-    
totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
-        totalBlockletStatistic.getCount() + 1);
     // set the indexed data if it has any during fgdatamap pruning.
     BitSetGroup fgBitSetGroup = 
rawBlockletColumnChunks.getDataBlock().getIndexedData();
     rawBlockletColumnChunks.setBitSetGroup(fgBitSetGroup);
@@ -188,6 +188,11 @@ public class BlockletFilterScanner extends 
BlockletFullScanner {
       scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
           scanTime.getCount() + (System.currentTimeMillis() - startTime));
 
+      QueryStatistic scannedBlocklets = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
+          .get(QueryStatisticsConstants.BLOCKLET_SCANNED_NUM);
+      
scannedBlocklets.addCountStatistic(QueryStatisticsConstants.BLOCKLET_SCANNED_NUM,
+          scannedBlocklets.getCount() + 1);
+
       QueryStatistic scannedPages = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
           .get(QueryStatisticsConstants.PAGE_SCANNED);
       scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
@@ -211,6 +216,12 @@ public class BlockletFilterScanner extends 
BlockletFullScanner {
         .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
     validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
         validPages.getCount() + bitSetGroup.getValidPages());
+
+    QueryStatistic scannedBlocklets = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.BLOCKLET_SCANNED_NUM);
+    
scannedBlocklets.addCountStatistic(QueryStatisticsConstants.BLOCKLET_SCANNED_NUM,
+        scannedBlocklets.getCount() + 1);
+
     QueryStatistic scannedPages = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
         .get(QueryStatisticsConstants.PAGE_SCANNED);
     scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
@@ -351,10 +362,6 @@ public class BlockletFilterScanner extends 
BlockletFullScanner {
       RawBlockletColumnChunks rawBlockletColumnChunks)
       throws FilterUnsupportedException, IOException {
     long startTime = System.currentTimeMillis();
-    QueryStatistic totalBlockletStatistic = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
-    
totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
-        totalBlockletStatistic.getCount() + 1);
     // apply filter on actual data, for each page
     BitSet pages = this.filterExecuter.prunePages(rawBlockletColumnChunks);
     // if filter result is empty then return with empty result
@@ -367,6 +374,11 @@ public class BlockletFilterScanner extends 
BlockletFullScanner {
       scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
           scanTime.getCount() + (System.currentTimeMillis() - startTime));
 
+      QueryStatistic scannedBlocklets = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
+          .get(QueryStatisticsConstants.BLOCKLET_SCANNED_NUM);
+      
scannedBlocklets.addCountStatistic(QueryStatisticsConstants.BLOCKLET_SCANNED_NUM,
+          scannedBlocklets.getCount() + 1);
+
       QueryStatistic scannedPages = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
           .get(QueryStatisticsConstants.PAGE_SCANNED);
       scannedPages
@@ -388,6 +400,12 @@ public class BlockletFilterScanner extends 
BlockletFullScanner {
         .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
     validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
         validPages.getCount() + pages.cardinality());
+
+    QueryStatistic scannedBlocklets = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.BLOCKLET_SCANNED_NUM);
+    
scannedBlocklets.addCountStatistic(QueryStatisticsConstants.BLOCKLET_SCANNED_NUM,
+        scannedBlocklets.getCount() + 1);
+
     QueryStatistic scannedPages = 
queryStatisticsModel.getStatisticsTypeAndObjMap()
         .get(QueryStatisticsConstants.PAGE_SCANNED);
     scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
diff --git 
a/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsConstants.java
index 8a52294..31d9f93 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsConstants.java
@@ -52,6 +52,8 @@ public interface QueryStatisticsConstants {
 
   String VALID_SCAN_BLOCKLET_NUM = "The num of valid scanned blocklet";
 
+  String BLOCKLET_SCANNED_NUM = "The num of blocklet scanned";
+
   String VALID_PAGE_SCANNED = "The number of valid page scanned";
 
   String TOTAL_PAGE_SCANNED = "The number of total page scanned";
diff --git 
a/core/src/main/java/org/apache/carbondata/core/stats/TaskStatistics.java 
b/core/src/main/java/org/apache/carbondata/core/stats/TaskStatistics.java
index e63da18..2681f23 100644
--- a/core/src/main/java/org/apache/carbondata/core/stats/TaskStatistics.java
+++ b/core/src/main/java/org/apache/carbondata/core/stats/TaskStatistics.java
@@ -41,6 +41,7 @@ public class TaskStatistics implements Serializable {
       new Column("carbon_IO_time", 
QueryStatisticsConstants.READ_BLOCKlET_TIME),
       new Column("scan_blocks_num", QueryStatisticsConstants.SCAN_BLOCKS_NUM),
       new Column("total_blocklets", 
QueryStatisticsConstants.TOTAL_BLOCKLET_NUM),
+      new Column("scanned_blocklets", 
QueryStatisticsConstants.BLOCKLET_SCANNED_NUM),
       new Column("valid_blocklets", 
QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM),
       new Column("total_pages", QueryStatisticsConstants.TOTAL_PAGE_SCANNED),
       new Column("scanned_pages", QueryStatisticsConstants.PAGE_SCANNED),

Reply via email to