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 2022/04/19 03:22:18 UTC

[GitHub] [druid] suneet-s opened a new pull request, #12456: Introduce a monitor to report detailed segment metrics

suneet-s opened a new pull request, #12456:
URL: https://github.com/apache/druid/pull/12456

   ### Description
   
   The SchemaStatsMonitor reports detailed metrics for each segment in the cluster.
   
   A user will need to enable this since I am not yet certain what the impact will
   be if this operation happens on large clusters.
   
   The metric naming scheme of segment/detailed/* is because there are already
   metrics named segment/* which have a similar use, but are reported differently.
   
   I had considered reporting the number of rows per segment via the
   EmitClusterStatsAndMetricsModule since we have access to the DataSegment object
   in the coordinator when running through the details to decide which segments
   need compaction. This proved challenging since the number of rows wasn't
   readily available.
   
   <hr>
   
   This PR is still missing unit tests, but I was hoping to get an initial round of feedback before I go too far down this path.
   
   This PR has:
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] suneet-s commented on a diff in pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
suneet-s commented on code in PR #12456:
URL: https://github.com/apache/druid/pull/12456#discussion_r852561978


##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/SchemaStatsMonitor.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.sql.calcite.schema;
+
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.java.util.metrics.AbstractMonitor;
+import org.apache.druid.query.DruidMetrics;
+import org.apache.druid.timeline.SegmentId;
+
+import javax.inject.Inject;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * A monitor that provides stats on segments that are visible to the {@link DruidSchema}
+ */
+public class SchemaStatsMonitor extends AbstractMonitor
+{
+  private final DruidSchema druidSchema;
+
+  @Inject
+  SchemaStatsMonitor(DruidSchema druidSchema)
+  {
+    this.druidSchema = druidSchema;
+  }
+
+  @Override
+  public boolean doMonitor(ServiceEmitter emitter)
+  {
+    final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
+    for (Map.Entry<String, ConcurrentSkipListMap<SegmentId, AvailableSegmentMetadata>> datasourceAndSegmentsMetadata : druidSchema.getSegmentMetadataInfoUnsafe().entrySet()) {
+      builder.setDimension(DruidMetrics.DATASOURCE, datasourceAndSegmentsMetadata.getKey());
+      for (AvailableSegmentMetadata metadata : datasourceAndSegmentsMetadata.getValue().values()) {
+        builder.setDimension("segment/binaryVersion", metadata.getSegment().getBinaryVersion());
+        builder.setDimension("segment/hasLastCompactedState", metadata.getSegment().getLastCompactionState() != null);
+
+        emitter.emit(builder.build("segment/detailed/numRows", metadata.getNumRows()));
+        emitter.emit(builder.build("segment/detailed/numReplicas", metadata.getNumReplicas()));

Review Comment:
   Only these 2 metrics are not reportable from `EmitClusterStatsAndMetrics`, but I chose to report the other metrics in this module since they all seemed related.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] suneet-s commented on a diff in pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
suneet-s commented on code in PR #12456:
URL: https://github.com/apache/druid/pull/12456#discussion_r853105735


##########
docs/operations/metrics.md:
##########
@@ -244,7 +251,7 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
 |`segment/loadQueue/count`|Number of segments to load.|server.|Varies.|
 |`segment/dropQueue/count`|Number of segments to drop.|server.|Varies.|
 |`segment/size`|Total size of used segments in a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|Varies.|
-|`segment/count`|Number of used segments belonging to a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|< max|
+|`segsegment/count`|Number of used segments belonging to a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|< max|

Review Comment:
   yes a typo.
   
   `segment/used/count` sounds like a better name. Unfortunately, it's an existing metric that people might rely on, so if we change the name, I think we should consider how to minimize disruption for users on an upgrade. I don't know of a good pattern for this, so I'm going to skip making that change



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] suneet-s closed pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
suneet-s closed pull request #12456: Introduce a monitor to report detailed segment metrics
URL: https://github.com/apache/druid/pull/12456


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] suneet-s commented on a diff in pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
suneet-s commented on code in PR #12456:
URL: https://github.com/apache/druid/pull/12456#discussion_r852561524


##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java:
##########
@@ -107,6 +107,7 @@ public class DruidSchema extends AbstractSchema
   private final JoinableFactory joinableFactory;
   private final ExecutorService cacheExec;
   private final ExecutorService callbackExec;
+  private final ExecutorService metricsReporterExec;

Review Comment:
   Oops - I will remove all these changes to this class. This was leftover from an earlier prototype.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] suneet-s commented on a diff in pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
suneet-s commented on code in PR #12456:
URL: https://github.com/apache/druid/pull/12456#discussion_r853102413


##########
docs/operations/metrics.md:
##########
@@ -58,6 +58,13 @@ Metrics may have additional dimensions beyond those listed above.
 |`query/priority`|Assigned lane and priority, only if Laning strategy is enabled. Refer to [Laning strategies](../configuration/index.md#laning-strategies)|lane, dataSource, type|0|
 |`sqlQuery/time`|Milliseconds taken to complete a SQL query.|id, nativeQueryIds, dataSource, remoteAddress, success.|< 1s|
 |`sqlQuery/bytes`|Number of bytes returned in the SQL query response.|id, nativeQueryIds, dataSource, remoteAddress, success.| |
+|`segment/detailed/numRows`|Number of rows in a segment. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies. Good sized segments should be around 5M rows.|
+|`segment/detailed/numReplicas`|Number servers that this segment is available on. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies. Should be the number of replicas configured.|
+|`segment/detailed/sizeBytes`|Size of the segment in bytes. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies.|
+|`segment/detailed/numDimensions`|Number of dimensions in the segment metadata. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies.|
+|`segment/detailed/numMetrics`|Number of metrics in the segment metadata. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies.|
+|`segment/detailed/durationMillis`|The size of the interval that the segment covers in millis. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies.|

Review Comment:
   Good point. How about `segment/detailed/interval/durationMillis`?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] suneet-s commented on a diff in pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
suneet-s commented on code in PR #12456:
URL: https://github.com/apache/druid/pull/12456#discussion_r853117087


##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java:
##########
@@ -825,6 +837,14 @@ Map<SegmentId, AvailableSegmentMetadata> getSegmentMetadataSnapshot()
     return segmentMetadata;
   }
 
+  /**
+   * @return segmentMetadataInfo. Callers can only safely read the contents of the map.
+   */
+  public ConcurrentHashMap<String, ConcurrentSkipListMap<SegmentId, AvailableSegmentMetadata>> getSegmentMetadataInfoUnsafe()

Review Comment:
   > should we return a copy of the map rather than the actual?
   Yeah, I think that's the right thing to do. I will have to think about whether or not each thread should get it's own copy of the map or if it's better to just keep a single copy for reading and share that among threads.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] kfaraz commented on a diff in pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #12456:
URL: https://github.com/apache/druid/pull/12456#discussion_r852583990


##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/SchemaStatsMonitor.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.sql.calcite.schema;
+
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.java.util.metrics.AbstractMonitor;
+import org.apache.druid.query.DruidMetrics;
+import org.apache.druid.timeline.SegmentId;
+
+import javax.inject.Inject;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * A monitor that provides stats on segments that are visible to the {@link DruidSchema}
+ */
+public class SchemaStatsMonitor extends AbstractMonitor

Review Comment:
   Nit: I guess a better name would be `SegmentStatsMonitor` as that is what we are really reporting, even though we are tapping into the `DruidSchema` for that info.



##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java:
##########
@@ -825,6 +837,14 @@ Map<SegmentId, AvailableSegmentMetadata> getSegmentMetadataSnapshot()
     return segmentMetadata;
   }
 
+  /**
+   * @return segmentMetadataInfo. Callers can only safely read the contents of the map.
+   */
+  public ConcurrentHashMap<String, ConcurrentSkipListMap<SegmentId, AvailableSegmentMetadata>> getSegmentMetadataInfoUnsafe()

Review Comment:
   As we are placing the monitor in the same package and as this is marked to be an unsafe method, it should atleast be package protected.
   Seeing as this is a sensitive method,
   - should we annotate it as so?
   - should we return a copy of the map rather than the actual?



##########
docs/operations/metrics.md:
##########
@@ -244,7 +251,7 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
 |`segment/loadQueue/count`|Number of segments to load.|server.|Varies.|
 |`segment/dropQueue/count`|Number of segments to drop.|server.|Varies.|
 |`segment/size`|Total size of used segments in a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|Varies.|
-|`segment/count`|Number of used segments belonging to a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|< max|
+|`segsegment/count`|Number of used segments belonging to a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|< max|

Review Comment:
   Is this a typo?
   Reading the description, it seems that this should rather be reported as `segment/used/count`. What do you think?



##########
docs/operations/metrics.md:
##########
@@ -58,6 +58,13 @@ Metrics may have additional dimensions beyond those listed above.
 |`query/priority`|Assigned lane and priority, only if Laning strategy is enabled. Refer to [Laning strategies](../configuration/index.md#laning-strategies)|lane, dataSource, type|0|
 |`sqlQuery/time`|Milliseconds taken to complete a SQL query.|id, nativeQueryIds, dataSource, remoteAddress, success.|< 1s|
 |`sqlQuery/bytes`|Number of bytes returned in the SQL query response.|id, nativeQueryIds, dataSource, remoteAddress, success.| |
+|`segment/detailed/numRows`|Number of rows in a segment. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies. Good sized segments should be around 5M rows.|
+|`segment/detailed/numReplicas`|Number servers that this segment is available on. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies. Should be the number of replicas configured.|
+|`segment/detailed/sizeBytes`|Size of the segment in bytes. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies.|
+|`segment/detailed/numDimensions`|Number of dimensions in the segment metadata. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies.|
+|`segment/detailed/numMetrics`|Number of metrics in the segment metadata. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies.|
+|`segment/detailed/durationMillis`|The size of the interval that the segment covers in millis. Only reported if `SchemaStatsMonitor` is enabled.|`dataSource`, `segment/binaryVersion`, `segment/hasLastCompactedState`| Varies.|

Review Comment:
   Suggestion: `durationMillis` seems a little ambiguous. Maybe `segIntervalMillis`?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] suneet-s commented on a diff in pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
suneet-s commented on code in PR #12456:
URL: https://github.com/apache/druid/pull/12456#discussion_r853108276


##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/SchemaStatsMonitor.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.sql.calcite.schema;
+
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.java.util.metrics.AbstractMonitor;
+import org.apache.druid.query.DruidMetrics;
+import org.apache.druid.timeline.SegmentId;
+
+import javax.inject.Inject;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * A monitor that provides stats on segments that are visible to the {@link DruidSchema}
+ */
+public class SchemaStatsMonitor extends AbstractMonitor

Review Comment:
   I actually debated calling this `SegmentStatsMonitor`, but chose not to because the metrics are being emitted from the broker, which is not where I would expect a segment stats monitor to be. I thought that name would be confusing because as a user I would expect the SegmentStatsMonitor should be loadable in the historical since that's where all the segments are. With a name like `Schema...` I thought it would be more obvious that this should only be loaded in the broker.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] suneet-s commented on a diff in pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
suneet-s commented on code in PR #12456:
URL: https://github.com/apache/druid/pull/12456#discussion_r853117087


##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java:
##########
@@ -825,6 +837,14 @@ Map<SegmentId, AvailableSegmentMetadata> getSegmentMetadataSnapshot()
     return segmentMetadata;
   }
 
+  /**
+   * @return segmentMetadataInfo. Callers can only safely read the contents of the map.
+   */
+  public ConcurrentHashMap<String, ConcurrentSkipListMap<SegmentId, AvailableSegmentMetadata>> getSegmentMetadataInfoUnsafe()

Review Comment:
   > should we return a copy of the map rather than the actual?
   
   Yeah, I think that's the right thing to do. I will have to think about whether or not each thread should get it's own copy of the map or if it's better to just keep a single copy for reading and share that among threads.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [druid] suneet-s commented on pull request #12456: Introduce a monitor to report detailed segment metrics

Posted by GitBox <gi...@apache.org>.
suneet-s commented on PR #12456:
URL: https://github.com/apache/druid/pull/12456#issuecomment-1103211839

   Thinking about this some more - I don't like this approach. I'm going to try wiring this up into the DataSegmentChangeHandler addSegment / removeSegment callbacks so that this is more efficient for clusters with a large number of segments


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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