kfaraz commented on code in PR #17847:
URL: https://github.com/apache/druid/pull/17847#discussion_r2066483481


##########
indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java:
##########
@@ -819,13 +819,13 @@ private TaskStatus generateAndPublishSegments(
       final PartitionAnalysis partitionAnalysis
   ) throws IOException, InterruptedException
   {
-    final SegmentGenerationMetrics buildSegmentsSegmentGenerationMetrics = new 
SegmentGenerationMetrics();
+    final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
+    final SegmentGenerationMetrics buildSegmentsSegmentGenerationMetrics = new 
SegmentGenerationMetrics(tuningConfig.getMessageGapAggStatsEnabled());

Review Comment:
   The message gap will only ever be emitted for streaming tasks. It should not 
be a part of `IndexTuningConfig` and we should always pass false to the 
`SegmentGenerationMetrics` here.



##########
server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java:
##########
@@ -288,6 +295,15 @@ public Object startJob()
     initializeExecutors();
     resetNextFlush();
     sinkSchemaAnnouncer.start();
+
+    if (timeExecutor != null) {
+      timeExecutor.scheduleAtFixedRate(
+          () -> currTimeMs = System.currentTimeMillis(),
+          0,
+          1,
+          TimeUnit.MILLISECONDS
+      );

Review Comment:
   This doesn't seem like the right approach. We shouldn't need a separate 
executor just to update the `currTimeMs`.
   
   



##########
benchmarks/src/test/java/org/apache/druid/benchmark/indexing/StreamAppenderatorBenchmark.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.benchmark.indexing;
+
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.segment.incremental.SimpleRowIngestionMeters;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
+import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorTester;
+import org.apache.druid.segment.realtime.sink.Committers;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.infra.Blackhole;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+import java.io.IOException;
+
+@Warmup(iterations = 15)
+@Measurement(iterations = 15)
+public class StreamAppenderatorBenchmark extends AppenderatorBenchmark

Review Comment:
   @jtuglu-netflix , would you mind moving the benchmarks to a separate PR and 
keeping this PR solely for the new metrics?
   
   I am assuming that the benchmarks don't have anything to do with these 
metrics in particular. Please correct me if I am missing something.



##########
indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java:
##########
@@ -132,6 +132,19 @@ public boolean doMonitor(ServiceEmitter emitter)
       emitter.emit(builder.setMetric("ingest/events/messageGap", messageGap));
     }
 
+    final long minMessageGap = metrics.minMessageGap();
+    final long maxMessageGap = metrics.maxMessageGap();
+    // Best-effort way to ensure parity amongst emitted metrics
+    if (metrics.isMessageGapAggStatsEnabled()) {
+      if (minMessageGap != Long.MAX_VALUE) {
+        emitter.emit(builder.setMetric("ingest/events/minMessageGap", 
minMessageGap));

Review Comment:
   Does the min message gap really add any value?
   I don't imagine any SLAs relying on the min message gap.
   I think just average and max should be enough.
   
   And since these are only two new metrics reported in every metric emission 
period, I think we should not need to add a new config. It is fine to emit new 
metrics as long as we are not changing values of any of the existing metrics.



##########
server/src/main/java/org/apache/druid/segment/realtime/SegmentGenerationMetrics.java:
##########
@@ -105,6 +123,14 @@ public void setSinkCount(long sinkCount)
     this.sinkCount.set(sinkCount);
   }
 
+  public void reportMessageGap(final long messageGap)
+  {
+    final long numEvent = this.numMessageGap.incrementAndGet();
+    this.avgMessageGap.getAndUpdate(oldAvg -> oldAvg + ((messageGap - oldAvg) 
/ numEvent));

Review Comment:
   I would agree with @maytasm  on this.
   Just keep a `AtomicDouble totalMessageGap` (to avoid the overflow problem)
   and the total number of events.
   Do not compute the average until it needs to be reported.



##########
server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java:
##########
@@ -337,6 +353,12 @@ public AppenderatorAddResult add(
       throw e;
     }
 
+    // cache volatile locally so it's likely to be a register read later
+    final long systemTime = currTimeMs;
+    if (messageGapAggStats) {
+      metrics.reportMessageGap(systemTime - row.getTimestampFromEpoch());

Review Comment:
   The logic of computing the actual message gap should live inside 
`SegmentGenerationMetrics` itself. The code here should just pass the row 
timestamp.



##########
indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java:
##########
@@ -1291,7 +1293,8 @@ public IndexTuningConfig(
         @JsonProperty("maxSavedParseExceptions") @Nullable Integer 
maxSavedParseExceptions,
         @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge,
         @JsonProperty("awaitSegmentAvailabilityTimeoutMillis") @Nullable Long 
awaitSegmentAvailabilityTimeoutMillis,
-        @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads
+        @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads,
+        @JsonProperty("messageGapAggStatsEnabled") @Nullable Boolean 
messageGapAggStatsEnabled

Review Comment:
   `IndexTask` and `IndexTuningConfig` should not have this field as it is 
relevant only for streaming tasks.



##########
server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java:
##########
@@ -354,40 +376,43 @@ public AppenderatorAddResult add(
 
     boolean isPersistRequired = false;
     boolean persist = false;
-    List<String> persistReasons = new ArrayList<>();
+    final String[] persistReasons = new String[NUMBER_OF_PERSIST_REASONS];

Review Comment:
   Why does this need to be changed to an array?



##########
server/src/main/java/org/apache/druid/segment/realtime/SegmentGenerationMetrics.java:
##########
@@ -55,6 +56,23 @@ public class SegmentGenerationMetrics
 
   private final AtomicLong maxSegmentHandoffTime = new 
AtomicLong(NO_EMIT_SEGMENT_HANDOFF_TIME);
 
+  // Message gap accounting
+  private final AtomicLong minMessageGap = new AtomicLong(Long.MAX_VALUE);
+  private final AtomicLong maxMessageGap = new AtomicLong(Long.MIN_VALUE);
+  private final AtomicLong numMessageGap = new AtomicLong(0);
+  private final AtomicDouble avgMessageGap = new AtomicDouble(0);

Review Comment:
   Instead of these two, I would advise keeping an AtomicReference of a pair of 
total sum (double) and total number of events.
   
   If we keep two separate atomic fields, the snapshotting can potentially be 
inconsistent due to race conditions.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to