You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2020/03/18 05:24:44 UTC

[GitHub] [druid] gianm opened a new pull request #9533: Broker: Add ability to inline subqueries.

gianm opened a new pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533
 
 
   Changes:
   
   - ClientQuerySegmentWalker: Add ability to inline queries. (This is the main change.)
   - Query: Add "getSubQueryId" and "withSubQueryId" methods.
   - QueryMetrics: Add "subQueryId" dimension.
   - ServerConfig: Add new "maxSubqueryRows" parameter, which is used b y ClientQuerySegmentWalker to limit how many rows can be inlined per query.
   - IndexedTableJoinMatcher: Allow creating keys on top of unknown types, by assuming they are strings. This is useful because not all types are known for fields in query results.
   - InlineDataSource: Store RowSignature rather than component parts. Add more zealous "equals" and "hashCode" methods to ease testing.
   - Moved QuerySegmentWalker test code from CalciteTests and SpecificSegmentsQueryWalker in druid-sql to QueryStackTests in druid-server. Use this to spin up a new ClientQuerySegmentWalkerTest.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394614845
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java
 ##########
 @@ -100,6 +103,10 @@ public ServerConfig()
   @Min(1)
   private long maxScatterGatherBytes = Long.MAX_VALUE;
 
+  @JsonProperty
+  @Min(1)
+  private int maxSubqueryRows = 100000;
 
 Review comment:
   this should probably be added to the documentation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394547684
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/Query.java
 ##########
 @@ -131,6 +131,11 @@
   @Nullable
   String getId();
 
+  Query<T> withSubQueryId(String subQueryId);
 
 Review comment:
   Maybe worth to add a javadoc explaining what is subQueryId and why we need it?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394666435
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/InlineDataSource.java
 ##########
 @@ -104,13 +113,21 @@ public static InlineDataSource fromIterable(
   @JsonProperty
   public List<String> getColumnNames()
   {
-    return columnNames;
+    return signature.getColumnNames();
   }
 
   @JsonProperty
+  @JsonInclude(JsonInclude.Include.NON_NULL)
 
 Review comment:
   Good point, I'll add this in the next patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on issue #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on issue #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#issuecomment-600770515
 
 
   This pull request **introduces 1 alert** when merging e6461d97086ae68042d75cfc6d6ba7c916e90a8f into 4c620b8f1c67e06e039b260de991c0e90f087010 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-d7010867b66456caf0b5d5f0fc26a8a00fe3a741)
   
   **new alerts:**
   
   * 1 for Contradictory type checks

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394666459
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/Query.java
 ##########
 @@ -131,6 +131,11 @@
   @Nullable
   String getId();
 
+  Query<T> withSubQueryId(String subQueryId);
 
 Review comment:
   Good idea, I'll add this in the next patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394536179
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/InlineDataSource.java
 ##########
 @@ -104,13 +113,21 @@ public static InlineDataSource fromIterable(
   @JsonProperty
   public List<String> getColumnNames()
   {
-    return columnNames;
+    return signature.getColumnNames();
   }
 
   @JsonProperty
+  @JsonInclude(JsonInclude.Include.NON_NULL)
 
 Review comment:
   Please add `@Nullable`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on issue #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
gianm commented on issue #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#issuecomment-600785808
 
 
   > 1 for Contradictory type checks
   
   It's complaining that a condition in an `assert` is always false. OK, fair enough, but isn't that the point of an `assert`? I say yes, so, I think we can ignore this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394666485
 
 

 ##########
 File path: server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java
 ##########
 @@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.FunctionalIterable;
+import org.apache.druid.java.util.common.guava.LazySequence;
+import org.apache.druid.query.FinalizeResultsQueryRunner;
+import org.apache.druid.query.NoopQueryRunner;
+import org.apache.druid.query.Queries;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.QueryRunner;
+import org.apache.druid.query.QueryRunnerFactory;
+import org.apache.druid.query.QueryRunnerFactoryConglomerate;
+import org.apache.druid.query.QuerySegmentWalker;
+import org.apache.druid.query.QueryToolChest;
+import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.planning.DataSourceAnalysis;
+import org.apache.druid.query.spec.SpecificSegmentQueryRunner;
+import org.apache.druid.query.spec.SpecificSegmentSpec;
+import org.apache.druid.segment.ReferenceCountingSegment;
+import org.apache.druid.segment.Segment;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.segment.join.Joinables;
+import org.apache.druid.timeline.TimelineObjectHolder;
+import org.apache.druid.timeline.VersionedIntervalTimeline;
+import org.apache.druid.timeline.partition.PartitionChunk;
+import org.apache.druid.timeline.partition.PartitionHolder;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+
+/**
+ * Mimics the behavior of {@link org.apache.druid.client.CachingClusteredClient} when it queries data servers (like
+ * Historicals, which use {@link org.apache.druid.server.coordination.ServerManager}). Used by {@link QueryStackTests}.
+ *
+ * This class's logic is like a mashup of those two classes. With the right abstractions, it may be possible to get rid
+ * of this class and replace it with the production classes.
 
 Review comment:
   I'll add this in the next patch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394615327
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java
 ##########
 @@ -100,6 +103,10 @@ public ServerConfig()
   @Min(1)
   private long maxScatterGatherBytes = Long.MAX_VALUE;
 
+  @JsonProperty
+  @Min(1)
+  private int maxSubqueryRows = 100000;
 
 Review comment:
   Oh yeah, good point.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394201216
 
 

 ##########
 File path: benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java
 ##########
 @@ -183,20 +183,19 @@ public void setup()
     log.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment);
     final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
 
-    final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerate = CalciteTests.createQueryRunnerFactoryConglomerate();
-    closer.register(conglomerate.rhs);
+    final QueryRunnerFactoryConglomerate conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(closer);
 
 Review comment:
   :+1:

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on issue #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on issue #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#issuecomment-600717090
 
 
   This pull request **introduces 1 alert** when merging 6f73c73aa1012c1f237d1e2e776e577e4e6c45d3 into 4c620b8f1c67e06e039b260de991c0e90f087010 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-02c07724a028f0fce26a8fbbad6c7d093380bb5c)
   
   **new alerts:**
   
   * 1 for Contradictory type checks

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] lgtm-com[bot] commented on issue #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on issue #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#issuecomment-600448166
 
 
   This pull request **introduces 1 alert** when merging 91ea2e9a760e3724a0584ae117d936c20a52ec37 into 4c620b8f1c67e06e039b260de991c0e90f087010 - [view on LGTM.com](https://lgtm.com/projects/g/apache/druid/rev/pr-7846186a424468f0cd22bf8454eb5d61c2b1f575)
   
   **new alerts:**
   
   * 1 for Contradictory type checks

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] gianm commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
gianm commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394628939
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java
 ##########
 @@ -100,6 +103,10 @@ public ServerConfig()
   @Min(1)
   private long maxScatterGatherBytes = Long.MAX_VALUE;
 
+  @JsonProperty
+  @Min(1)
+  private int maxSubqueryRows = 100000;
 
 Review comment:
   I'm hoping to update all the docs at once in a future patch

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jon-wei merged pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
jon-wei merged pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9533: Broker: Add ability to inline subqueries.

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9533: Broker: Add ability to inline subqueries.
URL: https://github.com/apache/druid/pull/9533#discussion_r394613436
 
 

 ##########
 File path: server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java
 ##########
 @@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.FunctionalIterable;
+import org.apache.druid.java.util.common.guava.LazySequence;
+import org.apache.druid.query.FinalizeResultsQueryRunner;
+import org.apache.druid.query.NoopQueryRunner;
+import org.apache.druid.query.Queries;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.QueryRunner;
+import org.apache.druid.query.QueryRunnerFactory;
+import org.apache.druid.query.QueryRunnerFactoryConglomerate;
+import org.apache.druid.query.QuerySegmentWalker;
+import org.apache.druid.query.QueryToolChest;
+import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.planning.DataSourceAnalysis;
+import org.apache.druid.query.spec.SpecificSegmentQueryRunner;
+import org.apache.druid.query.spec.SpecificSegmentSpec;
+import org.apache.druid.segment.ReferenceCountingSegment;
+import org.apache.druid.segment.Segment;
+import org.apache.druid.segment.join.JoinableFactory;
+import org.apache.druid.segment.join.Joinables;
+import org.apache.druid.timeline.TimelineObjectHolder;
+import org.apache.druid.timeline.VersionedIntervalTimeline;
+import org.apache.druid.timeline.partition.PartitionChunk;
+import org.apache.druid.timeline.partition.PartitionHolder;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+
+/**
+ * Mimics the behavior of {@link org.apache.druid.client.CachingClusteredClient} when it queries data servers (like
+ * Historicals, which use {@link org.apache.druid.server.coordination.ServerManager}). Used by {@link QueryStackTests}.
+ *
+ * This class's logic is like a mashup of those two classes. With the right abstractions, it may be possible to get rid
+ * of this class and replace it with the production classes.
 
 Review comment:
   Should this class be mentioned in `ServerManager` and `CachingClusteredClient` so that these classes are synced up?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org