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


##########
server/src/main/java/org/apache/druid/server/metrics/SegmentDiscoveryStatsProviderImpl.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.metrics;
+
+import org.apache.druid.server.coordinator.stats.Dimension;
+import org.apache.druid.server.coordinator.stats.RowKey;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.checkerframework.checker.lock.qual.GuardedBy;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class SegmentDiscoveryStatsProviderImpl implements 
SegmentDiscoveryStatsProvider

Review Comment:
   Since we just want to emit the current total count for each datasource, I 
guess you can skip adding this class.



##########
server/src/main/java/org/apache/druid/server/metrics/SegmentDiscoveryStatsMonitor.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.metrics;
+
+import com.google.inject.Inject;
+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.server.coordinator.stats.RowKey;
+
+import java.util.Map;
+
+public class SegmentDiscoveryStatsMonitor extends AbstractMonitor
+{
+  private final SegmentDiscoveryStatsProvider statsProvider;
+
+  @Inject
+  public SegmentDiscoveryStatsMonitor(SegmentDiscoveryStatsProvider 
statsProvider)
+  {
+    this.statsProvider = statsProvider;
+  }
+
+  @Override
+  public boolean doMonitor(ServiceEmitter emitter)
+  {
+    emit(emitter, "segment/discover/success", 
statsProvider.getTotalSuccessfulSegmentLoadCount());

Review Comment:
   From pov of the Broker, servers and segments may come and go. The metric 
which is useful for tests and otherwise is the total count of segments 
currently available.
   
   So, we should have a `segment/available/count` metric which gives the 
current total count rather than a delta count which denotes the number of 
segments discovered since the last emission window.
   I guess that could be useful too but we probably don't need it right now.



##########
server/src/main/java/org/apache/druid/server/metrics/SegmentDiscoveryStatsProvider.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.metrics;
+
+import org.apache.druid.server.coordinator.stats.RowKey;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Map;
+
+public interface SegmentDiscoveryStatsProvider
+{
+  /**
+   * Return the number of successful segment loads for each datasource during 
the emission period.
+   */
+  Map<RowKey, Long> getTotalSuccessfulSegmentLoadCount();

Review Comment:
   ```suggestion
     Map<RowKey, Long> getAvailableSegmentCount();
   ```



##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/EmbeddedMSQRealtimeQueryTest.java:
##########
@@ -177,7 +177,8 @@ void setUpEach()
         agg -> agg.hasSumAtLeast(totalRows)
     );
     broker.latchableEmitter().waitForEvent(
-        event -> event.hasDimension(DruidMetrics.DATASOURCE, dataSource)
+        event -> event.hasMetricName("segment/discover/success")
+                      .hasDimension(DruidMetrics.DATASOURCE, dataSource)

Review Comment:
   We should also be checking the value of the metric.



##########
server/src/main/java/org/apache/druid/server/metrics/SegmentDiscoveryStatsProviderImpl.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.metrics;
+
+import org.apache.druid.server.coordinator.stats.Dimension;
+import org.apache.druid.server.coordinator.stats.RowKey;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.checkerframework.checker.lock.qual.GuardedBy;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class SegmentDiscoveryStatsProviderImpl implements 
SegmentDiscoveryStatsProvider
+{
+
+  private final ConcurrentHashMap<RowKey, AtomicLong> 
totalSuccessfulSegmentLoadCount = new ConcurrentHashMap<>();
+  @GuardedBy("totalSuccessfulSegmentLoadCount")
+  private Map<RowKey, Long> previousSuccessfulSegmentLoadCount = new 
HashMap<>();
+
+  @Override
+  public Map<RowKey, Long> getTotalSuccessfulSegmentLoadCount()
+  {
+    final Map<RowKey, Long> total = 
CollectionUtils.mapValues(totalSuccessfulSegmentLoadCount, AtomicLong::get);
+    synchronized (totalSuccessfulSegmentLoadCount) {
+      final Map<RowKey, Long> delta = getDeltaValues(total, 
previousSuccessfulSegmentLoadCount);
+      previousSuccessfulSegmentLoadCount = total;
+      return delta;
+    }
+  }
+
+  @Override
+  public long incrementSuccessfulSegmentDiscoverCount(DataSegment segment)
+  {
+    return incrementAndGetLong(totalSuccessfulSegmentLoadCount, 
getMetricKey(segment));
+  }
+
+  private static long incrementAndGetLong(ConcurrentHashMap<RowKey, 
AtomicLong> counters, RowKey key)
+  {
+    AtomicLong counter = counters.get(key);
+    if (counter == null) {
+      counter = counters.computeIfAbsent(key, k -> new AtomicLong());
+    }
+    return counter.incrementAndGet();
+  }
+
+  private Map<RowKey, Long> getDeltaValues(Map<RowKey, Long> total, 
Map<RowKey, Long> prev)
+  {
+    final Map<RowKey, Long> deltaValues = new HashMap<>();
+    total.forEach(
+        (dataSource, totalCount) -> deltaValues.put(
+            dataSource,
+            totalCount - prev.getOrDefault(dataSource, 0L)
+        )
+    );
+    return deltaValues;
+  }
+
+  private static RowKey getMetricKey(final DataSegment segment)
+  {
+    if (segment == null) {
+      return RowKey.empty();
+    }
+    return RowKey.with(Dimension.DATASOURCE, segment.getDataSource()).build();

Review Comment:
   It would be nice to emit counts per datasource-interval-version.



##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedKafkaClusterMetricsTest.java:
##########
@@ -113,6 +113,7 @@ public void stop()
             .addProperty("druid.manager.segments.killUnused.bufferPeriod", 
"PT0.1s")
             .addProperty("druid.manager.segments.killUnused.dutyPeriod", 
"PT1s");
     coordinator.addProperty("druid.manager.segments.useIncrementalCache", 
"ifSynced");
+    broker.addProperty("druid.monitoring.monitors", 
"[\"org.apache.druid.server.metrics.SegmentDiscoveryStatsMonitor\"]");

Review Comment:
   If it is not being used in the test, we can skip adding it.



##########
server/src/main/java/org/apache/druid/server/metrics/SegmentDiscoveryStatsMonitor.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.metrics;
+
+import com.google.inject.Inject;
+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.server.coordinator.stats.RowKey;
+
+import java.util.Map;
+
+public class SegmentDiscoveryStatsMonitor extends AbstractMonitor

Review Comment:
   A better name would probably be `BrokerSegmentCountStatsMonitor`



##########
server/src/main/java/org/apache/druid/server/metrics/NoopSegmentDiscoveryStatsProviderImpl.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metrics;
+
+import org.apache.druid.server.coordinator.stats.RowKey;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Map;
+
+public class NoopSegmentDiscoveryStatsProviderImpl implements 
SegmentDiscoveryStatsProvider

Review Comment:
   The Noop impl might not be needed too since the interface really needs just 
the one method.



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