KYLIN-2436 add a configuration knob to disable spilling of aggregation cache
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/ecf6a69f Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/ecf6a69f Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/ecf6a69f Branch: refs/heads/master-hbase0.98 Commit: ecf6a69fece7cbda3a9bd8d678c928224ce677aa Parents: 41332f3 Author: gaodayue <gaoda...@meituan.com> Authored: Mon Feb 6 14:59:06 2017 +0800 Committer: gaodayue <gaoda...@meituan.com> Committed: Fri Feb 10 12:43:34 2017 +0800 ---------------------------------------------------------------------- .../apache/kylin/common/KylinConfigBase.java | 4 + .../cube/inmemcubing/InMemCubeBuilder.java | 2 +- .../kylin/gridtable/GTAggregateScanner.java | 54 ++++---- .../apache/kylin/gridtable/GTScanRequest.java | 6 +- .../GTScanSelfTerminatedException.java | 4 + .../gridtable/AggregationCacheSpillTest.java | 4 +- .../hbase/cube/v2/CubeHBaseEndpointRPC.java | 1 + .../coprocessor/endpoint/CubeVisitService.java | 12 +- .../endpoint/generated/CubeVisitProtos.java | 123 ++++++++++++++++--- .../endpoint/protobuf/CubeVisit.proto | 1 + 10 files changed, 162 insertions(+), 49 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java ---------------------------------------------------------------------- diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java index dce4149..81f0187 100644 --- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java +++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java @@ -609,6 +609,10 @@ abstract public class KylinConfigBase implements Serializable { return Double.parseDouble(this.getOptional("kylin.storage.hbase.coprocessor-mem-gb", "3.0")); } + public boolean getQueryCoprocessorSpillEnabled() { + return Boolean.parseBoolean(this.getOptional("kylin.storage.hbase.coprocessor-spill-enabled", "true")); + } + public int getQueryCoprocessorTimeoutSeconds() { return Integer.parseInt(this.getOptional("kylin.storage.hbase.coprocessor-timeout-seconds", "0")); } http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java index 398c594..a74f0c0 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java @@ -335,7 +335,7 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder { Pair<ImmutableBitSet, ImmutableBitSet> dimensionMetricsBitSet = InMemCubeBuilderUtils.getDimensionAndMetricColumnBitSet(baseCuboidId, measureCount); GTScanRequest req = new GTScanRequestBuilder().setInfo(baseCuboid.getInfo()).setRanges(null).setDimensions(null).setAggrGroupBy(dimensionMetricsBitSet.getFirst()).setAggrMetrics(dimensionMetricsBitSet.getSecond()).setAggrMetricsFuncs(metricsAggrFuncs).setFilterPushDown(null).createGTScanRequest(); - GTAggregateScanner aggregationScanner = new GTAggregateScanner(baseInput, req, Long.MAX_VALUE); + GTAggregateScanner aggregationScanner = new GTAggregateScanner(baseInput, req); aggregationScanner.trackMemoryLevel(baseCuboidMemTracker); int count = 0; http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java index 9158aa3..147dbc1 100644 --- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java +++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java @@ -63,15 +63,20 @@ public class GTAggregateScanner implements IGTScanner { final String[] metricsAggrFuncs; final IGTScanner inputScanner; final AggregationCache aggrCache; - final long spillThreshold; + final long spillThreshold; // 0 means no memory control && no spill final int storagePushDownLimit;//default to be Int.MAX final long deadline; + final boolean spillEnabled; private int aggregatedRowCount = 0; private MemoryWaterLevel memTracker; private boolean[] aggrMask; - public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req, long deadline) { + public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req) { + this(inputScanner, req, Long.MAX_VALUE, true); + } + + public GTAggregateScanner(IGTScanner inputScanner, GTScanRequest req, long deadline, boolean spillEnabled) { if (!req.hasAggregation()) throw new IllegalStateException(); @@ -86,6 +91,7 @@ public class GTAggregateScanner implements IGTScanner { this.aggrMask = new boolean[metricsAggrFuncs.length]; this.storagePushDownLimit = req.getStoragePushDownLimit(); this.deadline = deadline; + this.spillEnabled = spillEnabled; Arrays.fill(aggrMask, true); } @@ -276,10 +282,16 @@ public class GTAggregateScanner implements IGTScanner { if (memTracker != null) { memTracker.markHigh(); } - if (spillThreshold > 0) { + + final long estMemSize = estimatedMemSize(); + if (spillThreshold > 0 && estMemSize > spillThreshold) { // spill to disk when aggBufMap used too large memory - if (estimatedMemSize() > spillThreshold) { - spillBuffMap(); + if (spillEnabled) { + spillBuffMap(estMemSize); + aggBufMap = createBuffMap(); + + } else { + throw new GTScanSelfTerminatedException("Aggregation using more than " + spillThreshold + " memory and spill is disabled"); } } } @@ -306,17 +318,13 @@ public class GTAggregateScanner implements IGTScanner { return true; } - private void spillBuffMap() throws RuntimeException { - if (aggBufMap.isEmpty()) - return; - + private void spillBuffMap(long estMemSize) throws RuntimeException { try { - Dump dump = new Dump(aggBufMap); + Dump dump = new Dump(aggBufMap, estMemSize); dump.flush(); dumps.add(dump); - aggBufMap = createBuffMap(); } catch (Exception e) { - throw new RuntimeException("AggregationCache spill failed: " + e.getMessage()); + throw new RuntimeException("AggregationCache failed to spill", e); } } @@ -372,9 +380,9 @@ public class GTAggregateScanner implements IGTScanner { }; } else { // the spill case - - logger.info("Last spill, current AggregationCache memory estimated size is: " + getEstimateSizeOfAggrCache()); - this.spillBuffMap(); + if (!aggBufMap.isEmpty()) { + this.spillBuffMap(getEstimateSizeOfAggrCache()); // TODO allow merge in-mem map with spilled dumps + } return new Iterator<GTRecord>() { final DumpMerger merger = new DumpMerger(dumps); @@ -430,12 +438,16 @@ public class GTAggregateScanner implements IGTScanner { } class Dump implements Iterable<Pair<byte[], byte[]>> { - File dumpedFile; - DataInputStream dis; + final File dumpedFile; SortedMap<byte[], MeasureAggregator[]> buffMap; + final long estMemSize; - public Dump(SortedMap<byte[], MeasureAggregator[]> buffMap) throws IOException { + DataInputStream dis; + + public Dump(SortedMap<byte[], MeasureAggregator[]> buffMap, long estMemSize) throws IOException { + this.dumpedFile = File.createTempFile("KYLIN_SPILL_", ".tmp"); this.buffMap = buffMap; + this.estMemSize = estMemSize; } @Override @@ -482,13 +494,13 @@ public class GTAggregateScanner implements IGTScanner { } public void flush() throws IOException { + logger.info("AggregationCache(size={} est_mem_size={} threshold={}) will spill to {}", + buffMap.size(), estMemSize, spillThreshold, dumpedFile.getAbsolutePath()); + if (buffMap != null) { DataOutputStream dos = null; Object[] aggrResult = null; try { - dumpedFile = File.createTempFile("KYLIN_AGGR_", ".tmp"); - - logger.info("AggregationCache will dump to file: " + dumpedFile.getAbsolutePath()); dos = new DataOutputStream(new FileOutputStream(dumpedFile)); dos.writeInt(buffMap.size()); for (Entry<byte[], MeasureAggregator[]> entry : buffMap.entrySet()) { http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/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 7976e3f..7c94f5a 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 @@ -166,13 +166,13 @@ public class GTScanRequest { * Refer to CoprocessorBehavior for explanation */ public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, long deadline) throws IOException { - return decorateScanner(scanner, filterToggledOn, aggrToggledOn, false, deadline); + return decorateScanner(scanner, filterToggledOn, aggrToggledOn, false, deadline, true); } /** * hasPreFiltered indicate the data has been filtered before scanning */ - public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, boolean hasPreFiltered, long deadline) throws IOException { + public IGTScanner decorateScanner(IGTScanner scanner, boolean filterToggledOn, boolean aggrToggledOn, boolean hasPreFiltered, long deadline, boolean spillEnabled) throws IOException { IGTScanner result = scanner; if (!filterToggledOn) { //Skip reading this section if you're not profiling! int scanned = lookAndForget(result); @@ -194,7 +194,7 @@ public class GTScanRequest { } else if (this.hasAggregation()) { logger.info("pre aggregating results before returning"); this.doingStorageAggregation = true; - result = new GTAggregateScanner(result, this, deadline); + result = new GTAggregateScanner(result, this, deadline, spillEnabled); } else { logger.info("has no aggregation, skip it"); } http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java index 4775ac6..30d3aaa 100644 --- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java +++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTScanSelfTerminatedException.java @@ -18,6 +18,10 @@ package org.apache.kylin.gridtable; +/** + * Implementations of {@link IGTScanner} should throw {@link GTScanSelfTerminatedException} or its subclasses + * in cases where the scan runs out of resources (time, memory, etc) and can not be continued. + */ public class GTScanSelfTerminatedException extends RuntimeException { public GTScanSelfTerminatedException(String s) { http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java ---------------------------------------------------------------------- diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java index 74c2659..7abb069 100644 --- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java +++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java @@ -84,7 +84,7 @@ public class AggregationCacheSpillTest extends LocalFileMetadataTestCase { GTScanRequest scanRequest = new GTScanRequestBuilder().setInfo(INFO).setRanges(null).setDimensions(new ImmutableBitSet(0, 3)).setAggrGroupBy(new ImmutableBitSet(0, 3)).setAggrMetrics(new ImmutableBitSet(3, 6)).setAggrMetricsFuncs(new String[] { "SUM", "SUM", "COUNT_DISTINCT" }).setFilterPushDown(null).setAggCacheMemThreshold(0.5).createGTScanRequest(); - GTAggregateScanner scanner = new GTAggregateScanner(inputScanner, scanRequest, Long.MAX_VALUE); + GTAggregateScanner scanner = new GTAggregateScanner(inputScanner, scanRequest); int count = 0; for (GTRecord record : scanner) { @@ -126,7 +126,7 @@ public class AggregationCacheSpillTest extends LocalFileMetadataTestCase { // all-in-mem testcase GTScanRequest scanRequest = new GTScanRequestBuilder().setInfo(INFO).setRanges(null).setDimensions(new ImmutableBitSet(0, 3)).setAggrGroupBy(new ImmutableBitSet(1, 3)).setAggrMetrics(new ImmutableBitSet(3, 6)).setAggrMetricsFuncs(new String[] { "SUM", "SUM", "COUNT_DISTINCT" }).setFilterPushDown(null).setAggCacheMemThreshold(0.5).createGTScanRequest(); - GTAggregateScanner scanner = new GTAggregateScanner(inputScanner, scanRequest, Long.MAX_VALUE); + GTAggregateScanner scanner = new GTAggregateScanner(inputScanner, scanRequest); int count = 0; for (GTRecord record : scanner) { http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/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 cad5a3f..dd9f74c 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 @@ -163,6 +163,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC { if (queryId != null) { builder.setQueryId(queryId); } + builder.setSpillEnabled(cubeSeg.getConfig().getQueryCoprocessorSpillEnabled()); for (final Pair<byte[], byte[]> epRange : getEPKeyRanges(cuboidBaseShard, shardNum, totalShards)) { executorService.submit(new Runnable() { http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/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 810747f..3e0a065 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 @@ -51,6 +51,7 @@ import org.apache.kylin.cube.kv.RowConstants; import org.apache.kylin.gridtable.GTRecord; import org.apache.kylin.gridtable.GTScanExceedThresholdException; import org.apache.kylin.gridtable.GTScanRequest; +import org.apache.kylin.gridtable.GTScanSelfTerminatedException; import org.apache.kylin.gridtable.GTScanTimeoutException; import org.apache.kylin.gridtable.IGTScanner; import org.apache.kylin.gridtable.IGTStore; @@ -280,7 +281,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement 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.filterToggledOn(), behavior.aggrToggledOn(), deadline); + IGTScanner finalScanner = scanReq.decorateScanner(rawScanner, behavior.filterToggledOn(), behavior.aggrToggledOn(), false, deadline, request.getSpillEnabled()); ByteBuffer buffer = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE); @@ -315,12 +316,11 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement break; } } - } catch (GTScanTimeoutException e) { + } catch (GTScanSelfTerminatedException e) { + // the query is using too much resource, we mark it as abnormal finish instead of + // throwing RuntimeException to avoid client retrying RPC. scanNormalComplete.setValue(false); - logger.info("The cube visit did not finish normally because scan timeout", e); - } catch (GTScanExceedThresholdException e) { - scanNormalComplete.setValue(false); - logger.info("The cube visit did not finish normally because scan num exceeds threshold", e); + logger.warn("Abort scan: {}", e.getMessage()); } finally { finalScanner.close(); } http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java ---------------------------------------------------------------------- diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java index 628d453..def0182 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/generated/CubeVisitProtos.java @@ -107,6 +107,16 @@ public final class CubeVisitProtos { */ com.google.protobuf.ByteString getQueryIdBytes(); + + // optional bool spillEnabled = 7 [default = true]; + /** + * <code>optional bool spillEnabled = 7 [default = true];</code> + */ + boolean hasSpillEnabled(); + /** + * <code>optional bool spillEnabled = 7 [default = true];</code> + */ + boolean getSpillEnabled(); } /** * Protobuf type {@code CubeVisitRequest} @@ -192,6 +202,11 @@ public final class CubeVisitProtos { queryId_ = input.readBytes(); break; } + case 56: { + bitField0_ |= 0x00000020; + spillEnabled_ = input.readBool(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -921,6 +936,22 @@ public final class CubeVisitProtos { } } + // optional bool spillEnabled = 7 [default = true]; + public static final int SPILLENABLED_FIELD_NUMBER = 7; + private boolean spillEnabled_; + /** + * <code>optional bool spillEnabled = 7 [default = true];</code> + */ + public boolean hasSpillEnabled() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * <code>optional bool spillEnabled = 7 [default = true];</code> + */ + public boolean getSpillEnabled() { + return spillEnabled_; + } + private void initFields() { gtScanRequest_ = com.google.protobuf.ByteString.EMPTY; hbaseRawScan_ = com.google.protobuf.ByteString.EMPTY; @@ -928,6 +959,7 @@ public final class CubeVisitProtos { hbaseColumnsToGT_ = java.util.Collections.emptyList(); kylinProperties_ = ""; queryId_ = ""; + spillEnabled_ = true; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -975,6 +1007,9 @@ public final class CubeVisitProtos { if (((bitField0_ & 0x00000010) == 0x00000010)) { output.writeBytes(6, getQueryIdBytes()); } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeBool(7, spillEnabled_); + } getUnknownFields().writeTo(output); } @@ -1008,6 +1043,10 @@ public final class CubeVisitProtos { size += com.google.protobuf.CodedOutputStream .computeBytesSize(6, getQueryIdBytes()); } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(7, spillEnabled_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -1058,6 +1097,11 @@ public final class CubeVisitProtos { result = result && getQueryId() .equals(other.getQueryId()); } + result = result && (hasSpillEnabled() == other.hasSpillEnabled()); + if (hasSpillEnabled()) { + result = result && (getSpillEnabled() + == other.getSpillEnabled()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -1095,6 +1139,10 @@ public final class CubeVisitProtos { hash = (37 * hash) + QUERYID_FIELD_NUMBER; hash = (53 * hash) + getQueryId().hashCode(); } + if (hasSpillEnabled()) { + hash = (37 * hash) + SPILLENABLED_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getSpillEnabled()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -1221,6 +1269,8 @@ public final class CubeVisitProtos { bitField0_ = (bitField0_ & ~0x00000010); queryId_ = ""; bitField0_ = (bitField0_ & ~0x00000020); + spillEnabled_ = true; + bitField0_ = (bitField0_ & ~0x00000040); return this; } @@ -1278,6 +1328,10 @@ public final class CubeVisitProtos { to_bitField0_ |= 0x00000010; } result.queryId_ = queryId_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000020; + } + result.spillEnabled_ = spillEnabled_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -1339,6 +1393,9 @@ public final class CubeVisitProtos { queryId_ = other.queryId_; onChanged(); } + if (other.hasSpillEnabled()) { + setSpillEnabled(other.getSpillEnabled()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -1899,6 +1956,39 @@ public final class CubeVisitProtos { return this; } + // optional bool spillEnabled = 7 [default = true]; + private boolean spillEnabled_ = true; + /** + * <code>optional bool spillEnabled = 7 [default = true];</code> + */ + public boolean hasSpillEnabled() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + /** + * <code>optional bool spillEnabled = 7 [default = true];</code> + */ + public boolean getSpillEnabled() { + return spillEnabled_; + } + /** + * <code>optional bool spillEnabled = 7 [default = true];</code> + */ + public Builder setSpillEnabled(boolean value) { + bitField0_ |= 0x00000040; + spillEnabled_ = value; + onChanged(); + return this; + } + /** + * <code>optional bool spillEnabled = 7 [default = true];</code> + */ + public Builder clearSpillEnabled() { + bitField0_ = (bitField0_ & ~0x00000040); + spillEnabled_ = true; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:CubeVisitRequest) } @@ -4253,25 +4343,26 @@ public final class CubeVisitProtos { java.lang.String[] descriptorData = { "\npstorage-hbase/src/main/java/org/apache" + "/kylin/storage/hbase/cube/v2/coprocessor" + - "/endpoint/protobuf/CubeVisit.proto\"\323\001\n\020C" + + "/endpoint/protobuf/CubeVisit.proto\"\357\001\n\020C" + "ubeVisitRequest\022\025\n\rgtScanRequest\030\001 \002(\014\022\024" + "\n\014hbaseRawScan\030\002 \002(\014\022\032\n\022rowkeyPreambleSi" + "ze\030\003 \002(\005\0223\n\020hbaseColumnsToGT\030\004 \003(\0132\031.Cub" + "eVisitRequest.IntList\022\027\n\017kylinProperties" + - "\030\005 \002(\t\022\017\n\007queryId\030\006 \001(\t\032\027\n\007IntList\022\014\n\004in" + - "ts\030\001 \003(\005\"\321\002\n\021CubeVisitResponse\022\026\n\016compre" + - "ssedRows\030\001 \002(\014\022\'\n\005stats\030\002 \002(\0132\030.CubeVisi", - "tResponse.Stats\032\372\001\n\005Stats\022\030\n\020serviceStar" + - "tTime\030\001 \001(\003\022\026\n\016serviceEndTime\030\002 \001(\003\022\027\n\017s" + - "cannedRowCount\030\003 \001(\003\022\032\n\022aggregatedRowCou" + - "nt\030\004 \001(\003\022\025\n\rsystemCpuLoad\030\005 \001(\001\022\036\n\026freeP" + - "hysicalMemorySize\030\006 \001(\001\022\031\n\021freeSwapSpace" + - "Size\030\007 \001(\001\022\020\n\010hostname\030\010 \001(\t\022\016\n\006etcMsg\030\t" + - " \001(\t\022\026\n\016normalComplete\030\n \001(\0052F\n\020CubeVisi" + - "tService\0222\n\tvisitCube\022\021.CubeVisitRequest" + - "\032\022.CubeVisitResponseB`\nEorg.apache.kylin" + - ".storage.hbase.cube.v2.coprocessor.endpo", - "int.generatedB\017CubeVisitProtosH\001\210\001\001\240\001\001" + "\030\005 \002(\t\022\017\n\007queryId\030\006 \001(\t\022\032\n\014spillEnabled\030" + + "\007 \001(\010:\004true\032\027\n\007IntList\022\014\n\004ints\030\001 \003(\005\"\321\002\n" + + "\021CubeVisitResponse\022\026\n\016compressedRows\030\001 \002", + "(\014\022\'\n\005stats\030\002 \002(\0132\030.CubeVisitResponse.St" + + "ats\032\372\001\n\005Stats\022\030\n\020serviceStartTime\030\001 \001(\003\022" + + "\026\n\016serviceEndTime\030\002 \001(\003\022\027\n\017scannedRowCou" + + "nt\030\003 \001(\003\022\032\n\022aggregatedRowCount\030\004 \001(\003\022\025\n\r" + + "systemCpuLoad\030\005 \001(\001\022\036\n\026freePhysicalMemor" + + "ySize\030\006 \001(\001\022\031\n\021freeSwapSpaceSize\030\007 \001(\001\022\020" + + "\n\010hostname\030\010 \001(\t\022\016\n\006etcMsg\030\t \001(\t\022\026\n\016norm" + + "alComplete\030\n \001(\0052F\n\020CubeVisitService\0222\n\t" + + "visitCube\022\021.CubeVisitRequest\032\022.CubeVisit" + + "ResponseB`\nEorg.apache.kylin.storage.hba", + "se.cube.v2.coprocessor.endpoint.generate" + + "dB\017CubeVisitProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -4283,7 +4374,7 @@ public final class CubeVisitProtos { internal_static_CubeVisitRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CubeVisitRequest_descriptor, - new java.lang.String[] { "GtScanRequest", "HbaseRawScan", "RowkeyPreambleSize", "HbaseColumnsToGT", "KylinProperties", "QueryId", }); + new java.lang.String[] { "GtScanRequest", "HbaseRawScan", "RowkeyPreambleSize", "HbaseColumnsToGT", "KylinProperties", "QueryId", "SpillEnabled", }); internal_static_CubeVisitRequest_IntList_descriptor = internal_static_CubeVisitRequest_descriptor.getNestedTypes().get(0); internal_static_CubeVisitRequest_IntList_fieldAccessorTable = new http://git-wip-us.apache.org/repos/asf/kylin/blob/ecf6a69f/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto ---------------------------------------------------------------------- diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto index 1a56719..c7c2954 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/protobuf/CubeVisit.proto @@ -36,6 +36,7 @@ message CubeVisitRequest { repeated IntList hbaseColumnsToGT = 4; required string kylinProperties = 5; // kylin properties optional string queryId = 6; + optional bool spillEnabled = 7 [default = true]; message IntList { repeated int32 ints = 1; }