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/09/10 19:19:52 UTC

[GitHub] [druid] jihoonson commented on a change in pull request #10371: Auto-compaction snapshot status API

jihoonson commented on a change in pull request #10371:
URL: https://github.com/apache/druid/pull/10371#discussion_r486569512



##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java
##########
@@ -296,18 +296,87 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
 
     emitter.emit(
         new ServiceMetricEvent.Builder().build(
-            "compact/task/count",
+            "compact/task/scheduled/count",
             stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
         )
     );
 
+    emitter.emit(
+        new ServiceMetricEvent.Builder().build(
+            "compact/task/maxSlot/count",
+            stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT)
+        )
+    );
+
+    emitter.emit(
+        new ServiceMetricEvent.Builder().build(
+            "compact/task/availableSlot/count",
+            stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT)
+        )
+    );
+
+    stats.forEachDataSourceStat(
+        CompactSegments.TOTAL_SIZE_OF_SEGMENTS_AWAITING_COMPACTION,
+        (final String dataSource, final long count) -> {
+          emitter.emit(
+              new ServiceMetricEvent.Builder()
+                  .setDimension(DruidMetrics.DATASOURCE, dataSource)
+                  .build("segment/waitCompact/segmentByte", count)
+          );
+        }
+    );
+
+    stats.forEachDataSourceStat(
+        CompactSegments.TOTAL_COUNT_OF_SEGMENTS_AWAITING_COMPACTION,
+        (final String dataSource, final long count) -> {
+          emitter.emit(
+              new ServiceMetricEvent.Builder()
+                  .setDimension(DruidMetrics.DATASOURCE, dataSource)
+                  .build("segment/waitCompact/segmentCount", count)
+          );
+        }
+    );
+
+    stats.forEachDataSourceStat(
+        CompactSegments.TOTAL_INTERVAL_OF_SEGMENTS_AWAITING_COMPACTION,
+        (final String dataSource, final long count) -> {
+          emitter.emit(
+              new ServiceMetricEvent.Builder()
+                  .setDimension(DruidMetrics.DATASOURCE, dataSource)
+                  .build("segment/waitCompact/intervalCount", count)
+          );
+        }
+    );
+
+    stats.forEachDataSourceStat(
+        CompactSegments.TOTAL_SIZE_OF_SEGMENTS_COMPACTED,
+        (final String dataSource, final long count) -> {
+          emitter.emit(
+              new ServiceMetricEvent.Builder()
+                  .setDimension(DruidMetrics.DATASOURCE, dataSource)
+                  .build("segment/compacted/segmentByte", count)
+          );
+        }
+    );
+
+    stats.forEachDataSourceStat(
+        CompactSegments.TOTAL_COUNT_OF_SEGMENTS_COMPACTED,
+        (final String dataSource, final long count) -> {
+          emitter.emit(
+              new ServiceMetricEvent.Builder()
+                  .setDimension(DruidMetrics.DATASOURCE, dataSource)
+                  .build("segment/compacted/segmentCount", count)
+          );
+        }
+    );
+
     stats.forEachDataSourceStat(
-        "segmentsWaitCompact",
+        CompactSegments.TOTAL_INTERVAL_OF_SEGMENTS_COMPACTED,
         (final String dataSource, final long count) -> {
           emitter.emit(
               new ServiceMetricEvent.Builder()
                   .setDimension(DruidMetrics.DATASOURCE, dataSource)
-                  .build("segment/waitCompact/count", count)
+                  .build("segment/compacted/intervalCount", count)

Review comment:
       Even though this changes the metric name, it seems fine since this metrics has never been emitted because of the metric name mismatch (`segmentsWaitCompact` and `segmentSizeWaitCompact`).

##########
File path: server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.coordinator;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import javax.annotation.Nullable;
+import javax.validation.constraints.NotNull;
+import java.util.Objects;
+
+public class AutoCompactionSnapshot
+{
+  public enum AutoCompactionScheduleStatus
+  {
+    NOT_ENABLED,
+    RUNNING
+  }
+
+  @JsonProperty
+  private String dataSource;
+  @JsonProperty
+  private AutoCompactionScheduleStatus scheduleStatus;
+  @JsonProperty
+  private String latestScheduledTaskId;
+  @JsonProperty
+  private long byteAwaitingCompaction;
+  @JsonProperty
+  private long byteProcessed;

Review comment:
       Why not `byteCountProcessed` to align with other metrics?

##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java
##########
@@ -296,18 +296,87 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
 
     emitter.emit(
         new ServiceMetricEvent.Builder().build(
-            "compact/task/count",
+            "compact/task/scheduled/count",

Review comment:
       We shouldn't change the metric name for backwards compatibility.




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



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