minor refactors on StorageSideBehavior
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/a201c5b0 Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/a201c5b0 Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/a201c5b0 Branch: refs/heads/KYLIN-1726 Commit: a201c5b0f8e7706ef2cf7cbf9b6d43d3a6bc4a57 Parents: 6db4b17 Author: Hongbin Ma <mahong...@apache.org> Authored: Mon Sep 12 13:41:05 2016 +0800 Committer: Hongbin Ma <mahong...@apache.org> Committed: Mon Sep 12 23:53:48 2016 +0800 ---------------------------------------------------------------------- .../apache/kylin/gridtable/GTScanRequest.java | 10 +++++----- .../kylin/gridtable/StorageSideBehavior.java | 14 +++++++++++++- .../apache/kylin/query/ITKylinQueryTest.java | 20 ++++++++++++++++++-- .../observer/AggregationScanner.java | 6 +++--- .../hbase/cube/v2/HBaseReadonlyStore.java | 4 ++++ .../coprocessor/endpoint/CubeVisitService.java | 7 ++----- 6 files changed, 45 insertions(+), 16 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/a201c5b0/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java index 3e57e86..4f68806 100644 --- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java +++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanRequest.java @@ -158,14 +158,14 @@ public class GTScanRequest { } /** - * doFilter,doAggr,doMemCheck are only for profiling use. + * filterToggledOn,aggrToggledOn are only for profiling/test use. * in normal cases they are all true. - * <p/> + * * Refer to CoprocessorBehavior for explanation */ - public IGTScanner decorateScanner(IGTScanner scanner, boolean doFilter, boolean doAggr, long deadline) throws IOException { + public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, long deadline) throws IOException { IGTScanner result = scanner; - if (!doFilter) { //Skip reading this section if you're not profiling! + if (!filterToggledOn) { //Skip reading this section if you're not profiling! int scanned = lookAndForget(result); return new EmptyGTScanner(scanned); } else { @@ -174,7 +174,7 @@ public class GTScanRequest { result = new GTFilterScanner(result, this); } - if (!doAggr) {//Skip reading this section if you're not profiling! + if (!aggrToggledOn) {//Skip reading this section if you're not profiling! long scanned = result.getScannedRowCount(); lookAndForget(result); return new EmptyGTScanner(scanned); http://git-wip-us.apache.org/repos/asf/kylin/blob/a201c5b0/core-cube/src/main/java/org/apache/kylin/gridtable/StorageSideBehavior.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/StorageSideBehavior.java b/core-cube/src/main/java/org/apache/kylin/gridtable/StorageSideBehavior.java index 7fa93e7..b01ac3f 100644 --- a/core-cube/src/main/java/org/apache/kylin/gridtable/StorageSideBehavior.java +++ b/core-cube/src/main/java/org/apache/kylin/gridtable/StorageSideBehavior.java @@ -26,5 +26,17 @@ public enum StorageSideBehavior { SCAN_FILTER, //only scan+filter used,used for profiling filter speed. Will not return any result SCAN_FILTER_AGGR, //aggregate the result. Will return results SCAN_FILTER_AGGR_CHECKMEM, //default full operations. Will return results - SCAN_FILTER_AGGR_CHECKMEM_WITHDELAY, // on each scan operation, delay for 10s to simulate slow queries, for test use + SCAN_FILTER_AGGR_CHECKMEM_WITHDELAY; // on each scan operation, delay for 10s to simulate slow queries, for test use + + public boolean filterToggledOn() { + return this.ordinal() >= SCAN_FILTER.ordinal(); + } + + public boolean aggrToggledOn() { + return this.ordinal() >= SCAN_FILTER_AGGR.ordinal(); + } + + public boolean delayToggledOn() { + return this.ordinal() >= SCAN_FILTER_AGGR_CHECKMEM_WITHDELAY.ordinal(); + } } http://git-wip-us.apache.org/repos/asf/kylin/blob/a201c5b0/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java ---------------------------------------------------------------------- diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java index de68c7a..c1c9767 100644 --- a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java +++ b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java @@ -129,10 +129,21 @@ public class ITKylinQueryTest extends KylinTestBase { thrown.expect(SQLException.class); //should not break at table duplicate check, should fail at model duplicate check - thrown.expectCause(new BaseMatcher<Throwable>() { + thrown.expect(new BaseMatcher<Throwable>() { @Override public boolean matches(Object item) { - if (item instanceof GTScanSelfTerminatedException) { + + //find the "root" + Throwable throwable = (Throwable) item; + while (true) { + if (throwable.getCause() != null) { + throwable = throwable.getCause(); + } else { + break; + } + } + + if (throwable instanceof GTScanSelfTerminatedException) { return true; } return false; @@ -143,6 +154,11 @@ public class ITKylinQueryTest extends KylinTestBase { } }); + runTimetoutQueries(); + + } + + protected void runTimetoutQueries() throws Exception { try { Map<String, String> toggles = Maps.newHashMap(); http://git-wip-us.apache.org/repos/asf/kylin/blob/a201c5b0/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java ---------------------------------------------------------------------- diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java index a77f988..a900ea1 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java @@ -25,9 +25,9 @@ import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.regionserver.RegionScanner; +import org.apache.kylin.gridtable.StorageSideBehavior; import org.apache.kylin.measure.MeasureAggregator; import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey; -import org.apache.kylin.gridtable.StorageSideBehavior; import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorFilter; import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorProjector; import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorRowType; @@ -86,11 +86,11 @@ public class AggregationScanner implements RegionScanner { meaninglessByte += cell.getRowArray()[i]; } } else { - if (behavior.ordinal() >= StorageSideBehavior.SCAN_FILTER.ordinal()) { + if (behavior.filterToggledOn()) { if (filter != null && filter.evaluate(tuple) == false) continue; - if (behavior.ordinal() >= StorageSideBehavior.SCAN_FILTER_AGGR.ordinal()) { + if (behavior.aggrToggledOn()) { AggrKey aggKey = projector.getAggrKey(results); MeasureAggregator[] bufs = aggCache.getBuffer(aggKey); aggregators.aggregate(bufs, results); http://git-wip-us.apache.org/repos/asf/kylin/blob/a201c5b0/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/HBaseReadonlyStore.java ---------------------------------------------------------------------- diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/HBaseReadonlyStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/HBaseReadonlyStore.java index 1d8ad79..4c02dff 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/HBaseReadonlyStore.java +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/HBaseReadonlyStore.java @@ -45,6 +45,10 @@ public class HBaseReadonlyStore implements IGTStore { private int rowkeyPreambleSize; private boolean withDelay = false; + + /** + * @param withDelay is for test use + */ public HBaseReadonlyStore(CellListIterator cellListIterator, GTScanRequest gtScanRequest, List<Pair<byte[], byte[]>> hbaseColumns, List<List<Integer>> hbaseColumnsToGT, int rowkeyPreambleSize, boolean withDelay) { this.cellListIterator = cellListIterator; this.info = gtScanRequest.getInfo(); http://git-wip-us.apache.org/repos/asf/kylin/blob/a201c5b0/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java ---------------------------------------------------------------------- diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java index 36adca1..ffe41c5 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java @@ -284,13 +284,10 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement } }; - IGTStore store = new HBaseReadonlyStore(cellListIterator, scanReq, hbaseRawScans.get(0).hbaseColumns, hbaseColumnsToGT, // - request.getRowkeyPreambleSize(), StorageSideBehavior.SCAN_FILTER_AGGR_CHECKMEM_WITHDELAY.toString().equals(scanReq.getStorageBehavior())); + IGTStore store = new HBaseReadonlyStore(cellListIterator, scanReq, hbaseRawScans.get(0).hbaseColumns, hbaseColumnsToGT, request.getRowkeyPreambleSize(), behavior.delayToggledOn()); IGTScanner rawScanner = store.scan(scanReq); - IGTScanner finalScanner = scanReq.decorateScanner(rawScanner, // - behavior.ordinal() >= StorageSideBehavior.SCAN_FILTER.ordinal(), // - behavior.ordinal() >= StorageSideBehavior.SCAN_FILTER_AGGR.ordinal(), deadline); + IGTScanner finalScanner = scanReq.decorateScanner(rawScanner, behavior.filterToggledOn(), behavior.aggrToggledOn(), deadline); ByteBuffer buffer = ByteBuffer.allocate(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE);