Repository: kylin
Updated Branches:
  refs/heads/2.0-rc f91e738a7 -> edab5a433


KYLIN-1301 fix segment pruning failure in 2.x versions


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

Branch: refs/heads/2.0-rc
Commit: edab5a433cb3260065ebddb6869505175db30758
Parents: f91e738
Author: honma <ho...@ebay.com>
Authored: Mon Jan 11 11:25:44 2016 +0800
Committer: honma <ho...@ebay.com>
Committed: Mon Jan 11 11:25:44 2016 +0800

----------------------------------------------------------------------
 .../kylin/gridtable/GTScanRangePlanner.java       | 13 +++++++++++--
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java       |  4 ++++
 .../kylin/storage/hbase/cube/v2/CubeHBaseRPC.java | 18 +++++++++---------
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java   |  5 ++++-
 .../storage/hbase/cube/v2/CubeSegmentScanner.java | 14 ++++++++++----
 5 files changed, 38 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/edab5a43/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRangePlanner.java
----------------------------------------------------------------------
diff --git 
a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRangePlanner.java 
b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRangePlanner.java
index f0e42a0..dad0f7c 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRangePlanner.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRangePlanner.java
@@ -43,7 +43,16 @@ public class GTScanRangePlanner {
     final private RecordComparator rangeEndComparator;
     final private RecordComparator rangeStartEndComparator;
 
+    /**
+     * @param info
+     * @param segmentStartAndEnd in GT encoding
+     * @param partitionColRef the TblColRef in GT
+     */
     public GTScanRangePlanner(GTInfo info, Pair<ByteArray, ByteArray> 
segmentStartAndEnd, TblColRef partitionColRef) {
+        if (partitionColRef != null && segmentStartAndEnd == null) {
+            throw new IllegalArgumentException("segmentStartAndEnd not 
provided when partitionColRef is set to " + partitionColRef);
+        }
+
         this.info = info;
         this.segmentStartAndEnd = segmentStartAndEnd;
         this.partitionColRef = partitionColRef;
@@ -91,9 +100,9 @@ public class GTScanRangePlanner {
         List<GTRecord> fuzzyKeys;
 
         for (ColumnRange range : andDimRanges) {
-
             if (partitionColRef != null && 
range.column.equals(partitionColRef)) {
-
+                logger.debug("Pre-check partition col filter, partitionColRef 
{}, segmentstartandend {}, range begin {}, range end {}",//
+                        new Object[] { partitionColRef, segmentStartAndEnd, 
range.begin, range.end });
                 if 
(rangeStartEndComparator.comparator.compare(segmentStartAndEnd.getFirst(), 
range.end) <= 0 //
                         && 
rangeStartEndComparator.comparator.compare(range.begin, 
segmentStartAndEnd.getSecond()) < 0) {
                     //segment range is [Closed,Open)

http://git-wip-us.apache.org/repos/asf/kylin/blob/edab5a43/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
----------------------------------------------------------------------
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
index 6ff07a4..a193d10 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
@@ -63,9 +63,13 @@ import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.HBaseZeroCopyByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
 
+    public static final Logger logger = 
LoggerFactory.getLogger(CubeHBaseEndpointRPC.class);
+
     private static ExecutorService executorService = 
Executors.newCachedThreadPool();
 
     static class EndpointResultsAsGTScanner implements IGTScanner {

http://git-wip-us.apache.org/repos/asf/kylin/blob/edab5a43/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
----------------------------------------------------------------------
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
index 1335af0..d319c2d 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
@@ -106,7 +106,7 @@ public abstract class CubeHBaseRPC {
                 return shardEnd;
             }
         });
-        
+
         Preconditions.checkState(startKeys.size() == endKeys.size());
         List<Pair<byte[], byte[]>> hbaseFuzzyKeys = 
translateFuzzyKeys(fuzzyKeys);
 
@@ -115,7 +115,7 @@ public abstract class CubeHBaseRPC {
         int hbaseMaxResultSize = config.getHBaseScanMaxResultSize();
         if (isMemoryHungry(selectedColBlocks))
             hbaseCaching /= 10;
-        
+
         for (short i = 0; i < startKeys.size(); ++i) {
             ret.add(new RawScan(startKeys.get(i), endKeys.get(i), 
selectedColumns, hbaseFuzzyKeys, hbaseCaching, hbaseMaxResultSize));
         }
@@ -240,7 +240,7 @@ public abstract class CubeHBaseRPC {
 
     protected void logScan(RawScan rawScan, String tableName) {
         StringBuilder info = new StringBuilder();
-        info.append("\nVisiting hbase table ").append(tableName).append(": ");
+        info.append("Visiting hbase table ").append(tableName).append(": ");
         if (cuboid.requirePostAggregation()) {
             info.append("cuboid require post aggregation, from ");
         } else {
@@ -251,15 +251,15 @@ public abstract class CubeHBaseRPC {
         info.append(cuboid.getId());
         info.append("\nStart: ");
         info.append(rawScan.getStartKeyAsString());
-        info.append(" - ");
-        info.append(Bytes.toStringBinary(rawScan.startKey));
+        info.append(" (");
+        info.append(Bytes.toStringBinary(rawScan.startKey) + ")");
         info.append("\nStop:  ");
         info.append(rawScan.getEndKeyAsString());
-        info.append(" - ");
-        info.append(Bytes.toStringBinary(rawScan.endKey));
-        if (rawScan.fuzzyKeys != null) {
+        info.append(" (");
+        info.append(Bytes.toStringBinary(rawScan.endKey) + ")");
+        if (rawScan.fuzzyKeys != null && rawScan.fuzzyKeys.size() != 0) {
             info.append("\nFuzzy key counts: " + rawScan.fuzzyKeys.size());
-            info.append("\nFuzzy: ");
+            info.append(". Fuzzy keys : ");
             info.append(rawScan.getFuzzyKeyAsString());
         } else {
             info.append("\nNo Fuzzy Key");

http://git-wip-us.apache.org/repos/asf/kylin/blob/edab5a43/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
----------------------------------------------------------------------
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
index 69b95ca..b623a65 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
@@ -19,6 +19,8 @@ import org.apache.kylin.gridtable.GTScanRequest;
 import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.gridtable.IGTStore;
 import org.apache.kylin.storage.hbase.HBaseConnection;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
@@ -27,6 +29,7 @@ import com.google.common.collect.Lists;
  * for test use only
  */
 public class CubeHBaseScanRPC extends CubeHBaseRPC {
+    public static final Logger logger = 
LoggerFactory.getLogger(CubeHBaseScanRPC.class);
 
     static class TrimmedInfoGTRecordAdapter implements Iterable<GTRecord> {
 
@@ -118,7 +121,7 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
             }
         };
 
-        IGTStore store = new HBaseReadonlyStore(cellListIterator, scanRequest, 
rawScans.get(0).hbaseColumns, hbaseColumnsToGT,cubeSeg.getRowKeyPreambleSize());
+        IGTStore store = new HBaseReadonlyStore(cellListIterator, scanRequest, 
rawScans.get(0).hbaseColumns, hbaseColumnsToGT, 
cubeSeg.getRowKeyPreambleSize());
         IGTScanner rawScanner = store.scan(scanRequest);
 
         final IGTScanner decorateScanner = 
scanRequest.decorateScanner(rawScanner);

http://git-wip-us.apache.org/repos/asf/kylin/blob/edab5a43/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeSegmentScanner.java
----------------------------------------------------------------------
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeSegmentScanner.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeSegmentScanner.java
index a1289c1..149a02d 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeSegmentScanner.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeSegmentScanner.java
@@ -37,12 +37,16 @@ import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.model.DataType;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 public class CubeSegmentScanner implements IGTScanner {
 
+    private static final Logger logger = 
LoggerFactory.getLogger(CubeSegmentScanner.class);
+
     private static final int MAX_SCAN_RANGES = 200;
 
     final CubeSegment cubeSeg;
@@ -74,12 +78,14 @@ public class CubeSegmentScanner implements IGTScanner {
         GTScanRangePlanner scanRangePlanner;
         if 
(cubeSeg.getCubeDesc().getModel().getPartitionDesc().isPartitioned()) {
             TblColRef tblColRef = 
cubeSeg.getCubeDesc().getModel().getPartitionDesc().getPartitionDateColumnRef();
-            Pair<ByteArray, ByteArray> segmentStartAndEnd = null;
+            Pair<ByteArray, ByteArray> segmentStartAndEnd;
             int index = mapping.getIndexOf(tblColRef);
             if (index >= 0) {
-                segmentStartAndEnd = getSegmentStartAndEnd(tblColRef, index);
+                segmentStartAndEnd = getSegmentStartAndEnd(index);
+            } else {
+                throw new IllegalStateException("Cannot found partition column 
on cuboid to gt mapping:" + tblColRef);
             }
-            scanRangePlanner = new GTScanRangePlanner(info, 
segmentStartAndEnd, tblColRef);
+            scanRangePlanner = new GTScanRangePlanner(info, 
segmentStartAndEnd, info.colRef(index));
         } else {
             scanRangePlanner = new GTScanRangePlanner(info, null, null);
         }
@@ -96,7 +102,7 @@ public class CubeSegmentScanner implements IGTScanner {
         scanner = new Scanner();
     }
 
-    private Pair<ByteArray, ByteArray> getSegmentStartAndEnd(TblColRef 
tblColRef, int index) {
+    private Pair<ByteArray, ByteArray> getSegmentStartAndEnd(int index) {
         ByteArray start;
         if (cubeSeg.getDateRangeStart() != Long.MIN_VALUE) {
             start = encodeTime(cubeSeg.getDateRangeStart(), index, 1);

Reply via email to