Skip to content

Commit

Permalink
Restore Sink Metric Emission Behaviour: Emit them per-Sink instead of…
Browse files Browse the repository at this point in the history
… per-FireHydrant (#17170)

* Emit aggregate segment processing metrics per sink instead of firehydrant

* add docs

* minor change

* checkstyle

* Fix DefaultQueryMetricsTest

* Minor changes in SinkMetricsEmittingQueryRunner

* spotbugs

* Address review comments

* Use ImmutableSet and ImmutableMap

* Create a helper class for saving state of StubServiceEmitter

* Add SinkQuerySegmentWalkerBenchmark

* Create SegmentMetrics class for tracking segment metrics

---------

Co-authored-by: Akshat Jain <[email protected]>
  • Loading branch information
findingrish and Akshat-Jain authored Dec 18, 2024
1 parent 9ff1173 commit d5eb94d
Show file tree
Hide file tree
Showing 7 changed files with 523 additions and 68 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,156 @@
/*
* 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;

import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.core.LoggingEmitter;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
import org.apache.druid.segment.realtime.appenderator.Appenderator;
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.apache.druid.timeline.partition.LinearShardSpec;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;

import java.io.File;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;

@State(Scope.Benchmark)
@Fork(value = 1)
@Warmup(iterations = 3)
@Measurement(iterations = 5)
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public class SinkQuerySegmentWalkerBenchmark
{
static {
NullHandling.initializeForTests();
}

@Param({"10", "50", "100", "200"})
private int numFireHydrants;

private final LoggingEmitter loggingEmitter = new LoggingEmitter(new Logger(LoggingEmitter.class), LoggingEmitter.Level.INFO, new DefaultObjectMapper());
private final ServiceEmitter serviceEmitter = new ServiceEmitter("test", "test", loggingEmitter);
private File cacheDir;

private Appenderator appenderator;

@Setup(Level.Trial)
public void setup() throws Exception
{
final String userConfiguredCacheDir = System.getProperty("druid.benchmark.cacheDir", System.getenv("DRUID_BENCHMARK_CACHE_DIR"));
cacheDir = new File(userConfiguredCacheDir);
final StreamAppenderatorTester tester =
new StreamAppenderatorTester.Builder().maxRowsInMemory(1)
.basePersistDirectory(cacheDir)
.withServiceEmitter(serviceEmitter)
.build();

appenderator = tester.getAppenderator();
appenderator.startJob();

final SegmentIdWithShardSpec segmentIdWithShardSpec = new SegmentIdWithShardSpec(
StreamAppenderatorTester.DATASOURCE,
Intervals.of("2000/2001"),
"A",
new LinearShardSpec(0)
);

for (int i = 0; i < numFireHydrants; i++) {
final MapBasedInputRow inputRow = new MapBasedInputRow(
DateTimes.of("2000").getMillis(),
ImmutableList.of("dim"),
ImmutableMap.of(
"dim",
"bar_" + i,
"met",
1
)
);
appenderator.add(segmentIdWithShardSpec, inputRow, Suppliers.ofInstance(Committers.nil()));
}
}

@TearDown(Level.Trial)
public void tearDown() throws Exception
{
appenderator.close();
FileUtils.deleteDirectory(cacheDir);
}

@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void emitSinkMetrics(Blackhole blackhole) throws Exception
{
{
final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder()
.dataSource(StreamAppenderatorTester.DATASOURCE)
.intervals(ImmutableList.of(Intervals.of("2000/2001")))
.aggregators(
Arrays.asList(
new LongSumAggregatorFactory("count", "count"),
new LongSumAggregatorFactory("met", "met")
)
)
.granularity(Granularities.DAY)
.build();

final List<Result<TimeseriesResultValue>> results =
QueryPlus.wrap(query1).run(appenderator, ResponseContext.createEmpty()).toList();
blackhole.consume(results);

serviceEmitter.flush();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,10 @@
*/
public class DefaultQueryMetrics<QueryType extends Query<?>> implements QueryMetrics<QueryType>
{
public static final String QUERY_WAIT_TIME = "query/wait/time";
public static final String QUERY_SEGMENT_TIME = "query/segment/time";
public static final String QUERY_SEGMENT_AND_CACHE_TIME = "query/segmentAndCache/time";

protected final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
protected final Map<String, Number> metrics = new HashMap<>();

Expand Down Expand Up @@ -235,19 +239,19 @@ public QueryMetrics<QueryType> reportQueryBytes(long byteCount)
@Override
public QueryMetrics<QueryType> reportWaitTime(long timeNs)
{
return reportMillisTimeMetric("query/wait/time", timeNs);
return reportMillisTimeMetric(QUERY_WAIT_TIME, timeNs);
}

@Override
public QueryMetrics<QueryType> reportSegmentTime(long timeNs)
{
return reportMillisTimeMetric("query/segment/time", timeNs);
return reportMillisTimeMetric(QUERY_SEGMENT_TIME, timeNs);
}

@Override
public QueryMetrics<QueryType> reportSegmentAndCacheTime(long timeNs)
{
return reportMillisTimeMetric("query/segmentAndCache/time", timeNs);
return reportMillisTimeMetric(QUERY_SEGMENT_AND_CACHE_TIME, timeNs);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ public class StubServiceEmitter extends ServiceEmitter implements MetricsVerifie
{
private final List<Event> events = new ArrayList<>();
private final List<AlertEvent> alertEvents = new ArrayList<>();
private final ConcurrentHashMap<String, List<ServiceMetricEvent>> metricEvents = new ConcurrentHashMap<>();
private final ConcurrentHashMap<String, List<ServiceMetricEventSnapshot>> metricEvents = new ConcurrentHashMap<>();

public StubServiceEmitter()
{
Expand All @@ -56,7 +56,7 @@ public void emit(Event event)
if (event instanceof ServiceMetricEvent) {
ServiceMetricEvent metricEvent = (ServiceMetricEvent) event;
metricEvents.computeIfAbsent(metricEvent.getMetric(), name -> new ArrayList<>())
.add(metricEvent);
.add(new ServiceMetricEventSnapshot(metricEvent));
} else if (event instanceof AlertEvent) {
alertEvents.add((AlertEvent) event);
}
Expand All @@ -76,7 +76,7 @@ public List<Event> getEvents()
*
* @return Map from metric name to list of events emitted for that metric.
*/
public Map<String, List<ServiceMetricEvent>> getMetricEvents()
public Map<String, List<ServiceMetricEventSnapshot>> getMetricEvents()
{
return metricEvents;
}
Expand All @@ -96,18 +96,18 @@ public List<Number> getMetricValues(
)
{
final List<Number> values = new ArrayList<>();
final List<ServiceMetricEvent> events =
final List<ServiceMetricEventSnapshot> events =
metricEvents.getOrDefault(metricName, Collections.emptyList());
final Map<String, Object> filters =
dimensionFilters == null ? Collections.emptyMap() : dimensionFilters;
for (ServiceMetricEvent event : events) {
for (ServiceMetricEventSnapshot event : events) {
final Map<String, Object> userDims = event.getUserDims();
boolean match = filters.keySet().stream()
.map(d -> filters.get(d).equals(userDims.get(d)))
.reduce((a, b) -> a && b)
.orElse(true);
if (match) {
values.add(event.getValue());
values.add(event.getMetricEvent().getValue());
}
}

Expand All @@ -131,4 +131,32 @@ public void flush()
public void close()
{
}

/**
* Helper class to encapsulate a ServiceMetricEvent and its user dimensions.
* Since {@link StubServiceEmitter} doesn't actually emit metrics and saves the emitted metrics in-memory,
* this helper class saves a copy of {@link ServiceMetricEvent#userDims} of emitted metrics
* via {@link ServiceMetricEvent#getUserDims()} as it can get mutated.
*/
public static class ServiceMetricEventSnapshot
{
private final ServiceMetricEvent metricEvent;
private final Map<String, Object> userDims;

public ServiceMetricEventSnapshot(ServiceMetricEvent metricEvent)
{
this.metricEvent = metricEvent;
this.userDims = metricEvent.getUserDims();
}

public ServiceMetricEvent getMetricEvent()
{
return metricEvent;
}

public Map<String, Object> getUserDims()
{
return userDims;
}
}
}
Loading

0 comments on commit d5eb94d

Please sign in to comment.