refine v2 query engine log format

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

Branch: refs/heads/KYLIN-1122
Commit: 5d35f97fbd05fa8437d05eb9a53b7a31416e03b2
Parents: f47ae3a
Author: honma <ho...@ebay.com>
Authored: Thu Jan 14 14:22:52 2016 +0800
Committer: Xiaoyu Wang <wangxia...@apache.org>
Committed: Mon Jan 18 13:38:30 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/common/util/ByteArray.java  |  8 ++++++++
 .../apache/kylin/gridtable/GTScanRangePlanner.java    | 12 ++++++++++--
 .../storage/hbase/cube/v2/CubeHBaseEndpointRPC.java   | 14 ++++++++++----
 .../storage/hbase/cube/v2/CubeSegmentScanner.java     |  1 +
 4 files changed, 29 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/5d35f97f/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java
----------------------------------------------------------------------
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java 
b/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java
index 015d5c5..5e35257 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java
@@ -166,6 +166,14 @@ public class ByteArray implements Comparable<ByteArray>, 
Serializable {
             return Bytes.compareTo(this.data, this.offset, this.length, 
o.data, o.offset, o.length);
     }
 
+    public String toReadableText() {
+        if (data == null) {
+            return null;
+        } else {
+            return BytesUtil.toHex(data, offset, length);
+        }
+    }
+
     @Override
     public String toString() {
         if (data == null)

http://git-wip-us.apache.org/repos/asf/kylin/blob/5d35f97f/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 6ff1ab4..a72426d 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
@@ -90,6 +90,14 @@ public class GTScanRangePlanner {
         return mergedRanges;
     }
 
+    private String makeReadable(ByteArray byteArray) {
+        if (byteArray == null) {
+            return null;
+        } else {
+            return byteArray.toReadableText();
+        }
+    }
+
     private GTScanRange newScanRange(Collection<ColumnRange> andDimRanges) {
         GTRecord pkStart = new GTRecord(info);
         GTRecord pkEnd = new GTRecord(info);
@@ -99,12 +107,12 @@ public class GTScanRangePlanner {
 
         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)
                 } else {
+                    logger.debug("Pre-check partition col filter failed, 
partitionColRef {}, segment start {}, segment end {}, range begin {}, range end 
{}",//
+                            new Object[] { partitionColRef, 
makeReadable(segmentStartAndEnd.getFirst()), 
makeReadable(segmentStartAndEnd.getSecond()), makeReadable(range.begin), 
makeReadable(range.end) });
                     return null;
                 }
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/5d35f97f/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 be8c011..1881a23 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
@@ -204,7 +204,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
     public IGTScanner getGTScanner(final List<GTScanRequest> scanRequests) 
throws IOException {
 
         final String toggle = BackdoorToggles.getCoprocessorBehavior() == null 
? CoprocessorBehavior.SCAN_FILTER_AGGR_CHECKMEM.toString() : 
BackdoorToggles.getCoprocessorBehavior();
-        logger.debug("The execution of this query will use " + toggle + " as 
endpoint's behavior");
+        logger.debug("New scanner for current segment {} will use {} as 
endpoint's behavior", cubeSeg, toggle);
 
         short cuboidBaseShard = 
cubeSeg.getCuboidBaseShard(this.cuboid.getId());
         short shardNum = cubeSeg.getCuboidShardNum(this.cuboid.getId());
@@ -225,6 +225,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
             
hbaseColumnsToGTIntList.add(IntList.newBuilder().addAllInts(list).build());
         }
 
+        boolean scanLogged = false;
         for (GTScanRequest req : scanRequests) {
             ByteBuffer buffer = 
ByteBuffer.allocate(BytesSerializer.SERIALIZE_BUFFER_SIZE);
             GTScanRequest.serializer.serialize(req, buffer);
@@ -239,10 +240,15 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
             
rawScanByteStrings.add(HBaseZeroCopyByteString.wrap(rawScanBuffer.array(), 
rawScanBuffer.position(), rawScanBuffer.limit()));
 
             logger.debug("Serialized scanRequestBytes {} bytes, 
rawScanBytesString {} bytes", buffer.limit() - buffer.position(), 
rawScanBuffer.limit() - rawScanBuffer.position());
-            logScan(rawScan, cubeSeg.getStorageLocationIdentifier());
+
+            if (!scanLogged) {
+                logger.info("The scan(s) info for current segment is as below, 
shard part of start/end key is set to 0", cubeSeg);
+                logScan(rawScan, cubeSeg.getStorageLocationIdentifier());
+                scanLogged = true;
+            }
         }
 
-        logger.debug("Start to executing: {} shards starting from {}", 
shardNum, cuboidBaseShard);
+        logger.debug("Submitting rpc to {} shards starting from shard {}, scan 
requests count {}", new Object[] { shardNum, cuboidBaseShard, 
scanRequests.size() });
 
         final AtomicInteger totalScannedCount = new AtomicInteger(0);
         final ExpectedSizeIterator epResultItr = new 
ExpectedSizeIterator(scanRequests.size() * shardNum);
@@ -288,7 +294,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
     private String getStatsString(Map.Entry<byte[], 
CubeVisitProtos.CubeVisitResponse> result) {
         StringBuilder sb = new StringBuilder();
         Stats stats = result.getValue().getStats();
-        sb.append("Shard " + BytesUtil.toHex(result.getKey()) + " on host: " + 
stats.getHostname() + ".");
+        sb.append("Endpoint RPC returned from HTable " + 
cubeSeg.getStorageLocationIdentifier() + " Shard " + 
BytesUtil.toHex(result.getKey()) + " on host: " + stats.getHostname() + ".");
         sb.append("Total scanned row: " + stats.getScannedRowCount() + ". ");
         sb.append("Total filtered/aggred row: " + 
stats.getAggregatedRowCount() + ". ");
         sb.append("Time elapsed in EP: " + (stats.getServiceEndTime() - 
stats.getServiceStartTime()) + "(ms). ");

http://git-wip-us.apache.org/repos/asf/kylin/blob/5d35f97f/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 ad0cb32..aa02036 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
@@ -234,6 +234,7 @@ public class CubeSegmentScanner implements IGTScanner {
 
             try {
                 if (scanRequests.size() == 0) {
+                    logger.info("Segment {} will be skipped", cubeSeg);
                     internal = new EmptyGTScanner();
                 } else {
                     internal = rpc.getGTScanner(scanRequests);

Reply via email to