From d3768ada6ec516602e130e952c45b3d393249161 Mon Sep 17 00:00:00 2001 From: Misha Date: Tue, 17 Feb 2026 16:14:47 +0100 Subject: [PATCH 01/12] metrics filtering in LoggingEmitter --- docs/configuration/index.md | 2 + processing/pom.xml | 3 + .../java/util/emitter/core/Emitters.java | 7 + .../util/emitter/core/LoggingEmitter.java | 94 +++++- .../emitter/core/LoggingEmitterConfig.java | 18 + .../resources/defaultLoggingMetricNames.json | 308 ++++++++++++++++++ .../core/LoggingEmitterConfigTest.java | 12 + .../util/emitter/core/LoggingEmitterTest.java | 68 ++++ 8 files changed, 511 insertions(+), 1 deletion(-) create mode 100644 processing/src/main/resources/defaultLoggingMetricNames.json create mode 100644 processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java diff --git a/docs/configuration/index.md b/docs/configuration/index.md index f6ddebcb0b24..952f390984a1 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -2019,6 +2019,8 @@ log4j config to route these logs to different sources based on the feed of the e |--------|-----------|--------| |`druid.emitter.logging.loggerClass`|The class used for logging.|`org.apache.druid.java.util.emitter.core.LoggingEmitter`| |`druid.emitter.logging.logLevel`|Choices: debug, info, warn, error. The log level at which message are logged.|info| +|`druid.emitter.logging.filterMetrics`|If `true`, only metric events present in the built-in or custom allowlist are logged.|`false`| +|`druid.emitter.logging.metricAllowlistPath`|Optional path to a custom metric allowlist JSON file. If unset, the built-in `defaultLoggingMetricNames.json` from the classpath is used. The file must be a JSON array of metric names.|not set| #### HTTP emitter module diff --git a/processing/pom.xml b/processing/pom.xml index dcc9da8a5de6..3bf60b04387d 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -531,6 +531,9 @@ + + src/main/resources + ${project.build.directory}/hyperic-sigar-${sigar.base.version}/sigar-bin/lib diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java index 00d424b88fc4..2cc231637c3c 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java @@ -140,6 +140,13 @@ static Map makeLoggingMap(Properties props) loggingMap.put( "logLevel", props.getProperty("org.apache.druid.java.util.emitter.logging.level", "debug") ); + loggingMap.put( + "filterMetrics", + Boolean.parseBoolean(props.getProperty("org.apache.druid.java.util.emitter.logging.filterMetrics", "false")) + ); + loggingMap.put( + "metricAllowlistPath", props.getProperty("org.apache.druid.java.util.emitter.logging.metricAllowlistPath") + ); return loggingMap; } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java index afc8e07e2c28..bb840c8df2b0 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java @@ -22,13 +22,24 @@ /** */ +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.slf4j.MarkerFactory; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.util.Set; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicBoolean; @@ -36,22 +47,39 @@ */ public class LoggingEmitter implements Emitter { + private static final String DEFAULT_METRIC_ALLOWLIST_PATH = "defaultLoggingMetricNames.json"; + private final Logger log; private final Level level; private final ObjectMapper jsonMapper; + private final boolean filterMetrics; + private final Set metricAllowlist; private final AtomicBoolean started = new AtomicBoolean(false); public LoggingEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper) { - this(new Logger(config.getLoggerClass()), Level.toLevel(config.getLogLevel()), jsonMapper); + this( + new Logger(config.getLoggerClass()), + Level.toLevel(config.getLogLevel()), + jsonMapper, + config.isFilterMetrics(), + config.isFilterMetrics() ? loadMetricAllowlist(jsonMapper, config.getMetricAllowlistPath()) : ImmutableSet.of() + ); } public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper) + { + this(log, level, jsonMapper, false, ImmutableSet.of()); + } + + public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper, boolean filterMetrics, Set metricAllowlist) { this.log = log; this.level = level; this.jsonMapper = jsonMapper; + this.filterMetrics = filterMetrics; + this.metricAllowlist = metricAllowlist; } @Override @@ -95,6 +123,9 @@ public void emit(Event event) throw new RejectedExecutionException("Service not started."); } } + if (shouldFilterOutMetric(event)) { + return; + } try { switch (level) { case TRACE: @@ -170,9 +201,70 @@ public String toString() return "LoggingEmitter{" + "log=" + log + ", level=" + level + + ", filterMetrics=" + filterMetrics + '}'; } + private boolean shouldFilterOutMetric(Event event) + { + if (!filterMetrics || !(event instanceof ServiceMetricEvent)) { + return false; + } + final ServiceMetricEvent metricEvent = (ServiceMetricEvent) event; + return !metricAllowlist.contains(metricEvent.getMetric()); + } + + private static Set loadMetricAllowlist(ObjectMapper mapper, String metricAllowlistPath) + { + final String source = Strings.isNullOrEmpty(metricAllowlistPath) ? DEFAULT_METRIC_ALLOWLIST_PATH : metricAllowlistPath; + try (final InputStream is = openAllowlistFile(metricAllowlistPath)) { + final JsonNode metricConfig = mapper.readTree(is); + if (metricConfig.isArray()) { + final ImmutableSet.Builder metricNames = ImmutableSet.builder(); + for (JsonNode metric : metricConfig) { + if (!metric.isTextual()) { + throw new ISE("Metric allowlist file [%s] contains a non-string metric name", source); + } + metricNames.add(metric.asText()); + } + return metricNames.build(); + } else { + throw new ISE("Metric allowlist file [%s] must be a JSON array of metric names", source); + } + } + catch (IOException e) { + throw new ISE(e, "Failed to parse metric allowlist file [%s]", source); + } + } + + private static InputStream openAllowlistFile(String metricAllowlistPath) throws IOException + { + if (Strings.isNullOrEmpty(metricAllowlistPath)) { + return openDefaultAllowlistFile(); + } + try { + return new FileInputStream(metricAllowlistPath); + } + catch (FileNotFoundException e) { + return openDefaultAllowlistFile(); + } + } + + private static InputStream openDefaultAllowlistFile() + { + final InputStream is = LoggingEmitter.class.getClassLoader().getResourceAsStream(DEFAULT_METRIC_ALLOWLIST_PATH); + if (is == null) { + throw new ISE("Metric allowlist file [%s] not found on classpath", DEFAULT_METRIC_ALLOWLIST_PATH); + } + return is; + } + + @VisibleForTesting + Set getMetricAllowlist() + { + return metricAllowlist; + } + public enum Level { TRACE, diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java index 39b85126f773..54fe5407262f 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java @@ -35,6 +35,12 @@ public class LoggingEmitterConfig @JsonProperty private String logLevel = "info"; + @JsonProperty + private boolean filterMetrics; + + @JsonProperty + private String metricAllowlistPath; + public String getLoggerClass() { return loggerClass; @@ -45,12 +51,24 @@ public String getLogLevel() return logLevel; } + public boolean isFilterMetrics() + { + return filterMetrics; + } + + public String getMetricAllowlistPath() + { + return metricAllowlistPath; + } + @Override public String toString() { return "LoggingEmitterConfig{" + "loggerClass='" + loggerClass + '\'' + ", logLevel='" + logLevel + '\'' + + ", filterMetrics=" + filterMetrics + + ", metricAllowlistPath='" + metricAllowlistPath + '\'' + '}'; } } diff --git a/processing/src/main/resources/defaultLoggingMetricNames.json b/processing/src/main/resources/defaultLoggingMetricNames.json new file mode 100644 index 000000000000..2df9dc102b83 --- /dev/null +++ b/processing/src/main/resources/defaultLoggingMetricNames.json @@ -0,0 +1,308 @@ +[ + "cgroup/cpu/cores_quota", + "cgroup/cpu/shares", + "cgroup/cpu/usage/sys/percentage", + "cgroup/cpu/usage/total/percentage", + "cgroup/cpu/usage/user/percentage", + "cgroup/cpuset/cpu_count", + "cgroup/cpuset/effective_cpu_count", + "cgroup/cpuset/effective_mems_count", + "cgroup/cpuset/mems_count", + "cgroup/disk/read/count", + "cgroup/disk/read/size", + "cgroup/disk/write/count", + "cgroup/disk/write/size", + "cgroup/memory/limit/bytes", + "cgroup/memory/usage/bytes", + "compact/segmentAnalyzer/fetchAndProcessMillis", + "compact/task/count", + "compactTask/availableSlot/count", + "compactTask/maxSlot/count", + "coordinator/global/time", + "coordinator/time", + "groupBy/maxMergeDictionarySize", + "groupBy/maxSpilledBytes", + "groupBy/mergeDictionarySize", + "groupBy/spilledBytes", + "groupBy/spilledQueries", + "ingest/count", + "ingest/events/avgMessageGap", + "ingest/events/duplicate", + "ingest/events/maxMessageGap", + "ingest/events/messageGap", + "ingest/events/minMessageGap", + "ingest/events/processed", + "ingest/events/processedWithError", + "ingest/events/thrownAway", + "ingest/events/unparseable", + "ingest/handoff/count", + "ingest/handoff/failed", + "ingest/handoff/time", + "ingest/input/bytes", + "ingest/kafka/avgLag", + "ingest/kafka/avgLag/time", + "ingest/kafka/fetchOffsets/time", + "ingest/kafka/lag", + "ingest/kafka/lag/time", + "ingest/kafka/maxLag", + "ingest/kafka/maxLag/time", + "ingest/kafka/partitionLag", + "ingest/kinesis/avgLag/time", + "ingest/kinesis/fetchOffsets/time", + "ingest/kinesis/lag/time", + "ingest/kinesis/maxLag/time", + "ingest/kinesis/partitionLag/time", + "ingest/merge/cpu", + "ingest/merge/time", + "ingest/notices/queueSize", + "ingest/notices/time", + "ingest/pause/time", + "ingest/persists/backPressure", + "ingest/persists/count", + "ingest/persists/cpu", + "ingest/persists/failed", + "ingest/persists/time", + "ingest/rows/output", + "ingest/segments/count", + "ingest/shuffle/bytes", + "ingest/shuffle/requests", + "ingest/sink/count", + "ingest/tombstones/count", + "interval/compacted/count", + "interval/skipCompact/count", + "interval/waitCompact/count", + "jetty/numOpenConnections", + "jetty/threadPool/busy", + "jetty/threadPool/idle", + "jetty/threadPool/isLowOnThreads", + "jetty/threadPool/max", + "jetty/threadPool/min", + "jetty/threadPool/queueSize", + "jetty/threadPool/ready", + "jetty/threadPool/total", + "jetty/threadPool/utilizationRate", + "jetty/threadPool/utilized", + "jvm/bufferpool/capacity", + "jvm/bufferpool/count", + "jvm/bufferpool/used", + "jvm/gc/count", + "jvm/gc/cpu", + "jvm/mem/committed", + "jvm/mem/init", + "jvm/mem/max", + "jvm/mem/used", + "jvm/pool/committed", + "jvm/pool/init", + "jvm/pool/max", + "jvm/pool/used", + "kill/eligibleUnusedSegments/count", + "kill/pendingSegments/count", + "kill/task/count", + "killTask/availableSlot/count", + "killTask/maxSlot/count", + "mergeBuffer/acquisitionTimeNs", + "mergeBuffer/maxAcquisitionTimeNs", + "mergeBuffer/pendingRequests", + "mergeBuffer/queries", + "mergeBuffer/used", + "metadata/kill/audit/count", + "metadata/kill/compaction/count", + "metadata/kill/datasource/count", + "metadata/kill/rule/count", + "metadata/kill/supervisor/count", + "metadatacache/init/time", + "metadatacache/schemaPoll/time", + "query/byteLimit/exceeded/count", + "query/bytes", + "query/cache/memcached/delta", + "query/cache/memcached/total", + "query/count", + "query/cpu/time", + "query/failed/count", + "query/interrupted/count", + "query/node/bytes", + "query/node/time", + "query/node/ttfb", + "query/priority", + "query/resultCache/hit", + "query/rowLimit/exceeded/count", + "query/segment/time", + "query/segmentAndCache/time", + "query/segments/count", + "query/success/count", + "query/time", + "query/timeout/count", + "query/wait/time", + "s3/upload/part/queueSize", + "s3/upload/part/queuedTime", + "s3/upload/part/time", + "s3/upload/total/bytes", + "s3/upload/total/time", + "segment/added/bytes", + "segment/assignSkipped/count", + "segment/assigned/count", + "segment/availableDeepStorageOnly/count", + "segment/compacted/bytes", + "segment/compacted/count", + "segment/count", + "segment/deleted/count", + "segment/dropQueue/count", + "segment/dropSkipped/count", + "segment/dropped/count", + "segment/kill/jobsProcessed/count", + "segment/kill/queueProcess/time", + "segment/kill/queueReset/time", + "segment/kill/skippedIntervals/count", + "segment/kill/unusedIntervals/count", + "segment/killed/deepStorage/count", + "segment/killed/metadataStore/count", + "segment/loadQueue/assigned", + "segment/loadQueue/cancelled", + "segment/loadQueue/count", + "segment/loadQueue/failed", + "segment/loadQueue/size", + "segment/loadQueue/success", + "segment/loading/rateKbps", + "segment/max", + "segment/metadataCache/dataSource/deleted", + "segment/metadataCache/deleted", + "segment/metadataCache/interval/count", + "segment/metadataCache/pending/count", + "segment/metadataCache/pending/deleted", + "segment/metadataCache/pending/skipped", + "segment/metadataCache/pending/updated", + "segment/metadataCache/skipped", + "segment/metadataCache/sync/time", + "segment/metadataCache/transactions/readOnly", + "segment/metadataCache/transactions/readWrite", + "segment/metadataCache/transactions/writeOnly", + "segment/metadataCache/used/count", + "segment/metadataCache/used/stale", + "segment/metadataCache/used/updated", + "segment/moveSkipped/count", + "segment/moved/bytes", + "segment/moved/count", + "segment/nuked/bytes", + "segment/overShadowed/count", + "segment/pending/count", + "segment/pendingDelete", + "segment/rowCount/avg", + "segment/rowCount/range/count", + "segment/scan/active", + "segment/scan/pending", + "segment/schemaCache/backfill/count", + "segment/schemaCache/dataSource/removed", + "segment/schemaCache/deepStorageOnly/count", + "segment/schemaCache/deepStorageOnly/refresh/time", + "segment/schemaCache/pendingBackfill/count", + "segment/schemaCache/poll/count", + "segment/schemaCache/poll/failed", + "segment/schemaCache/realtime/count", + "segment/schemaCache/refresh/count", + "segment/schemaCache/refresh/time", + "segment/schemaCache/refreshSkipped/count", + "segment/schemaCache/rowSignature/changed", + "segment/schemaCache/rowSignature/column/count", + "segment/schemaCache/used/count", + "segment/schemaCache/usedFingerprint/count", + "segment/size", + "segment/skipCompact/bytes", + "segment/skipCompact/count", + "segment/unavailable/count", + "segment/underReplicated/count", + "segment/unneeded/count", + "segment/unneededEternityTombstone/count", + "segment/used", + "segment/used/count", + "segment/used/deepStorageOnly/count", + "segment/usedPercent", + "segment/waitCompact/bytes", + "segment/waitCompact/count", + "serverview/init/time", + "serverview/sync/healthy", + "serverview/sync/unstableTime", + "sqlQuery/bytes", + "sqlQuery/planningTimeMs", + "sqlQuery/time", + "subquery/byteLimit/count", + "subquery/bytes", + "subquery/fallback/count", + "subquery/fallback/insufficientType/count", + "subquery/fallback/unknownReason/count", + "subquery/rowLimit/count", + "subquery/rows", + "sys/cpu", + "sys/disk/queue", + "sys/disk/read/count", + "sys/disk/read/size", + "sys/disk/transferTime", + "sys/disk/write/count", + "sys/disk/write/size", + "sys/fs/files/count", + "sys/fs/files/free", + "sys/fs/max", + "sys/fs/used", + "sys/mem/free", + "sys/mem/max", + "sys/mem/used", + "sys/net/read/dropped", + "sys/net/read/errors", + "sys/net/read/packets", + "sys/net/read/size", + "sys/net/write/collisions", + "sys/net/write/errors", + "sys/net/write/packets", + "sys/net/write/size", + "sys/storage/used", + "sys/swap/free", + "sys/swap/max", + "sys/swap/pageIn", + "sys/swap/pageOut", + "sys/tcpv4/activeOpens", + "sys/tcpv4/attemptFails", + "sys/tcpv4/estabResets", + "sys/tcpv4/in/errs", + "sys/tcpv4/in/segs", + "sys/tcpv4/out/rsts", + "sys/tcpv4/out/segs", + "sys/tcpv4/passiveOpens", + "sys/tcpv4/retrans/segs", + "sys/uptime", + "task/action/batch/attempts", + "task/action/batch/queueTime", + "task/action/batch/runTime", + "task/action/batch/size", + "task/action/failed/count", + "task/action/run/time", + "task/action/success/count", + "task/autoScaler/requiredCount", + "task/autoScaler/scaleActionTime", + "task/failed/count", + "task/pending/count", + "task/pending/time", + "task/run/time", + "task/running/count", + "task/segmentAvailability/wait/time", + "task/success/count", + "task/waiting/count", + "task/waiting/time", + "taskSlot/blacklisted/count", + "taskSlot/idle/count", + "taskSlot/lazy/count", + "taskSlot/total/count", + "taskSlot/used/count", + "tier/historical/count", + "tier/replication/factor", + "tier/required/capacity", + "tier/total/capacity", + "worker/task/assigned/count", + "worker/task/completed/count", + "worker/task/failed/count", + "worker/task/running/count", + "worker/task/success/count", + "worker/taskSlot/idle/count", + "worker/taskSlot/total/count", + "worker/taskSlot/used/count", + "zk/connected", + "zk/reconnect/time" +] diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java index 9aa93d5d0f5a..5886d6634d36 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java @@ -38,6 +38,8 @@ public void testDefaults() ); Assert.assertEquals("getLoggerClass", LoggingEmitter.class.getName(), config.getLoggerClass()); Assert.assertEquals("getLogLevel", "info", config.getLogLevel()); + Assert.assertFalse("isFilterMetrics", config.isFilterMetrics()); + Assert.assertNull("getMetricAllowlistPath", config.getMetricAllowlistPath()); } @Test @@ -52,6 +54,8 @@ public void testDefaultsLegacy() Assert.assertEquals("getLoggerClass", LoggingEmitter.class.getName(), config.getLoggerClass()); Assert.assertEquals("getLogLevel", "debug", config.getLogLevel()); + Assert.assertFalse("isFilterMetrics", config.isFilterMetrics()); + Assert.assertNull("getMetricAllowlistPath", config.getMetricAllowlistPath()); } @Test @@ -60,6 +64,8 @@ public void testSettingEverything() final Properties props = new Properties(); props.setProperty("org.apache.druid.java.util.emitter.loggerClass", "Foo"); props.setProperty("org.apache.druid.java.util.emitter.logLevel", "INFO"); + props.setProperty("org.apache.druid.java.util.emitter.filterMetrics", "true"); + props.setProperty("org.apache.druid.java.util.emitter.metricAllowlistPath", "/tmp/logging-metrics.json"); final ObjectMapper objectMapper = new ObjectMapper(); final LoggingEmitterConfig config = objectMapper.convertValue( @@ -69,6 +75,8 @@ public void testSettingEverything() Assert.assertEquals("getLoggerClass", "Foo", config.getLoggerClass()); Assert.assertEquals("getLogLevel", "INFO", config.getLogLevel()); + Assert.assertTrue("isFilterMetrics", config.isFilterMetrics()); + Assert.assertEquals("getMetricAllowlistPath", "/tmp/logging-metrics.json", config.getMetricAllowlistPath()); } @Test @@ -77,6 +85,8 @@ public void testSettingEverythingLegacy() final Properties props = new Properties(); props.setProperty("org.apache.druid.java.util.emitter.logging.class", "Foo"); props.setProperty("org.apache.druid.java.util.emitter.logging.level", "INFO"); + props.setProperty("org.apache.druid.java.util.emitter.logging.filterMetrics", "true"); + props.setProperty("org.apache.druid.java.util.emitter.logging.metricAllowlistPath", "/tmp/logging-metrics.json"); final ObjectMapper objectMapper = new ObjectMapper(); final LoggingEmitterConfig config = objectMapper.convertValue( @@ -86,5 +96,7 @@ public void testSettingEverythingLegacy() Assert.assertEquals("getLoggerClass", "Foo", config.getLoggerClass()); Assert.assertEquals("getLogLevel", "INFO", config.getLogLevel()); + Assert.assertTrue("isFilterMetrics", config.isFilterMetrics()); + Assert.assertEquals("getMetricAllowlistPath", "/tmp/logging-metrics.json", config.getMetricAllowlistPath()); } } diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java new file mode 100644 index 000000000000..b92de35624ed --- /dev/null +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java @@ -0,0 +1,68 @@ +/* + * 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.java.util.emitter.core; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +public class LoggingEmitterTest +{ + @Test + public void testFallsBackToDefaultAllowlistWhenCustomPathIsMissing() + { + final Map configMap = new HashMap<>(); + configMap.put("filterMetrics", true); + configMap.put("metricAllowlistPath", "/tmp/missing-" + UUID.randomUUID() + ".json"); + final LoggingEmitterConfig config = new ObjectMapper().convertValue(configMap, LoggingEmitterConfig.class); + + final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); + Assert.assertNotNull(emitter); + Assert.assertEquals(306, emitter.getMetricAllowlist().size()); + Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/cpu")); + Assert.assertTrue(emitter.getMetricAllowlist().contains("query/time")); + } + + @Test + public void testReadsCustomAllowlistAsMetricNameArray() throws IOException + { + final Path allowlist = Files.createTempFile("allowlist-array", ".json"); + Files.writeString(allowlist, "[\"jvm/gc/cpu\", \"jvm/gc/count\"]"); + + final Map configMap = new HashMap<>(); + configMap.put("filterMetrics", true); + configMap.put("metricAllowlistPath", allowlist.toAbsolutePath().toString()); + final LoggingEmitterConfig config = new ObjectMapper().convertValue(configMap, LoggingEmitterConfig.class); + + final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); + Assert.assertNotNull(emitter); + Assert.assertEquals(2, emitter.getMetricAllowlist().size()); + Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/cpu")); + Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/count")); + Assert.assertFalse(emitter.getMetricAllowlist().contains("query/time")); + } +} From 618a540e585e99a4fe1b6ef3918134396ff9a936 Mon Sep 17 00:00:00 2001 From: Misha Date: Tue, 17 Feb 2026 16:47:26 +0100 Subject: [PATCH 02/12] static checks fix --- docs/configuration/index.md | 4 ++-- .../apache/druid/java/util/emitter/core/LoggingEmitter.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 952f390984a1..054a640a3c59 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -2019,8 +2019,8 @@ log4j config to route these logs to different sources based on the feed of the e |--------|-----------|--------| |`druid.emitter.logging.loggerClass`|The class used for logging.|`org.apache.druid.java.util.emitter.core.LoggingEmitter`| |`druid.emitter.logging.logLevel`|Choices: debug, info, warn, error. The log level at which message are logged.|info| -|`druid.emitter.logging.filterMetrics`|If `true`, only metric events present in the built-in or custom allowlist are logged.|`false`| -|`druid.emitter.logging.metricAllowlistPath`|Optional path to a custom metric allowlist JSON file. If unset, the built-in `defaultLoggingMetricNames.json` from the classpath is used. The file must be a JSON array of metric names.|not set| +|`druid.emitter.logging.filterMetrics`|If `true`, only metric events present in the built-in or custom allow list are logged.|`false`| +|`druid.emitter.logging.metricAllowlistPath`|Optional path to a custom metric allow list JSON file. If unset, the built-in `defaultLoggingMetricNames.json` from the classpath is used. The file must be a JSON array of metric names.|not set| #### HTTP emitter module diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java index bb840c8df2b0..813bc723950d 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java @@ -237,7 +237,7 @@ private static Set loadMetricAllowlist(ObjectMapper mapper, String metri } } - private static InputStream openAllowlistFile(String metricAllowlistPath) throws IOException + private static InputStream openAllowlistFile(String metricAllowlistPath) { if (Strings.isNullOrEmpty(metricAllowlistPath)) { return openDefaultAllowlistFile(); From 716d337edeb3bfa518ca70184d355414d2d9af74 Mon Sep 17 00:00:00 2001 From: Misha Date: Thu, 19 Feb 2026 08:28:01 +0100 Subject: [PATCH 03/12] fix review comments --- docs/configuration/index.md | 2 +- .../util/emitter/core/LoggingEmitter.java | 39 ++- .../resources/defaultLoggingMetricNames.json | 308 ------------------ .../main/resources/defaultLoggingMetrics.json | 308 ++++++++++++++++++ .../LoggingEmitterAllowlistConfigTest.java | 67 ++++ .../util/emitter/core/LoggingEmitterTest.java | 156 +++++++-- 6 files changed, 528 insertions(+), 352 deletions(-) delete mode 100644 processing/src/main/resources/defaultLoggingMetricNames.json create mode 100644 processing/src/main/resources/defaultLoggingMetrics.json create mode 100644 processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 054a640a3c59..9a156a8b2009 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -2020,7 +2020,7 @@ log4j config to route these logs to different sources based on the feed of the e |`druid.emitter.logging.loggerClass`|The class used for logging.|`org.apache.druid.java.util.emitter.core.LoggingEmitter`| |`druid.emitter.logging.logLevel`|Choices: debug, info, warn, error. The log level at which message are logged.|info| |`druid.emitter.logging.filterMetrics`|If `true`, only metric events present in the built-in or custom allow list are logged.|`false`| -|`druid.emitter.logging.metricAllowlistPath`|Optional path to a custom metric allow list JSON file. If unset, the built-in `defaultLoggingMetricNames.json` from the classpath is used. The file must be a JSON array of metric names.|not set| +|`druid.emitter.logging.metricAllowlistPath`|Optional path to a custom metric allow list JSON file. If unset, the built-in `defaultLoggingMetrics.json` from the classpath is used. The file must be a JSON object where keys are metric names (for example, `{"query/time":[]}`).|not set| #### HTTP emitter module diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java index 813bc723950d..b2c01876af95 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java @@ -27,6 +27,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; @@ -47,7 +48,7 @@ */ public class LoggingEmitter implements Emitter { - private static final String DEFAULT_METRIC_ALLOWLIST_PATH = "defaultLoggingMetricNames.json"; + private static final String DEFAULT_METRIC_ALLOWLIST_PATH = "defaultLoggingMetrics.json"; private final Logger log; private final Level level; @@ -73,6 +74,23 @@ public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper) this(log, level, jsonMapper, false, ImmutableSet.of()); } + public LoggingEmitter( + Logger log, + Level level, + ObjectMapper jsonMapper, + boolean shouldFilterMetrics, + String allowedMetricsPath + ) + { + this( + log, + level, + jsonMapper, + shouldFilterMetrics, + shouldFilterMetrics ? loadMetricAllowlist(jsonMapper, allowedMetricsPath) : ImmutableSet.of() + ); + } + public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper, boolean filterMetrics, Set metricAllowlist) { this.log = log; @@ -219,18 +237,12 @@ private static Set loadMetricAllowlist(ObjectMapper mapper, String metri final String source = Strings.isNullOrEmpty(metricAllowlistPath) ? DEFAULT_METRIC_ALLOWLIST_PATH : metricAllowlistPath; try (final InputStream is = openAllowlistFile(metricAllowlistPath)) { final JsonNode metricConfig = mapper.readTree(is); - if (metricConfig.isArray()) { - final ImmutableSet.Builder metricNames = ImmutableSet.builder(); - for (JsonNode metric : metricConfig) { - if (!metric.isTextual()) { - throw new ISE("Metric allowlist file [%s] contains a non-string metric name", source); - } - metricNames.add(metric.asText()); - } - return metricNames.build(); - } else { - throw new ISE("Metric allowlist file [%s] must be a JSON array of metric names", source); + if (!metricConfig.isObject()) { + throw new ISE("Metric allowlist file [%s] must be a JSON object of metric names", source); } + final ImmutableSet.Builder metricNames = ImmutableSet.builder(); + metricConfig.fieldNames().forEachRemaining(metricNames::add); + return metricNames.build(); } catch (IOException e) { throw new ISE(e, "Failed to parse metric allowlist file [%s]", source); @@ -246,7 +258,8 @@ private static InputStream openAllowlistFile(String metricAllowlistPath) return new FileInputStream(metricAllowlistPath); } catch (FileNotFoundException e) { - return openDefaultAllowlistFile(); + throw new IAE(e, "Metric allowlist file [%s] not found", metricAllowlistPath); + } } diff --git a/processing/src/main/resources/defaultLoggingMetricNames.json b/processing/src/main/resources/defaultLoggingMetricNames.json deleted file mode 100644 index 2df9dc102b83..000000000000 --- a/processing/src/main/resources/defaultLoggingMetricNames.json +++ /dev/null @@ -1,308 +0,0 @@ -[ - "cgroup/cpu/cores_quota", - "cgroup/cpu/shares", - "cgroup/cpu/usage/sys/percentage", - "cgroup/cpu/usage/total/percentage", - "cgroup/cpu/usage/user/percentage", - "cgroup/cpuset/cpu_count", - "cgroup/cpuset/effective_cpu_count", - "cgroup/cpuset/effective_mems_count", - "cgroup/cpuset/mems_count", - "cgroup/disk/read/count", - "cgroup/disk/read/size", - "cgroup/disk/write/count", - "cgroup/disk/write/size", - "cgroup/memory/limit/bytes", - "cgroup/memory/usage/bytes", - "compact/segmentAnalyzer/fetchAndProcessMillis", - "compact/task/count", - "compactTask/availableSlot/count", - "compactTask/maxSlot/count", - "coordinator/global/time", - "coordinator/time", - "groupBy/maxMergeDictionarySize", - "groupBy/maxSpilledBytes", - "groupBy/mergeDictionarySize", - "groupBy/spilledBytes", - "groupBy/spilledQueries", - "ingest/count", - "ingest/events/avgMessageGap", - "ingest/events/duplicate", - "ingest/events/maxMessageGap", - "ingest/events/messageGap", - "ingest/events/minMessageGap", - "ingest/events/processed", - "ingest/events/processedWithError", - "ingest/events/thrownAway", - "ingest/events/unparseable", - "ingest/handoff/count", - "ingest/handoff/failed", - "ingest/handoff/time", - "ingest/input/bytes", - "ingest/kafka/avgLag", - "ingest/kafka/avgLag/time", - "ingest/kafka/fetchOffsets/time", - "ingest/kafka/lag", - "ingest/kafka/lag/time", - "ingest/kafka/maxLag", - "ingest/kafka/maxLag/time", - "ingest/kafka/partitionLag", - "ingest/kinesis/avgLag/time", - "ingest/kinesis/fetchOffsets/time", - "ingest/kinesis/lag/time", - "ingest/kinesis/maxLag/time", - "ingest/kinesis/partitionLag/time", - "ingest/merge/cpu", - "ingest/merge/time", - "ingest/notices/queueSize", - "ingest/notices/time", - "ingest/pause/time", - "ingest/persists/backPressure", - "ingest/persists/count", - "ingest/persists/cpu", - "ingest/persists/failed", - "ingest/persists/time", - "ingest/rows/output", - "ingest/segments/count", - "ingest/shuffle/bytes", - "ingest/shuffle/requests", - "ingest/sink/count", - "ingest/tombstones/count", - "interval/compacted/count", - "interval/skipCompact/count", - "interval/waitCompact/count", - "jetty/numOpenConnections", - "jetty/threadPool/busy", - "jetty/threadPool/idle", - "jetty/threadPool/isLowOnThreads", - "jetty/threadPool/max", - "jetty/threadPool/min", - "jetty/threadPool/queueSize", - "jetty/threadPool/ready", - "jetty/threadPool/total", - "jetty/threadPool/utilizationRate", - "jetty/threadPool/utilized", - "jvm/bufferpool/capacity", - "jvm/bufferpool/count", - "jvm/bufferpool/used", - "jvm/gc/count", - "jvm/gc/cpu", - "jvm/mem/committed", - "jvm/mem/init", - "jvm/mem/max", - "jvm/mem/used", - "jvm/pool/committed", - "jvm/pool/init", - "jvm/pool/max", - "jvm/pool/used", - "kill/eligibleUnusedSegments/count", - "kill/pendingSegments/count", - "kill/task/count", - "killTask/availableSlot/count", - "killTask/maxSlot/count", - "mergeBuffer/acquisitionTimeNs", - "mergeBuffer/maxAcquisitionTimeNs", - "mergeBuffer/pendingRequests", - "mergeBuffer/queries", - "mergeBuffer/used", - "metadata/kill/audit/count", - "metadata/kill/compaction/count", - "metadata/kill/datasource/count", - "metadata/kill/rule/count", - "metadata/kill/supervisor/count", - "metadatacache/init/time", - "metadatacache/schemaPoll/time", - "query/byteLimit/exceeded/count", - "query/bytes", - "query/cache/memcached/delta", - "query/cache/memcached/total", - "query/count", - "query/cpu/time", - "query/failed/count", - "query/interrupted/count", - "query/node/bytes", - "query/node/time", - "query/node/ttfb", - "query/priority", - "query/resultCache/hit", - "query/rowLimit/exceeded/count", - "query/segment/time", - "query/segmentAndCache/time", - "query/segments/count", - "query/success/count", - "query/time", - "query/timeout/count", - "query/wait/time", - "s3/upload/part/queueSize", - "s3/upload/part/queuedTime", - "s3/upload/part/time", - "s3/upload/total/bytes", - "s3/upload/total/time", - "segment/added/bytes", - "segment/assignSkipped/count", - "segment/assigned/count", - "segment/availableDeepStorageOnly/count", - "segment/compacted/bytes", - "segment/compacted/count", - "segment/count", - "segment/deleted/count", - "segment/dropQueue/count", - "segment/dropSkipped/count", - "segment/dropped/count", - "segment/kill/jobsProcessed/count", - "segment/kill/queueProcess/time", - "segment/kill/queueReset/time", - "segment/kill/skippedIntervals/count", - "segment/kill/unusedIntervals/count", - "segment/killed/deepStorage/count", - "segment/killed/metadataStore/count", - "segment/loadQueue/assigned", - "segment/loadQueue/cancelled", - "segment/loadQueue/count", - "segment/loadQueue/failed", - "segment/loadQueue/size", - "segment/loadQueue/success", - "segment/loading/rateKbps", - "segment/max", - "segment/metadataCache/dataSource/deleted", - "segment/metadataCache/deleted", - "segment/metadataCache/interval/count", - "segment/metadataCache/pending/count", - "segment/metadataCache/pending/deleted", - "segment/metadataCache/pending/skipped", - "segment/metadataCache/pending/updated", - "segment/metadataCache/skipped", - "segment/metadataCache/sync/time", - "segment/metadataCache/transactions/readOnly", - "segment/metadataCache/transactions/readWrite", - "segment/metadataCache/transactions/writeOnly", - "segment/metadataCache/used/count", - "segment/metadataCache/used/stale", - "segment/metadataCache/used/updated", - "segment/moveSkipped/count", - "segment/moved/bytes", - "segment/moved/count", - "segment/nuked/bytes", - "segment/overShadowed/count", - "segment/pending/count", - "segment/pendingDelete", - "segment/rowCount/avg", - "segment/rowCount/range/count", - "segment/scan/active", - "segment/scan/pending", - "segment/schemaCache/backfill/count", - "segment/schemaCache/dataSource/removed", - "segment/schemaCache/deepStorageOnly/count", - "segment/schemaCache/deepStorageOnly/refresh/time", - "segment/schemaCache/pendingBackfill/count", - "segment/schemaCache/poll/count", - "segment/schemaCache/poll/failed", - "segment/schemaCache/realtime/count", - "segment/schemaCache/refresh/count", - "segment/schemaCache/refresh/time", - "segment/schemaCache/refreshSkipped/count", - "segment/schemaCache/rowSignature/changed", - "segment/schemaCache/rowSignature/column/count", - "segment/schemaCache/used/count", - "segment/schemaCache/usedFingerprint/count", - "segment/size", - "segment/skipCompact/bytes", - "segment/skipCompact/count", - "segment/unavailable/count", - "segment/underReplicated/count", - "segment/unneeded/count", - "segment/unneededEternityTombstone/count", - "segment/used", - "segment/used/count", - "segment/used/deepStorageOnly/count", - "segment/usedPercent", - "segment/waitCompact/bytes", - "segment/waitCompact/count", - "serverview/init/time", - "serverview/sync/healthy", - "serverview/sync/unstableTime", - "sqlQuery/bytes", - "sqlQuery/planningTimeMs", - "sqlQuery/time", - "subquery/byteLimit/count", - "subquery/bytes", - "subquery/fallback/count", - "subquery/fallback/insufficientType/count", - "subquery/fallback/unknownReason/count", - "subquery/rowLimit/count", - "subquery/rows", - "sys/cpu", - "sys/disk/queue", - "sys/disk/read/count", - "sys/disk/read/size", - "sys/disk/transferTime", - "sys/disk/write/count", - "sys/disk/write/size", - "sys/fs/files/count", - "sys/fs/files/free", - "sys/fs/max", - "sys/fs/used", - "sys/mem/free", - "sys/mem/max", - "sys/mem/used", - "sys/net/read/dropped", - "sys/net/read/errors", - "sys/net/read/packets", - "sys/net/read/size", - "sys/net/write/collisions", - "sys/net/write/errors", - "sys/net/write/packets", - "sys/net/write/size", - "sys/storage/used", - "sys/swap/free", - "sys/swap/max", - "sys/swap/pageIn", - "sys/swap/pageOut", - "sys/tcpv4/activeOpens", - "sys/tcpv4/attemptFails", - "sys/tcpv4/estabResets", - "sys/tcpv4/in/errs", - "sys/tcpv4/in/segs", - "sys/tcpv4/out/rsts", - "sys/tcpv4/out/segs", - "sys/tcpv4/passiveOpens", - "sys/tcpv4/retrans/segs", - "sys/uptime", - "task/action/batch/attempts", - "task/action/batch/queueTime", - "task/action/batch/runTime", - "task/action/batch/size", - "task/action/failed/count", - "task/action/run/time", - "task/action/success/count", - "task/autoScaler/requiredCount", - "task/autoScaler/scaleActionTime", - "task/failed/count", - "task/pending/count", - "task/pending/time", - "task/run/time", - "task/running/count", - "task/segmentAvailability/wait/time", - "task/success/count", - "task/waiting/count", - "task/waiting/time", - "taskSlot/blacklisted/count", - "taskSlot/idle/count", - "taskSlot/lazy/count", - "taskSlot/total/count", - "taskSlot/used/count", - "tier/historical/count", - "tier/replication/factor", - "tier/required/capacity", - "tier/total/capacity", - "worker/task/assigned/count", - "worker/task/completed/count", - "worker/task/failed/count", - "worker/task/running/count", - "worker/task/success/count", - "worker/taskSlot/idle/count", - "worker/taskSlot/total/count", - "worker/taskSlot/used/count", - "zk/connected", - "zk/reconnect/time" -] diff --git a/processing/src/main/resources/defaultLoggingMetrics.json b/processing/src/main/resources/defaultLoggingMetrics.json new file mode 100644 index 000000000000..433f49e002b1 --- /dev/null +++ b/processing/src/main/resources/defaultLoggingMetrics.json @@ -0,0 +1,308 @@ +{ + "cgroup/cpu/cores_quota": [], + "cgroup/cpu/shares": [], + "cgroup/cpu/usage/sys/percentage": [], + "cgroup/cpu/usage/total/percentage": [], + "cgroup/cpu/usage/user/percentage": [], + "cgroup/cpuset/cpu_count": [], + "cgroup/cpuset/effective_cpu_count": [], + "cgroup/cpuset/effective_mems_count": [], + "cgroup/cpuset/mems_count": [], + "cgroup/disk/read/count": [], + "cgroup/disk/read/size": [], + "cgroup/disk/write/count": [], + "cgroup/disk/write/size": [], + "cgroup/memory/limit/bytes": [], + "cgroup/memory/usage/bytes": [], + "compact/segmentAnalyzer/fetchAndProcessMillis": [], + "compact/task/count": [], + "compactTask/availableSlot/count": [], + "compactTask/maxSlot/count": [], + "coordinator/global/time": [], + "coordinator/time": [], + "groupBy/maxMergeDictionarySize": [], + "groupBy/maxSpilledBytes": [], + "groupBy/mergeDictionarySize": [], + "groupBy/spilledBytes": [], + "groupBy/spilledQueries": [], + "ingest/count": [], + "ingest/events/avgMessageGap": [], + "ingest/events/duplicate": [], + "ingest/events/maxMessageGap": [], + "ingest/events/messageGap": [], + "ingest/events/minMessageGap": [], + "ingest/events/processed": [], + "ingest/events/processedWithError": [], + "ingest/events/thrownAway": [], + "ingest/events/unparseable": [], + "ingest/handoff/count": [], + "ingest/handoff/failed": [], + "ingest/handoff/time": [], + "ingest/input/bytes": [], + "ingest/kafka/avgLag": [], + "ingest/kafka/avgLag/time": [], + "ingest/kafka/fetchOffsets/time": [], + "ingest/kafka/lag": [], + "ingest/kafka/lag/time": [], + "ingest/kafka/maxLag": [], + "ingest/kafka/maxLag/time": [], + "ingest/kafka/partitionLag": [], + "ingest/kinesis/avgLag/time": [], + "ingest/kinesis/fetchOffsets/time": [], + "ingest/kinesis/lag/time": [], + "ingest/kinesis/maxLag/time": [], + "ingest/kinesis/partitionLag/time": [], + "ingest/merge/cpu": [], + "ingest/merge/time": [], + "ingest/notices/queueSize": [], + "ingest/notices/time": [], + "ingest/pause/time": [], + "ingest/persists/backPressure": [], + "ingest/persists/count": [], + "ingest/persists/cpu": [], + "ingest/persists/failed": [], + "ingest/persists/time": [], + "ingest/rows/output": [], + "ingest/segments/count": [], + "ingest/shuffle/bytes": [], + "ingest/shuffle/requests": [], + "ingest/sink/count": [], + "ingest/tombstones/count": [], + "interval/compacted/count": [], + "interval/skipCompact/count": [], + "interval/waitCompact/count": [], + "jetty/numOpenConnections": [], + "jetty/threadPool/busy": [], + "jetty/threadPool/idle": [], + "jetty/threadPool/isLowOnThreads": [], + "jetty/threadPool/max": [], + "jetty/threadPool/min": [], + "jetty/threadPool/queueSize": [], + "jetty/threadPool/ready": [], + "jetty/threadPool/total": [], + "jetty/threadPool/utilizationRate": [], + "jetty/threadPool/utilized": [], + "jvm/bufferpool/capacity": [], + "jvm/bufferpool/count": [], + "jvm/bufferpool/used": [], + "jvm/gc/count": [], + "jvm/gc/cpu": [], + "jvm/mem/committed": [], + "jvm/mem/init": [], + "jvm/mem/max": [], + "jvm/mem/used": [], + "jvm/pool/committed": [], + "jvm/pool/init": [], + "jvm/pool/max": [], + "jvm/pool/used": [], + "kill/eligibleUnusedSegments/count": [], + "kill/pendingSegments/count": [], + "kill/task/count": [], + "killTask/availableSlot/count": [], + "killTask/maxSlot/count": [], + "mergeBuffer/acquisitionTimeNs": [], + "mergeBuffer/maxAcquisitionTimeNs": [], + "mergeBuffer/pendingRequests": [], + "mergeBuffer/queries": [], + "mergeBuffer/used": [], + "metadata/kill/audit/count": [], + "metadata/kill/compaction/count": [], + "metadata/kill/datasource/count": [], + "metadata/kill/rule/count": [], + "metadata/kill/supervisor/count": [], + "metadatacache/init/time": [], + "metadatacache/schemaPoll/time": [], + "query/byteLimit/exceeded/count": [], + "query/bytes": [], + "query/cache/memcached/delta": [], + "query/cache/memcached/total": [], + "query/count": [], + "query/cpu/time": [], + "query/failed/count": [], + "query/interrupted/count": [], + "query/node/bytes": [], + "query/node/time": [], + "query/node/ttfb": [], + "query/priority": [], + "query/resultCache/hit": [], + "query/rowLimit/exceeded/count": [], + "query/segment/time": [], + "query/segmentAndCache/time": [], + "query/segments/count": [], + "query/success/count": [], + "query/time": [], + "query/timeout/count": [], + "query/wait/time": [], + "s3/upload/part/queueSize": [], + "s3/upload/part/queuedTime": [], + "s3/upload/part/time": [], + "s3/upload/total/bytes": [], + "s3/upload/total/time": [], + "segment/added/bytes": [], + "segment/assignSkipped/count": [], + "segment/assigned/count": [], + "segment/availableDeepStorageOnly/count": [], + "segment/compacted/bytes": [], + "segment/compacted/count": [], + "segment/count": [], + "segment/deleted/count": [], + "segment/dropQueue/count": [], + "segment/dropSkipped/count": [], + "segment/dropped/count": [], + "segment/kill/jobsProcessed/count": [], + "segment/kill/queueProcess/time": [], + "segment/kill/queueReset/time": [], + "segment/kill/skippedIntervals/count": [], + "segment/kill/unusedIntervals/count": [], + "segment/killed/deepStorage/count": [], + "segment/killed/metadataStore/count": [], + "segment/loadQueue/assigned": [], + "segment/loadQueue/cancelled": [], + "segment/loadQueue/count": [], + "segment/loadQueue/failed": [], + "segment/loadQueue/size": [], + "segment/loadQueue/success": [], + "segment/loading/rateKbps": [], + "segment/max": [], + "segment/metadataCache/dataSource/deleted": [], + "segment/metadataCache/deleted": [], + "segment/metadataCache/interval/count": [], + "segment/metadataCache/pending/count": [], + "segment/metadataCache/pending/deleted": [], + "segment/metadataCache/pending/skipped": [], + "segment/metadataCache/pending/updated": [], + "segment/metadataCache/skipped": [], + "segment/metadataCache/sync/time": [], + "segment/metadataCache/transactions/readOnly": [], + "segment/metadataCache/transactions/readWrite": [], + "segment/metadataCache/transactions/writeOnly": [], + "segment/metadataCache/used/count": [], + "segment/metadataCache/used/stale": [], + "segment/metadataCache/used/updated": [], + "segment/moveSkipped/count": [], + "segment/moved/bytes": [], + "segment/moved/count": [], + "segment/nuked/bytes": [], + "segment/overShadowed/count": [], + "segment/pending/count": [], + "segment/pendingDelete": [], + "segment/rowCount/avg": [], + "segment/rowCount/range/count": [], + "segment/scan/active": [], + "segment/scan/pending": [], + "segment/schemaCache/backfill/count": [], + "segment/schemaCache/dataSource/removed": [], + "segment/schemaCache/deepStorageOnly/count": [], + "segment/schemaCache/deepStorageOnly/refresh/time": [], + "segment/schemaCache/pendingBackfill/count": [], + "segment/schemaCache/poll/count": [], + "segment/schemaCache/poll/failed": [], + "segment/schemaCache/realtime/count": [], + "segment/schemaCache/refresh/count": [], + "segment/schemaCache/refresh/time": [], + "segment/schemaCache/refreshSkipped/count": [], + "segment/schemaCache/rowSignature/changed": [], + "segment/schemaCache/rowSignature/column/count": [], + "segment/schemaCache/used/count": [], + "segment/schemaCache/usedFingerprint/count": [], + "segment/size": [], + "segment/skipCompact/bytes": [], + "segment/skipCompact/count": [], + "segment/unavailable/count": [], + "segment/underReplicated/count": [], + "segment/unneeded/count": [], + "segment/unneededEternityTombstone/count": [], + "segment/used": [], + "segment/used/count": [], + "segment/used/deepStorageOnly/count": [], + "segment/usedPercent": [], + "segment/waitCompact/bytes": [], + "segment/waitCompact/count": [], + "serverview/init/time": [], + "serverview/sync/healthy": [], + "serverview/sync/unstableTime": [], + "sqlQuery/bytes": [], + "sqlQuery/planningTimeMs": [], + "sqlQuery/time": [], + "subquery/byteLimit/count": [], + "subquery/bytes": [], + "subquery/fallback/count": [], + "subquery/fallback/insufficientType/count": [], + "subquery/fallback/unknownReason/count": [], + "subquery/rowLimit/count": [], + "subquery/rows": [], + "sys/cpu": [], + "sys/disk/queue": [], + "sys/disk/read/count": [], + "sys/disk/read/size": [], + "sys/disk/transferTime": [], + "sys/disk/write/count": [], + "sys/disk/write/size": [], + "sys/fs/files/count": [], + "sys/fs/files/free": [], + "sys/fs/max": [], + "sys/fs/used": [], + "sys/mem/free": [], + "sys/mem/max": [], + "sys/mem/used": [], + "sys/net/read/dropped": [], + "sys/net/read/errors": [], + "sys/net/read/packets": [], + "sys/net/read/size": [], + "sys/net/write/collisions": [], + "sys/net/write/errors": [], + "sys/net/write/packets": [], + "sys/net/write/size": [], + "sys/storage/used": [], + "sys/swap/free": [], + "sys/swap/max": [], + "sys/swap/pageIn": [], + "sys/swap/pageOut": [], + "sys/tcpv4/activeOpens": [], + "sys/tcpv4/attemptFails": [], + "sys/tcpv4/estabResets": [], + "sys/tcpv4/in/errs": [], + "sys/tcpv4/in/segs": [], + "sys/tcpv4/out/rsts": [], + "sys/tcpv4/out/segs": [], + "sys/tcpv4/passiveOpens": [], + "sys/tcpv4/retrans/segs": [], + "sys/uptime": [], + "task/action/batch/attempts": [], + "task/action/batch/queueTime": [], + "task/action/batch/runTime": [], + "task/action/batch/size": [], + "task/action/failed/count": [], + "task/action/run/time": [], + "task/action/success/count": [], + "task/autoScaler/requiredCount": [], + "task/autoScaler/scaleActionTime": [], + "task/failed/count": [], + "task/pending/count": [], + "task/pending/time": [], + "task/run/time": [], + "task/running/count": [], + "task/segmentAvailability/wait/time": [], + "task/success/count": [], + "task/waiting/count": [], + "task/waiting/time": [], + "taskSlot/blacklisted/count": [], + "taskSlot/idle/count": [], + "taskSlot/lazy/count": [], + "taskSlot/total/count": [], + "taskSlot/used/count": [], + "tier/historical/count": [], + "tier/replication/factor": [], + "tier/required/capacity": [], + "tier/total/capacity": [], + "worker/task/assigned/count": [], + "worker/task/completed/count": [], + "worker/task/failed/count": [], + "worker/task/running/count": [], + "worker/task/success/count": [], + "worker/taskSlot/idle/count": [], + "worker/taskSlot/total/count": [], + "worker/taskSlot/used/count": [], + "zk/connected": [], + "zk/reconnect/time": [] +} diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java new file mode 100644 index 000000000000..49e78607e788 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java @@ -0,0 +1,67 @@ +/* + * 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.java.util.emitter.core; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +public class LoggingEmitterAllowlistConfigTest +{ + @Test + public void testFallsBackToDefaultAllowlistWhenCustomPathIsMissing() + { + final Map configMap = new HashMap<>(); + configMap.put("filterMetrics", true); + configMap.put("metricAllowlistPath", "/tmp/missing-" + UUID.randomUUID() + ".json"); + final LoggingEmitterConfig config = new ObjectMapper().convertValue(configMap, LoggingEmitterConfig.class); + + final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); + Assert.assertNotNull(emitter); + Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/cpu")); + Assert.assertTrue(emitter.getMetricAllowlist().contains("query/time")); + } + + @Test + public void testReadsCustomAllowlistAsMetricObject() throws IOException + { + final Path allowlist = Files.createTempFile("allowlist-object", ".json"); + Files.writeString(allowlist, "{\"jvm/gc/cpu\": [], \"jvm/gc/count\": []}"); + + final Map configMap = new HashMap<>(); + configMap.put("filterMetrics", true); + configMap.put("metricAllowlistPath", allowlist.toAbsolutePath().toString()); + final LoggingEmitterConfig config = new ObjectMapper().convertValue(configMap, LoggingEmitterConfig.class); + + final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); + Assert.assertNotNull(emitter); + Assert.assertEquals(2, emitter.getMetricAllowlist().size()); + Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/cpu")); + Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/count")); + Assert.assertFalse(emitter.getMetricAllowlist().contains("query/time")); + } +} diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java index b92de35624ed..0b28232a61e5 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java @@ -19,50 +19,146 @@ package org.apache.druid.java.util.emitter.core; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.java.util.emitter.service.UnitEvent; import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import java.io.File; import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; -import java.nio.file.Path; -import java.util.HashMap; -import java.util.Map; -import java.util.UUID; +import java.util.ArrayList; +import java.util.List; public class LoggingEmitterTest { + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + + private List serializedObjects; + private ObjectMapper trackingMapper; + + @Before + public void setUp() + { + serializedObjects = new ArrayList<>(); + trackingMapper = new ObjectMapper() + { + @Override + public String writeValueAsString(Object value) throws JsonProcessingException + { + serializedObjects.add(value); + return super.writeValueAsString(value); + } + }; + } + + private LoggingEmitter createEmitter(boolean shouldFilterMetrics, String allowedMetricsPath) + { + final LoggingEmitter emitter = new LoggingEmitter( + new Logger(LoggingEmitter.class), + LoggingEmitter.Level.WARN, + trackingMapper, + shouldFilterMetrics, + allowedMetricsPath + ); + emitter.start(); + return emitter; + } + @Test - public void testFallsBackToDefaultAllowlistWhenCustomPathIsMissing() + public void testEmitAllWhenFilteringDisabled() { - final Map configMap = new HashMap<>(); - configMap.put("filterMetrics", true); - configMap.put("metricAllowlistPath", "/tmp/missing-" + UUID.randomUUID() + ".json"); - final LoggingEmitterConfig config = new ObjectMapper().convertValue(configMap, LoggingEmitterConfig.class); - - final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); - Assert.assertNotNull(emitter); - Assert.assertEquals(306, emitter.getMetricAllowlist().size()); - Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/cpu")); - Assert.assertTrue(emitter.getMetricAllowlist().contains("query/time")); + try (LoggingEmitter emitter = createEmitter(false, null)) { + emitter.emit(ServiceMetricEvent.builder().setMetric("query/bytes", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/gc/count", 512).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("some/random/metric", 1).build("test", "localhost")); + + Assert.assertEquals(3, serializedObjects.size()); + } + } + + @Test + public void testFilterWithDefaultResource() + { + try (LoggingEmitter emitter = createEmitter(true, null)) { + emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/gc/count", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("some/unlisted/metric", 1).build("test", "localhost")); + + Assert.assertEquals(1, serializedObjects.size()); + } + } + + @Test + public void testFilterWithCustomFilePath() throws IOException + { + final File allowFile = createAllowlistFile("{\"query/bytes\": [], \"segment/scan/active\": []}"); + try (LoggingEmitter emitter = createEmitter(true, allowFile.getAbsolutePath())) { + emitter.emit(ServiceMetricEvent.builder().setMetric("query/bytes", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/gc/count", 512).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("segment/scan/active", 2048).build("test", "localhost")); + + Assert.assertEquals(2, serializedObjects.size()); + } } @Test - public void testReadsCustomAllowlistAsMetricNameArray() throws IOException + public void testNonMetricEventsAlwaysPassThrough() throws IOException + { + final File allowFile = createAllowlistFile("{\"query/bytes\": []}"); + try (LoggingEmitter emitter = createEmitter(true, allowFile.getAbsolutePath())) { + emitter.emit(new UnitEvent("alerts", 42)); + + Assert.assertEquals(1, serializedObjects.size()); + } + } + + @Test + public void testMissingCustomPathThrowsAnError() + { + Assert.assertThrows(IAE.class, () -> createEmitter(true, "/nonexistent/path/to/allowedMetrics.json")); + } + + @Test + public void testEmptyAllowlistBlocksAllMetrics() throws IOException + { + final File allowFile = createAllowlistFile("{}"); + try (LoggingEmitter emitter = createEmitter(true, allowFile.getAbsolutePath())) { + emitter.emit(ServiceMetricEvent.builder().setMetric("query/bytes", 100).build("test", "localhost")); + emitter.emit(new UnitEvent("alerts", 42)); + + Assert.assertEquals(1, serializedObjects.size()); + } + } + + @Test + public void testFilterDisabledIgnoresPath() throws IOException + { + final File allowFile = createAllowlistFile("{\"query/bytes\": []}"); + try (LoggingEmitter emitter = createEmitter(false, allowFile.getAbsolutePath())) { + emitter.emit(ServiceMetricEvent.builder().setMetric("query/bytes", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/gc/count", 512).build("test", "localhost")); + + Assert.assertEquals(2, serializedObjects.size()); + } + } + + private File createAllowlistFile(String jsonContent) throws IOException { - final Path allowlist = Files.createTempFile("allowlist-array", ".json"); - Files.writeString(allowlist, "[\"jvm/gc/cpu\", \"jvm/gc/count\"]"); - - final Map configMap = new HashMap<>(); - configMap.put("filterMetrics", true); - configMap.put("metricAllowlistPath", allowlist.toAbsolutePath().toString()); - final LoggingEmitterConfig config = new ObjectMapper().convertValue(configMap, LoggingEmitterConfig.class); - - final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); - Assert.assertNotNull(emitter); - Assert.assertEquals(2, emitter.getMetricAllowlist().size()); - Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/cpu")); - Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/count")); - Assert.assertFalse(emitter.getMetricAllowlist().contains("query/time")); + final File file = tempFolder.newFile("allowedMetrics.json"); + try (Writer writer = new OutputStreamWriter(Files.newOutputStream(file.toPath()), StandardCharsets.UTF_8)) { + writer.write(jsonContent); + } + return file; } } From db8a01fdf820b94d18b89933343faac1329dd92b Mon Sep 17 00:00:00 2001 From: Misha Date: Thu, 19 Feb 2026 09:22:45 +0100 Subject: [PATCH 04/12] fix test --- .../util/emitter/core/LoggingEmitterAllowlistConfigTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java index 49e78607e788..45d89a9e6793 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java @@ -37,7 +37,6 @@ public void testFallsBackToDefaultAllowlistWhenCustomPathIsMissing() { final Map configMap = new HashMap<>(); configMap.put("filterMetrics", true); - configMap.put("metricAllowlistPath", "/tmp/missing-" + UUID.randomUUID() + ".json"); final LoggingEmitterConfig config = new ObjectMapper().convertValue(configMap, LoggingEmitterConfig.class); final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); From 1e000d03a686d0b8d0142df7938fa17dca46e98b Mon Sep 17 00:00:00 2001 From: Misha Date: Thu, 19 Feb 2026 09:47:13 +0100 Subject: [PATCH 05/12] imports cleanup --- .../util/emitter/core/LoggingEmitterAllowlistConfigTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java index 45d89a9e6793..dcf37d371981 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java @@ -28,7 +28,6 @@ import java.nio.file.Path; import java.util.HashMap; import java.util.Map; -import java.util.UUID; public class LoggingEmitterAllowlistConfigTest { From e4ac542a6e19013403f3f22b3c367cb3e4fcbde1 Mon Sep 17 00:00:00 2001 From: Misha Date: Fri, 6 Mar 2026 18:20:03 +0100 Subject: [PATCH 06/12] WIP unified emitter metric allowlist filtering --- .../emitter/prometheus/PrometheusEmitter.java | 41 +++++++++- .../prometheus/PrometheusEmitterConfig.java | 3 +- .../emitter/core/BaseHttpEmittingConfig.java | 6 +- .../emitter/core/GlobalEmitterConfig.java | 43 +++++++++++ .../util/emitter/core/HttpEmitterConfig.java | 2 + .../util/emitter/core/HttpPostEmitter.java | 29 ++++++- .../util/emitter/core/LoggingEmitter.java | 76 +++++-------------- .../emitter/core/LoggingEmitterConfig.java | 18 +---- .../emitter/core/MetricAllowlistLoader.java | 68 +++++++++++++++++ .../emitter/core/MetricAllowlistParser.java | 29 +++++++ .../emitter/core/MetricAllowlistParsers.java | 40 ++++++++++ .../emitter/core/MetricFilteringEmitter.java | 27 +++++++ .../emitter/core/ParametrizedUriEmitter.java | 29 ++++++- .../core/ParametrizedUriEmitterConfig.java | 4 +- .../LoggingEmitterAllowlistConfigTest.java | 2 +- .../core/LoggingEmitterConfigTest.java | 8 +- 16 files changed, 338 insertions(+), 87 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java create mode 100644 processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistLoader.java create mode 100644 processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParser.java create mode 100644 processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsers.java create mode 100644 processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricFilteringEmitter.java diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 60af27fcd4d1..0799ad4f3a5e 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -19,8 +19,9 @@ package org.apache.druid.emitter.prometheus; - +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Counter; @@ -32,14 +33,20 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.core.Emitter; import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.core.MetricAllowlistLoader; +import org.apache.druid.java.util.emitter.core.MetricAllowlistParser; +import org.apache.druid.java.util.emitter.core.MetricAllowlistParsers; +import org.apache.druid.java.util.emitter.core.MetricFilteringEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; @@ -47,12 +54,14 @@ /** * */ -public class PrometheusEmitter implements Emitter +public class PrometheusEmitter implements Emitter, MetricFilteringEmitter { private static final Logger log = new Logger(PrometheusEmitter.class); private final Metrics metrics; private final PrometheusEmitterConfig config; private final PrometheusEmitterConfig.Strategy strategy; + private final boolean filterMetrics; + private final Set metricAllowlist; private static final Pattern PATTERN = Pattern.compile("[^a-zA-Z0-9_][^a-zA-Z0-9_]*"); private static final String TAG_HOSTNAME = "host_name"; @@ -72,6 +81,19 @@ public PrometheusEmitter(PrometheusEmitterConfig config) { this.config = config; this.strategy = config.getStrategy(); + this.filterMetrics = config.isFilterMetrics(); + this.metricAllowlist = this.filterMetrics + ? MetricAllowlistLoader.loadAllowlist( + new ObjectMapper(), + config.getMetricAllowlistPath() + .orElseGet( + () -> Strings.isNullOrEmpty(config.getDimensionMapPath()) + ? "defaultMetrics.json" + : config.getDimensionMapPath() + ), + getMetricAllowlistParser() + ) + : Collections.emptySet(); metrics = new Metrics(config); } @@ -145,6 +167,9 @@ public void emit(Event event) private void emitMetric(ServiceMetricEvent metricEvent) { String name = metricEvent.getMetric(); + if (shouldFilterOutMetric(name)) { + return; + } String service = metricEvent.getService(); String host = metricEvent.getHost(); Map userDims = metricEvent.getUserDims(); @@ -286,6 +311,18 @@ public void setPushGateway(PushGateway pushGateway) this.pushGateway = pushGateway; } + @Override + public boolean shouldFilterOutMetric(String metricName) + { + return filterMetrics && !metricAllowlist.contains(metricName); + } + + @Override + public MetricAllowlistParser getMetricAllowlistParser() + { + return MetricAllowlistParsers::parseMetricNameObject; + } + /** * Cleans up stale metrics that have not been updated within the configured TTL. * This method is called periodically by the TTL scheduler when using the 'exporter' strategy with diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index 56def5c4c2d8..a2ef2ab76347 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.emitter.core.GlobalEmitterConfig; import org.joda.time.Duration; import javax.annotation.Nullable; @@ -35,7 +36,7 @@ /** * */ -public class PrometheusEmitterConfig +public class PrometheusEmitterConfig extends GlobalEmitterConfig { static final Pattern PATTERN = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java index 44ecaf9d334b..c682a10d7c66 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java @@ -27,7 +27,7 @@ import javax.validation.constraints.Min; import java.util.concurrent.TimeUnit; -public class BaseHttpEmittingConfig +public class BaseHttpEmittingConfig extends GlobalEmitterConfig { public static final long DEFAULT_FLUSH_MILLIS = 60 * 1000; public static final int DEFAULT_FLUSH_COUNTS = 500; @@ -181,6 +181,8 @@ protected String toStringBase() ", contentEncoding=" + contentEncoding + ", batchQueueSizeLimit=" + batchQueueSizeLimit + ", httpTimeoutAllowanceFactor=" + httpTimeoutAllowanceFactor + - ", minHttpTimeoutMillis=" + minHttpTimeoutMillis; + ", minHttpTimeoutMillis=" + minHttpTimeoutMillis + + ", filterMetrics=" + filterMetrics + + ", metricAllowlistPath='" + metricAllowlistPath + '\''; } } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java new file mode 100644 index 000000000000..93b3e8517821 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java @@ -0,0 +1,43 @@ +/* + * 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.java.util.emitter.core; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; + +public class GlobalEmitterConfig +{ + @JsonProperty + boolean filterMetrics; + + @JsonProperty + String metricAllowlistPath; + + public boolean isFilterMetrics() + { + return filterMetrics; + } + + public Optional getMetricAllowlistPath() + { + return Optional.ofNullable(metricAllowlistPath); + } +} diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java index bf1817fd5c5c..4061d59d38e6 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java @@ -52,6 +52,8 @@ public HttpEmitterConfig(BaseHttpEmittingConfig base, String recipientBaseUrl) this.batchQueueSizeLimit = base.batchQueueSizeLimit; this.httpTimeoutAllowanceFactor = base.httpTimeoutAllowanceFactor; this.minHttpTimeoutMillis = base.minHttpTimeoutMillis; + this.filterMetrics = base.filterMetrics; + this.metricAllowlistPath = base.metricAllowlistPath; } public String getRecipientBaseUrl() diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java index 5006e025ed3c..61eae03cc2a5 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java @@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.ListenableFuture; import org.asynchttpclient.RequestBuilder; @@ -46,6 +47,7 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayDeque; +import java.util.Set; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; @@ -59,7 +61,7 @@ import java.util.concurrent.locks.LockSupport; import java.util.zip.GZIPOutputStream; -public class HttpPostEmitter implements Flushable, Closeable, Emitter +public class HttpPostEmitter implements Flushable, Closeable, Emitter, MetricFilteringEmitter { private static final int MAX_EVENT_SIZE = 1023 * 1024; // Set max size slightly less than 1M to allow for metadata @@ -100,6 +102,8 @@ public class HttpPostEmitter implements Flushable, Closeable, Emitter private final AsyncHttpClient client; private final ObjectMapper jsonMapper; private final String url; + private final boolean filterMetrics; + private final Set metricAllowlist; private final ConcurrentLinkedQueue buffersToReuse = new ConcurrentLinkedQueue<>(); /** @@ -169,6 +173,14 @@ public HttpPostEmitter(HttpEmitterConfig config, AsyncHttpClient client, ObjectM this.largeEventThreshold = (bufferSize - batchOverhead - batchingStrategy.separatorLength()) / 2; this.client = client; this.jsonMapper = jsonMapper; + this.filterMetrics = config.isFilterMetrics(); + this.metricAllowlist = this.filterMetrics + ? MetricAllowlistLoader.loadAllowlist( + this.jsonMapper, + config.getMetricAllowlistPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH), + getMetricAllowlistParser() + ) + : Set.of(); try { this.url = new URL(config.getRecipientBaseUrl()).toString(); } @@ -223,6 +235,9 @@ private boolean isTerminated() @Override public void emit(Event event) { + if (event instanceof ServiceMetricEvent && shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric())) { + return; + } emitAndReturnBatch(event); } @@ -279,6 +294,18 @@ private byte[] eventToBytes(Event event) } } + @Override + public boolean shouldFilterOutMetric(String metricName) + { + return filterMetrics && !metricAllowlist.contains(metricName); + } + + @Override + public MetricAllowlistParser getMetricAllowlistParser() + { + return MetricAllowlistParsers::parseMetricNameObject; + } + private void writeLargeEvent(byte[] eventBytes) { // It's better to drop the oldest, not latest event, but dropping the oldest is not easy to implement, because diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java index b2c01876af95..8296c56c5de1 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java @@ -22,13 +22,10 @@ /** */ -import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import com.google.common.collect.ImmutableSet; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; @@ -36,19 +33,15 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.slf4j.MarkerFactory; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; import java.util.Set; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicBoolean; /** */ -public class LoggingEmitter implements Emitter +public class LoggingEmitter implements Emitter, MetricFilteringEmitter { - private static final String DEFAULT_METRIC_ALLOWLIST_PATH = "defaultLoggingMetrics.json"; + private static final String DEFAULT_METRIC_ALLOWLIST_PATH = MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH; private final Logger log; private final Level level; @@ -65,13 +58,13 @@ public LoggingEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper) Level.toLevel(config.getLogLevel()), jsonMapper, config.isFilterMetrics(), - config.isFilterMetrics() ? loadMetricAllowlist(jsonMapper, config.getMetricAllowlistPath()) : ImmutableSet.of() + config.getMetricAllowlistPath().orElse(DEFAULT_METRIC_ALLOWLIST_PATH) ); } public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper) { - this(log, level, jsonMapper, false, ImmutableSet.of()); + this(log, level, jsonMapper, false, ""); } public LoggingEmitter( @@ -87,7 +80,13 @@ public LoggingEmitter( level, jsonMapper, shouldFilterMetrics, - shouldFilterMetrics ? loadMetricAllowlist(jsonMapper, allowedMetricsPath) : ImmutableSet.of() + shouldFilterMetrics + ? MetricAllowlistLoader.loadAllowlist( + jsonMapper, + Strings.isNullOrEmpty(allowedMetricsPath) ? DEFAULT_METRIC_ALLOWLIST_PATH : allowedMetricsPath, + MetricAllowlistParsers::parseMetricNameObject + ) + : ImmutableSet.of() ); } @@ -97,7 +96,7 @@ public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper, boolean this.level = level; this.jsonMapper = jsonMapper; this.filterMetrics = filterMetrics; - this.metricAllowlist = metricAllowlist; + this.metricAllowlist = Set.copyOf(metricAllowlist); } @Override @@ -141,7 +140,7 @@ public void emit(Event event) throw new RejectedExecutionException("Service not started."); } } - if (shouldFilterOutMetric(event)) { + if (event instanceof ServiceMetricEvent && shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric())) { return; } try { @@ -223,53 +222,16 @@ public String toString() '}'; } - private boolean shouldFilterOutMetric(Event event) - { - if (!filterMetrics || !(event instanceof ServiceMetricEvent)) { - return false; - } - final ServiceMetricEvent metricEvent = (ServiceMetricEvent) event; - return !metricAllowlist.contains(metricEvent.getMetric()); - } - - private static Set loadMetricAllowlist(ObjectMapper mapper, String metricAllowlistPath) - { - final String source = Strings.isNullOrEmpty(metricAllowlistPath) ? DEFAULT_METRIC_ALLOWLIST_PATH : metricAllowlistPath; - try (final InputStream is = openAllowlistFile(metricAllowlistPath)) { - final JsonNode metricConfig = mapper.readTree(is); - if (!metricConfig.isObject()) { - throw new ISE("Metric allowlist file [%s] must be a JSON object of metric names", source); - } - final ImmutableSet.Builder metricNames = ImmutableSet.builder(); - metricConfig.fieldNames().forEachRemaining(metricNames::add); - return metricNames.build(); - } - catch (IOException e) { - throw new ISE(e, "Failed to parse metric allowlist file [%s]", source); - } - } - - private static InputStream openAllowlistFile(String metricAllowlistPath) + @Override + public boolean shouldFilterOutMetric(String metricName) { - if (Strings.isNullOrEmpty(metricAllowlistPath)) { - return openDefaultAllowlistFile(); - } - try { - return new FileInputStream(metricAllowlistPath); - } - catch (FileNotFoundException e) { - throw new IAE(e, "Metric allowlist file [%s] not found", metricAllowlistPath); - - } + return filterMetrics && !metricAllowlist.contains(metricName); } - private static InputStream openDefaultAllowlistFile() + @Override + public MetricAllowlistParser getMetricAllowlistParser() { - final InputStream is = LoggingEmitter.class.getClassLoader().getResourceAsStream(DEFAULT_METRIC_ALLOWLIST_PATH); - if (is == null) { - throw new ISE("Metric allowlist file [%s] not found on classpath", DEFAULT_METRIC_ALLOWLIST_PATH); - } - return is; + return MetricAllowlistParsers::parseMetricNameObject; } @VisibleForTesting diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java index 54fe5407262f..b6bbbc07c51d 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java @@ -25,7 +25,7 @@ /** */ -public class LoggingEmitterConfig +public class LoggingEmitterConfig extends GlobalEmitterConfig { @NotNull @JsonProperty @@ -35,12 +35,6 @@ public class LoggingEmitterConfig @JsonProperty private String logLevel = "info"; - @JsonProperty - private boolean filterMetrics; - - @JsonProperty - private String metricAllowlistPath; - public String getLoggerClass() { return loggerClass; @@ -51,16 +45,6 @@ public String getLogLevel() return logLevel; } - public boolean isFilterMetrics() - { - return filterMetrics; - } - - public String getMetricAllowlistPath() - { - return metricAllowlistPath; - } - @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistLoader.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistLoader.java new file mode 100644 index 000000000000..bcbd17358c8e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistLoader.java @@ -0,0 +1,68 @@ +/* + * 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.java.util.emitter.core; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; + +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.util.Set; + +public final class MetricAllowlistLoader +{ + + public static final String DEFAULT_METRIC_ALLOWLIST_PATH = "defaultLoggingMetrics.json"; + + public static Set loadAllowlist( + ObjectMapper mapper, + String allowlistPath, + MetricAllowlistParser parser + ) + { + try (final InputStream is = openAllowlistFile(allowlistPath)) { + return parser.parse(mapper.readTree(is), allowlistPath); + } + catch (IOException e) { + throw new ISE(e, "Failed to parse metric allowlist file [%s]", allowlistPath); + } + } + + private static InputStream openAllowlistFile(String allowlistPath) + { + if (Strings.isNullOrEmpty(allowlistPath)) { + throw new IAE("Metric allowlist file path is empty"); + } + try { + return new FileInputStream(allowlistPath); + } + catch (FileNotFoundException e) { + final InputStream classpathInputStream = MetricAllowlistLoader.class.getClassLoader().getResourceAsStream(allowlistPath); + if (classpathInputStream != null) { + return classpathInputStream; + } + throw new IAE(e, "Metric allowlist file [%s] not found", allowlistPath); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParser.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParser.java new file mode 100644 index 000000000000..59617f482894 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParser.java @@ -0,0 +1,29 @@ +/* + * 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.java.util.emitter.core; + +import com.fasterxml.jackson.databind.JsonNode; + +import java.util.Set; + +public interface MetricAllowlistParser +{ + Set parse(JsonNode metricConfig, String source); +} diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsers.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsers.java new file mode 100644 index 000000000000..de73e7b4a348 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsers.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.java.util.emitter.core; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.ISE; + +import java.util.Set; + +public final class MetricAllowlistParsers +{ + + public static Set parseMetricNameObject(JsonNode metricConfig, String source) + { + if (!metricConfig.isObject()) { + throw new ISE("Metric allowlist file [%s] must be a JSON object of metric names", source); + } + final ImmutableSet.Builder metricNames = ImmutableSet.builder(); + metricConfig.fieldNames().forEachRemaining(metricNames::add); + return metricNames.build(); + } +} diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricFilteringEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricFilteringEmitter.java new file mode 100644 index 000000000000..c2f7a9382b86 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricFilteringEmitter.java @@ -0,0 +1,27 @@ +/* + * 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.java.util.emitter.core; + +public interface MetricFilteringEmitter +{ + boolean shouldFilterOutMetric(String metricName); + + MetricAllowlistParser getMetricAllowlistParser(); +} diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java index 1aec8be0e4ca..16f2c6582328 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.asynchttpclient.AsyncHttpClient; import java.io.Closeable; @@ -37,7 +38,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.function.BiConsumer; -public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter +public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter, MetricFilteringEmitter { private static final Logger log = new Logger(ParametrizedUriEmitter.class); private static final Set ONLY_FEED_PARAM = ImmutableSet.of("feed"); @@ -68,6 +69,8 @@ private static UriExtractor makeUriExtractor(ParametrizedUriEmitterConfig config private final AsyncHttpClient client; private final ObjectMapper jsonMapper; private final ParametrizedUriEmitterConfig config; + private final boolean filterMetrics; + private final Set metricAllowlist; public ParametrizedUriEmitter( ParametrizedUriEmitterConfig config, @@ -89,6 +92,14 @@ public ParametrizedUriEmitter( this.client = client; this.jsonMapper = jsonMapper; this.uriExtractor = uriExtractor; + this.filterMetrics = config.isFilterMetrics(); + this.metricAllowlist = this.filterMetrics + ? MetricAllowlistLoader.loadAllowlist( + this.jsonMapper, + config.getMetricAllowlistPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH), + getMetricAllowlistParser() + ) + : Set.of(); } @Override @@ -117,6 +128,9 @@ public void start() @Override public void emit(Event event) { + if (event instanceof ServiceMetricEvent && shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric())) { + return; + } try { URI uri = uriExtractor.apply(event); // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed. @@ -206,4 +220,17 @@ public String toString() ", config=" + config + '}'; } + + @Override + public boolean shouldFilterOutMetric(String metricName) + { + return filterMetrics && !metricAllowlist.contains(metricName); + } + + @Override + public MetricAllowlistParser getMetricAllowlistParser() + { + return MetricAllowlistParsers::parseMetricNameObject; + } + } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java index 9982f134dbc6..de5da7d4b161 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java @@ -23,7 +23,7 @@ import javax.validation.constraints.NotNull; -public class ParametrizedUriEmitterConfig +public class ParametrizedUriEmitterConfig extends GlobalEmitterConfig { private static final BaseHttpEmittingConfig DEFAULT_HTTP_EMITTING_CONFIG = new BaseHttpEmittingConfig(); @@ -50,6 +50,8 @@ public String toString() return "ParametrizedUriEmitterConfig{" + "recipientBaseUrlPattern='" + recipientBaseUrlPattern + '\'' + ", httpEmittingConfig=" + httpEmittingConfig + + ", filterMetrics=" + filterMetrics + + ", metricAllowlistPath='" + metricAllowlistPath + '\'' + '}'; } } diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java index dcf37d371981..361fc0ab8e59 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java @@ -32,7 +32,7 @@ public class LoggingEmitterAllowlistConfigTest { @Test - public void testFallsBackToDefaultAllowlistWhenCustomPathIsMissing() + public void testUsesDefaultAllowlistWhenCustomPathIsNotProvided() { final Map configMap = new HashMap<>(); configMap.put("filterMetrics", true); diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java index 5886d6634d36..c856171e21a2 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java @@ -39,7 +39,7 @@ public void testDefaults() Assert.assertEquals("getLoggerClass", LoggingEmitter.class.getName(), config.getLoggerClass()); Assert.assertEquals("getLogLevel", "info", config.getLogLevel()); Assert.assertFalse("isFilterMetrics", config.isFilterMetrics()); - Assert.assertNull("getMetricAllowlistPath", config.getMetricAllowlistPath()); + Assert.assertFalse("getMetricAllowlistPath", config.getMetricAllowlistPath().isPresent()); } @Test @@ -55,7 +55,7 @@ public void testDefaultsLegacy() Assert.assertEquals("getLoggerClass", LoggingEmitter.class.getName(), config.getLoggerClass()); Assert.assertEquals("getLogLevel", "debug", config.getLogLevel()); Assert.assertFalse("isFilterMetrics", config.isFilterMetrics()); - Assert.assertNull("getMetricAllowlistPath", config.getMetricAllowlistPath()); + Assert.assertFalse("getMetricAllowlistPath", config.getMetricAllowlistPath().isPresent()); } @Test @@ -76,7 +76,7 @@ public void testSettingEverything() Assert.assertEquals("getLoggerClass", "Foo", config.getLoggerClass()); Assert.assertEquals("getLogLevel", "INFO", config.getLogLevel()); Assert.assertTrue("isFilterMetrics", config.isFilterMetrics()); - Assert.assertEquals("getMetricAllowlistPath", "/tmp/logging-metrics.json", config.getMetricAllowlistPath()); + Assert.assertEquals("getMetricAllowlistPath", "/tmp/logging-metrics.json", config.getMetricAllowlistPath().orElse(null)); } @Test @@ -97,6 +97,6 @@ public void testSettingEverythingLegacy() Assert.assertEquals("getLoggerClass", "Foo", config.getLoggerClass()); Assert.assertEquals("getLogLevel", "INFO", config.getLogLevel()); Assert.assertTrue("isFilterMetrics", config.isFilterMetrics()); - Assert.assertEquals("getMetricAllowlistPath", "/tmp/logging-metrics.json", config.getMetricAllowlistPath()); + Assert.assertEquals("getMetricAllowlistPath", "/tmp/logging-metrics.json", config.getMetricAllowlistPath().orElse(null)); } } From 7da8cd973eeb844d49f32356cc5ca5847d7beeab Mon Sep 17 00:00:00 2001 From: Misha Date: Tue, 10 Mar 2026 13:50:53 +0100 Subject: [PATCH 07/12] post-merge alignment --- .../emitter/prometheus/PrometheusEmitter.java | 4 +-- .../emitter/core/BaseHttpEmittingConfig.java | 4 +-- .../java/util/emitter/core/Emitters.java | 4 +-- .../emitter/core/GlobalEmitterConfig.java | 22 +++++++++++---- .../util/emitter/core/HttpEmitterConfig.java | 5 ++-- .../util/emitter/core/HttpPostEmitter.java | 4 +-- .../util/emitter/core/LoggingEmitter.java | 28 +++++++++---------- .../emitter/core/LoggingEmitterConfig.java | 4 +-- .../emitter/core/ParametrizedUriEmitter.java | 4 +-- .../core/ParametrizedUriEmitterConfig.java | 4 +-- .../LoggingEmitterAllowlistConfigTest.java | 18 ++++++------ .../core/LoggingEmitterConfigTest.java | 20 ++++++------- 12 files changed, 65 insertions(+), 56 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 0799ad4f3a5e..3f054b01ef6c 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -81,11 +81,11 @@ public PrometheusEmitter(PrometheusEmitterConfig config) { this.config = config; this.strategy = config.getStrategy(); - this.filterMetrics = config.isFilterMetrics(); + this.filterMetrics = config.isShouldFilterMetrics(); this.metricAllowlist = this.filterMetrics ? MetricAllowlistLoader.loadAllowlist( new ObjectMapper(), - config.getMetricAllowlistPath() + config.getMetricSpecPath() .orElseGet( () -> Strings.isNullOrEmpty(config.getDimensionMapPath()) ? "defaultMetrics.json" diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java index c682a10d7c66..d17c2f99b6c0 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java @@ -182,7 +182,7 @@ protected String toStringBase() ", batchQueueSizeLimit=" + batchQueueSizeLimit + ", httpTimeoutAllowanceFactor=" + httpTimeoutAllowanceFactor + ", minHttpTimeoutMillis=" + minHttpTimeoutMillis + - ", filterMetrics=" + filterMetrics + - ", metricAllowlistPath='" + metricAllowlistPath + '\''; + ", shouldFilterMetrics=" + isShouldFilterMetrics() + + ", metricSpecPath='" + getMetricSpecPath() + '\''; } } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java index 217b24831749..769615325d67 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java @@ -145,9 +145,9 @@ static Map makeLoggingMap(Properties props) "shouldFilterMetrics", Boolean.parseBoolean(props.getProperty("org.apache.druid.java.util.emitter.logging.shouldFilterMetrics")) ); } - if (props.containsKey("org.apache.druid.java.util.emitter.logging.allowedMetricsPath")) { + if (props.containsKey("org.apache.druid.java.util.emitter.logging.metricSpecPath")) { loggingMap.put( - "allowedMetricsPath", props.getProperty("org.apache.druid.java.util.emitter.logging.allowedMetricsPath") + "metricSpecPath", props.getProperty("org.apache.druid.java.util.emitter.logging.metricSpecPath") ); } return loggingMap; diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java index 93b3e8517821..ca74cc0b523e 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java @@ -26,18 +26,28 @@ public class GlobalEmitterConfig { @JsonProperty - boolean filterMetrics; + private boolean shouldFilterMetrics; @JsonProperty - String metricAllowlistPath; + private String metricSpecPath; - public boolean isFilterMetrics() + public boolean isShouldFilterMetrics() { - return filterMetrics; + return shouldFilterMetrics; } - public Optional getMetricAllowlistPath() + public Optional getMetricSpecPath() { - return Optional.ofNullable(metricAllowlistPath); + return Optional.ofNullable(metricSpecPath); + } + + public void setShouldFilterMetrics(boolean shouldFilterMetrics) + { + this.shouldFilterMetrics = shouldFilterMetrics; + } + + public void setMetricSpecPath(String metricSpecPath) + { + this.metricSpecPath = metricSpecPath; } } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java index 4061d59d38e6..26df2c0f6cb4 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java @@ -20,6 +20,7 @@ package org.apache.druid.java.util.emitter.core; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.commons.lang3.StringUtils; import org.apache.druid.metadata.PasswordProvider; import javax.validation.constraints.NotNull; @@ -52,8 +53,8 @@ public HttpEmitterConfig(BaseHttpEmittingConfig base, String recipientBaseUrl) this.batchQueueSizeLimit = base.batchQueueSizeLimit; this.httpTimeoutAllowanceFactor = base.httpTimeoutAllowanceFactor; this.minHttpTimeoutMillis = base.minHttpTimeoutMillis; - this.filterMetrics = base.filterMetrics; - this.metricAllowlistPath = base.metricAllowlistPath; + this.setShouldFilterMetrics(base.isShouldFilterMetrics()); + this.setMetricSpecPath(base.getMetricSpecPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH)); } public String getRecipientBaseUrl() diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java index 61eae03cc2a5..2748881ec9dc 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java @@ -173,11 +173,11 @@ public HttpPostEmitter(HttpEmitterConfig config, AsyncHttpClient client, ObjectM this.largeEventThreshold = (bufferSize - batchOverhead - batchingStrategy.separatorLength()) / 2; this.client = client; this.jsonMapper = jsonMapper; - this.filterMetrics = config.isFilterMetrics(); + this.filterMetrics = config.isShouldFilterMetrics(); this.metricAllowlist = this.filterMetrics ? MetricAllowlistLoader.loadAllowlist( this.jsonMapper, - config.getMetricAllowlistPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH), + config.getMetricSpecPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH), getMetricAllowlistParser() ) : Set.of(); diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java index 8296c56c5de1..ccec4500e75e 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java @@ -41,13 +41,11 @@ */ public class LoggingEmitter implements Emitter, MetricFilteringEmitter { - private static final String DEFAULT_METRIC_ALLOWLIST_PATH = MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH; - private final Logger log; private final Level level; private final ObjectMapper jsonMapper; - private final boolean filterMetrics; - private final Set metricAllowlist; + private final boolean shouldFilterMetrics; + private final Set metricNames; private final AtomicBoolean started = new AtomicBoolean(false); @@ -57,8 +55,8 @@ public LoggingEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper) new Logger(config.getLoggerClass()), Level.toLevel(config.getLogLevel()), jsonMapper, - config.isFilterMetrics(), - config.getMetricAllowlistPath().orElse(DEFAULT_METRIC_ALLOWLIST_PATH) + config.isShouldFilterMetrics(), + config.getMetricSpecPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH) ); } @@ -72,7 +70,7 @@ public LoggingEmitter( Level level, ObjectMapper jsonMapper, boolean shouldFilterMetrics, - String allowedMetricsPath + String metricSpecPath ) { this( @@ -83,20 +81,20 @@ public LoggingEmitter( shouldFilterMetrics ? MetricAllowlistLoader.loadAllowlist( jsonMapper, - Strings.isNullOrEmpty(allowedMetricsPath) ? DEFAULT_METRIC_ALLOWLIST_PATH : allowedMetricsPath, + Strings.isNullOrEmpty(metricSpecPath) ? MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH : metricSpecPath, MetricAllowlistParsers::parseMetricNameObject ) : ImmutableSet.of() ); } - public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper, boolean filterMetrics, Set metricAllowlist) + public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper, boolean shouldFilterMetrics, Set metricNames) { this.log = log; this.level = level; this.jsonMapper = jsonMapper; - this.filterMetrics = filterMetrics; - this.metricAllowlist = Set.copyOf(metricAllowlist); + this.shouldFilterMetrics = shouldFilterMetrics; + this.metricNames = Set.copyOf(metricNames); } @Override @@ -218,14 +216,14 @@ public String toString() return "LoggingEmitter{" + "log=" + log + ", level=" + level + - ", filterMetrics=" + filterMetrics + + ", shouldFilterMetrics=" + shouldFilterMetrics + '}'; } @Override public boolean shouldFilterOutMetric(String metricName) { - return filterMetrics && !metricAllowlist.contains(metricName); + return shouldFilterMetrics && !metricNames.contains(metricName); } @Override @@ -235,9 +233,9 @@ public MetricAllowlistParser getMetricAllowlistParser() } @VisibleForTesting - Set getMetricAllowlist() + Set getMetricNames() { - return metricAllowlist; + return metricNames; } public enum Level diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java index b6bbbc07c51d..53ccc8ada77c 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java @@ -51,8 +51,8 @@ public String toString() return "LoggingEmitterConfig{" + "loggerClass='" + loggerClass + '\'' + ", logLevel='" + logLevel + '\'' + - ", filterMetrics=" + filterMetrics + - ", metricAllowlistPath='" + metricAllowlistPath + '\'' + + ", shouldFilterMetrics=" + isShouldFilterMetrics() + + ", metricSpecPath='" + getMetricSpecPath() + '\'' + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java index 16f2c6582328..36c523152ae4 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java @@ -92,11 +92,11 @@ public ParametrizedUriEmitter( this.client = client; this.jsonMapper = jsonMapper; this.uriExtractor = uriExtractor; - this.filterMetrics = config.isFilterMetrics(); + this.filterMetrics = config.isShouldFilterMetrics(); this.metricAllowlist = this.filterMetrics ? MetricAllowlistLoader.loadAllowlist( this.jsonMapper, - config.getMetricAllowlistPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH), + config.getMetricSpecPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH), getMetricAllowlistParser() ) : Set.of(); diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java index de5da7d4b161..46a2977cac30 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java @@ -50,8 +50,8 @@ public String toString() return "ParametrizedUriEmitterConfig{" + "recipientBaseUrlPattern='" + recipientBaseUrlPattern + '\'' + ", httpEmittingConfig=" + httpEmittingConfig + - ", filterMetrics=" + filterMetrics + - ", metricAllowlistPath='" + metricAllowlistPath + '\'' + + ", shouldFilterMetrics=" + isShouldFilterMetrics() + + ", metricSpecPath='" + getMetricSpecPath() + '\'' + '}'; } } diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java index 361fc0ab8e59..34c353998573 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java @@ -35,13 +35,13 @@ public class LoggingEmitterAllowlistConfigTest public void testUsesDefaultAllowlistWhenCustomPathIsNotProvided() { final Map configMap = new HashMap<>(); - configMap.put("filterMetrics", true); + configMap.put("shouldFilterMetrics", true); final LoggingEmitterConfig config = new ObjectMapper().convertValue(configMap, LoggingEmitterConfig.class); final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); Assert.assertNotNull(emitter); - Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/cpu")); - Assert.assertTrue(emitter.getMetricAllowlist().contains("query/time")); + Assert.assertTrue(emitter.getMetricNames().contains("jvm/gc/cpu")); + Assert.assertTrue(emitter.getMetricNames().contains("query/time")); } @Test @@ -51,15 +51,15 @@ public void testReadsCustomAllowlistAsMetricObject() throws IOException Files.writeString(allowlist, "{\"jvm/gc/cpu\": [], \"jvm/gc/count\": []}"); final Map configMap = new HashMap<>(); - configMap.put("filterMetrics", true); - configMap.put("metricAllowlistPath", allowlist.toAbsolutePath().toString()); + configMap.put("shouldFilterMetrics", true); + configMap.put("metricSpecPath", allowlist.toAbsolutePath().toString()); final LoggingEmitterConfig config = new ObjectMapper().convertValue(configMap, LoggingEmitterConfig.class); final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); Assert.assertNotNull(emitter); - Assert.assertEquals(2, emitter.getMetricAllowlist().size()); - Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/cpu")); - Assert.assertTrue(emitter.getMetricAllowlist().contains("jvm/gc/count")); - Assert.assertFalse(emitter.getMetricAllowlist().contains("query/time")); + Assert.assertEquals(2, emitter.getMetricNames().size()); + Assert.assertTrue(emitter.getMetricNames().contains("jvm/gc/cpu")); + Assert.assertTrue(emitter.getMetricNames().contains("jvm/gc/count")); + Assert.assertFalse(emitter.getMetricNames().contains("query/time")); } } diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java index b7189699b61d..bda6314320da 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfigTest.java @@ -38,8 +38,8 @@ public void testDefaults() ); Assert.assertEquals("getLoggerClass", LoggingEmitter.class.getName(), config.getLoggerClass()); Assert.assertEquals("getLogLevel", "info", config.getLogLevel()); - Assert.assertFalse("shouldFilterMetrics", config.shouldFilterMetrics()); - Assert.assertNull("getAllowedMetricsPath", config.getAllowedMetricsPath()); + Assert.assertFalse("isShouldFilterMetrics", config.isShouldFilterMetrics()); + Assert.assertFalse("getMetricSpecPath", config.getMetricSpecPath().isPresent()); } @Test @@ -54,8 +54,8 @@ public void testDefaultsLegacy() Assert.assertEquals("getLoggerClass", LoggingEmitter.class.getName(), config.getLoggerClass()); Assert.assertEquals("getLogLevel", "debug", config.getLogLevel()); - Assert.assertFalse("shouldFilterMetrics", config.shouldFilterMetrics()); - Assert.assertNull("getAllowedMetricsPath", config.getAllowedMetricsPath()); + Assert.assertFalse("isShouldFilterMetrics", config.isShouldFilterMetrics()); + Assert.assertFalse("getMetricSpecPath", config.getMetricSpecPath().isPresent()); } @Test @@ -65,7 +65,7 @@ public void testSettingEverything() props.setProperty("org.apache.druid.java.util.emitter.loggerClass", "Foo"); props.setProperty("org.apache.druid.java.util.emitter.logLevel", "INFO"); props.setProperty("org.apache.druid.java.util.emitter.shouldFilterMetrics", "true"); - props.setProperty("org.apache.druid.java.util.emitter.allowedMetricsPath", "/tmp/allowedMetrics.json"); + props.setProperty("org.apache.druid.java.util.emitter.metricSpecPath", "/tmp/logging-metrics.json"); final ObjectMapper objectMapper = new ObjectMapper(); final LoggingEmitterConfig config = objectMapper.convertValue( @@ -75,8 +75,8 @@ public void testSettingEverything() Assert.assertEquals("getLoggerClass", "Foo", config.getLoggerClass()); Assert.assertEquals("getLogLevel", "INFO", config.getLogLevel()); - Assert.assertTrue("shouldFilterMetrics", config.shouldFilterMetrics()); - Assert.assertEquals("getAllowedMetricsPath", "/tmp/allowedMetrics.json", config.getAllowedMetricsPath()); + Assert.assertTrue("isShouldFilterMetrics", config.isShouldFilterMetrics()); + Assert.assertEquals("getMetricSpecPath", "/tmp/logging-metrics.json", config.getMetricSpecPath().orElse(null)); } @Test @@ -86,7 +86,7 @@ public void testSettingEverythingLegacy() props.setProperty("org.apache.druid.java.util.emitter.logging.class", "Foo"); props.setProperty("org.apache.druid.java.util.emitter.logging.level", "INFO"); props.setProperty("org.apache.druid.java.util.emitter.logging.shouldFilterMetrics", "true"); - props.setProperty("org.apache.druid.java.util.emitter.logging.allowedMetricsPath", "/custom/path.json"); + props.setProperty("org.apache.druid.java.util.emitter.logging.metricSpecPath", "/tmp/logging-metrics.json"); final ObjectMapper objectMapper = new ObjectMapper(); final LoggingEmitterConfig config = objectMapper.convertValue( @@ -96,7 +96,7 @@ public void testSettingEverythingLegacy() Assert.assertEquals("getLoggerClass", "Foo", config.getLoggerClass()); Assert.assertEquals("getLogLevel", "INFO", config.getLogLevel()); - Assert.assertTrue("shouldFilterMetrics", config.shouldFilterMetrics()); - Assert.assertEquals("getAllowedMetricsPath", "/custom/path.json", config.getAllowedMetricsPath()); + Assert.assertTrue("isShouldFilterMetrics", config.isShouldFilterMetrics()); + Assert.assertEquals("getMetricSpecPath", "/tmp/logging-metrics.json", config.getMetricSpecPath().orElse(null)); } } From 2a549fd6d1eb3d1c3c25e12e1f918eef043c2c37 Mon Sep 17 00:00:00 2001 From: Misha Date: Tue, 10 Mar 2026 17:19:32 +0100 Subject: [PATCH 08/12] address review comments --- docs/configuration/index.md | 2 +- .../druid/emitter/prometheus/Metrics.java | 4 +- .../emitter/prometheus/PrometheusEmitter.java | 65 ++-- .../prometheus/PrometheusEmitterConfig.java | 10 + ...ics.json => defaultPrometheusMetrics.json} | 0 .../druid/emitter/prometheus/MetricsTest.java | 42 ++- .../prometheus/PrometheusEmitterTest.java | 44 +++ .../core/AbstractFilteringEmitter.java | 106 ++++++ .../emitter/core/BaseHttpEmittingConfig.java | 1 + .../java/util/emitter/core/Emitters.java | 12 + .../emitter/core/GlobalEmitterConfig.java | 17 + .../util/emitter/core/HttpEmitterConfig.java | 3 +- .../util/emitter/core/HttpPostEmitter.java | 41 ++- .../util/emitter/core/LoggingEmitter.java | 57 ++-- .../emitter/core/LoggingEmitterConfig.java | 2 + .../emitter/core/MetricAllowlistLoader.java | 80 +++-- .../emitter/core/MetricAllowlistParser.java | 13 + .../emitter/core/MetricAllowlistParsers.java | 6 +- .../emitter/core/MetricFilteringEmitter.java | 13 + .../emitter/core/ParametrizedUriEmitter.java | 40 ++- .../core/ParametrizedUriEmitterConfig.java | 1 + .../main/resources/defaultLoggingMetrics.json | 308 ------------------ .../src/main/resources/defaultMetrics.json | 155 ++++++--- .../emitter/core/HttpEmitterConfigTest.java | 15 + .../emitter/core/HttpPostEmitterTest.java | 47 ++- .../LoggingEmitterAllowlistConfigTest.java | 12 +- .../core/MetricAllowlistParsersTest.java | 63 ++++ .../core/ParametrizedUriEmitterTest.java | 49 +++ 28 files changed, 700 insertions(+), 508 deletions(-) rename extensions-contrib/prometheus-emitter/src/main/resources/{defaultMetrics.json => defaultPrometheusMetrics.json} (100%) create mode 100644 processing/src/main/java/org/apache/druid/java/util/emitter/core/AbstractFilteringEmitter.java delete mode 100644 processing/src/main/resources/defaultLoggingMetrics.json create mode 100644 processing/src/test/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsersTest.java diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 04e5dc2af58b..8a10ec7baac7 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -2029,7 +2029,7 @@ log4j config to route these logs to different sources based on the feed of the e |`druid.emitter.logging.loggerClass`|The class used for logging.|`org.apache.druid.java.util.emitter.core.LoggingEmitter`| |`druid.emitter.logging.logLevel`|Choices: debug, info, warn, error. The log level at which message are logged.|info| |`druid.emitter.logging.shouldFilterMetrics`|When true, only metrics listed in the allow list are emitted; non-metric events (e.g. alerts) are always emitted. When false, all events are logged (backward-compatible).|false| -|`druid.emitter.logging.allowedMetricsPath`|Path to a JSON file whose keys are the allowed metric names. Only used when `shouldFilterMetrics` is true. If null or empty, the bundled classpath resource `defaultMetrics.json` is used. If a path is set but the file is missing, a warning is logged and the emitter falls back to the default classpath resource.|null| +|`druid.emitter.logging.metricSpecPath`|Path to a JSON file whose keys are the allowed metric names. Only used when `shouldFilterMetrics` is true. If null or empty, the bundled classpath resource `defaultMetrics.json` is used.|null| #### HTTP emitter module diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java index 4edb76e7d84c..c396ac186544 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -66,7 +66,7 @@ public DimensionsAndCollector getByName(String name, String service) public Metrics(PrometheusEmitterConfig config) { String namespace = config.getNamespace(); - String path = config.getDimensionMapPath(); + String path = config.getMetricSpecPath().orElse(config.getDimensionMapPath()); boolean isAddHostAsLabel = config.isAddHostAsLabel(); boolean isAddServiceAsLabel = config.isAddServiceAsLabel(); Map extraLabels = config.getExtraLabels(); @@ -136,7 +136,7 @@ private Map readConfig(String path) InputStream is; if (Strings.isNullOrEmpty(path)) { log.info("Using default metric configuration"); - is = this.getClass().getClassLoader().getResourceAsStream("defaultMetrics.json"); + is = this.getClass().getClassLoader().getResourceAsStream(PrometheusEmitterConfig.DEFAULT_METRIC_SPEC_PATH); } else { log.info("Using metric configuration at [%s]", path); is = new FileInputStream(new File(path)); diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java index 3f054b01ef6c..80169ffcea00 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Counter; @@ -31,12 +30,11 @@ import io.prometheus.client.exporter.PushGateway; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.emitter.core.Emitter; +import org.apache.druid.java.util.emitter.core.AbstractFilteringEmitter; import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.core.MetricAllowlistLoader; import org.apache.druid.java.util.emitter.core.MetricAllowlistParser; import org.apache.druid.java.util.emitter.core.MetricAllowlistParsers; -import org.apache.druid.java.util.emitter.core.MetricFilteringEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import java.io.IOException; @@ -46,22 +44,17 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Set; +import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; -/** - * - */ -public class PrometheusEmitter implements Emitter, MetricFilteringEmitter +public class PrometheusEmitter extends AbstractFilteringEmitter { private static final Logger log = new Logger(PrometheusEmitter.class); private final Metrics metrics; private final PrometheusEmitterConfig config; private final PrometheusEmitterConfig.Strategy strategy; - private final boolean filterMetrics; - private final Set metricAllowlist; private static final Pattern PATTERN = Pattern.compile("[^a-zA-Z0-9_][^a-zA-Z0-9_]*"); private static final String TAG_HOSTNAME = "host_name"; @@ -79,21 +72,25 @@ static PrometheusEmitter of(PrometheusEmitterConfig config) public PrometheusEmitter(PrometheusEmitterConfig config) { + super( + config.isShouldFilterMetrics(), + config.isShouldFilterMetrics() + ? config.getMetricSpecPath() + .or(() -> Optional.ofNullable(config.getDimensionMapPath())) + .map(path -> MetricAllowlistLoader.loadAllowlistFromFile( + new ObjectMapper(), + path, + MetricAllowlistParsers::parseMetricNameObject + )) + .orElseGet(() -> MetricAllowlistLoader.loadAllowlistFromClasspath( + new ObjectMapper(), + PrometheusEmitterConfig.DEFAULT_METRIC_SPEC_PATH, + MetricAllowlistParsers::parseMetricNameObject + )) + : Collections.emptySet() + ); this.config = config; this.strategy = config.getStrategy(); - this.filterMetrics = config.isShouldFilterMetrics(); - this.metricAllowlist = this.filterMetrics - ? MetricAllowlistLoader.loadAllowlist( - new ObjectMapper(), - config.getMetricSpecPath() - .orElseGet( - () -> Strings.isNullOrEmpty(config.getDimensionMapPath()) - ? "defaultMetrics.json" - : config.getDimensionMapPath() - ), - getMetricAllowlistParser() - ) - : Collections.emptySet(); metrics = new Metrics(config); } @@ -157,19 +154,21 @@ private static URL createURLSneakily(final String urlString) } @Override - public void emit(Event event) + protected boolean shouldFilterEvent(final Event event) { - if (event instanceof ServiceMetricEvent) { - emitMetric((ServiceMetricEvent) event); - } + return !(event instanceof ServiceMetricEvent) + || shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric()); } - private void emitMetric(ServiceMetricEvent metricEvent) + @Override + protected void emitFilteredEvent(final Event event) { - String name = metricEvent.getMetric(); - if (shouldFilterOutMetric(name)) { + // for events passed initial filering when it's diasabled + if (!(event instanceof ServiceMetricEvent)) { return; } + final ServiceMetricEvent metricEvent = (ServiceMetricEvent) event; + String name = metricEvent.getMetric(); String service = metricEvent.getService(); String host = metricEvent.getHost(); Map userDims = metricEvent.getUserDims(); @@ -311,12 +310,6 @@ public void setPushGateway(PushGateway pushGateway) this.pushGateway = pushGateway; } - @Override - public boolean shouldFilterOutMetric(String metricName) - { - return filterMetrics && !metricAllowlist.contains(metricName); - } - @Override public MetricAllowlistParser getMetricAllowlistParser() { diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java index a2ef2ab76347..16d6fd3d54fe 100644 --- a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java @@ -38,6 +38,7 @@ */ public class PrometheusEmitterConfig extends GlobalEmitterConfig { + public static final String DEFAULT_METRIC_SPEC_PATH = "defaultPrometheusMetrics.json"; static final Pattern PATTERN = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); @@ -47,6 +48,11 @@ public class PrometheusEmitterConfig extends GlobalEmitterConfig @JsonProperty private final String namespace; + /** + * Deprecated in favor of {@link GlobalEmitterConfig#getMetricSpecPath()} kept for backward compatibility. + * If metricSpecPath is not provided, the emitter falls back to this value. + */ + @Deprecated @JsonProperty @Nullable private final String dimensionMapPath; @@ -160,6 +166,10 @@ public String getNamespace() return namespace; } + /** + * Deprecated alias for {@link GlobalEmitterConfig#getMetricSpecPath()}. + */ + @Deprecated public String getDimensionMapPath() { return dimensionMapPath; diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultPrometheusMetrics.json similarity index 100% rename from extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json rename to extensions-contrib/prometheus-emitter/src/main/resources/defaultPrometheusMetrics.json diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java index 7139e446ab4f..489901bb34db 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java @@ -25,6 +25,9 @@ import org.junit.Assert; import org.junit.Test; +import java.net.URISyntaxException; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.HashMap; import java.util.Map; @@ -114,7 +117,19 @@ public void testMetricsConfigurationWithNonExistentMetric() @Test public void testMetricsConfigurationWithUnSupportedType() { - PrometheusEmitterConfig config = new PrometheusEmitterConfig(null, "test_5", "src/test/resources/defaultInvalidMetricsTest.json", null, null, true, true, null, null, null, null); + PrometheusEmitterConfig config = new PrometheusEmitterConfig( + null, + "test_5", + resolveResourcePath("defaultInvalidMetricsTest.json"), + null, + null, + true, + true, + null, + null, + null, + null + ); ISE iseException = Assert.assertThrows(ISE.class, () -> { new Metrics(config); }); @@ -124,7 +139,19 @@ public void testMetricsConfigurationWithUnSupportedType() @Test public void testMetricsConfigurationWithTimerHistogramBuckets() { - PrometheusEmitterConfig config = new PrometheusEmitterConfig(null, "test_6", "src/test/resources/defaultMetricsTest.json", null, null, true, true, null, null, null, null); + PrometheusEmitterConfig config = new PrometheusEmitterConfig( + null, + "test_6", + resolveResourcePath("defaultMetricsTest.json"), + null, + null, + true, + true, + null, + null, + null, + null + ); Metrics metrics = new Metrics(config); DimensionsAndCollector dimensionsAndCollector = metrics.getByName("query/time", "historical"); Assert.assertNotNull(dimensionsAndCollector); @@ -138,4 +165,15 @@ public void testMetricsConfigurationWithTimerHistogramBuckets() Assert.assertArrayEquals(expectedHistogramBuckets, dimensionsAndCollector.getHistogramBuckets(), 0.0); } + private static String resolveResourcePath(final String resourceName) + { + try { + final Path resourcePath = Paths.get(MetricsTest.class.getClassLoader().getResource(resourceName).toURI()); + return resourcePath.toString(); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + } diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java index 2158260afb3b..fb6bda4939a3 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.emitter.core.Emitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.java.util.emitter.service.UnitEvent; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -605,6 +606,49 @@ public void testCounterWithNegativeValue() emitter.close(); } + @Test + public void testFilteringAllowsConfiguredMetricOnly() throws Exception + { + final PrometheusEmitterConfig config = new PrometheusEmitterConfig( + PrometheusEmitterConfig.Strategy.exporter, + "druid", + null, + 0, + null, + false, + false, + null, + null, + false, + null + ); + config.setShouldFilterMetrics(true); + + final PrometheusEmitter emitter = new PrometheusEmitter(config); + Assert.assertFalse(emitter.shouldFilterOutMetric("query/time")); + Assert.assertTrue(emitter.shouldFilterOutMetric("some/unlisted/metric")); + emitter.emit( + ServiceMetricEvent.builder().setMetric("query/time", 10).setDimension("dataSource", "wikipedia").setDimension("type", "groupBy") + .build(ImmutableMap.of("service", "broker", "host", "druid.test.cn")) + ); + emitter.emit(ServiceMetricEvent.builder().setMetric("some/unlisted/metric", 5).build("broker", "druid.test.cn")); + emitter.emit(new UnitEvent("alerts", 1)); + + Assert.assertEquals( + 1.0, + CollectorRegistry.defaultRegistry.getSampleValue( + "druid_query_time_bucket", + new String[]{"dataSource", "type", "le"}, + new String[]{"wikipedia", "groupBy", "0.1"} + ), + 0.0 + ); + Assert.assertNull( + CollectorRegistry.defaultRegistry.getSampleValue("druid_some_unlisted_metric") + ); + emitter.close(); + } + @After public void tearDown() { diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/AbstractFilteringEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/AbstractFilteringEmitter.java new file mode 100644 index 000000000000..f2cf4fadd4b1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/AbstractFilteringEmitter.java @@ -0,0 +1,106 @@ +/* + * 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.java.util.emitter.core; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.util.Optional; +import java.util.Set; + +/** + * Base class for emitters that support metric-name-based filtering. + * + *

The base implementation provides a single filtering gate in {@link #emit(Event)} and then + * delegates actual emission to {@link #emitFilteredEvent(Event)}. + */ +public abstract class AbstractFilteringEmitter implements Emitter, MetricFilteringEmitter +{ + private final boolean shouldFilterMetrics; + private final Set allowedMetricNames; + + protected AbstractFilteringEmitter(final boolean shouldFilterMetrics, final Set allowedMetricNames) + { + this.shouldFilterMetrics = shouldFilterMetrics; + this.allowedMetricNames = Set.copyOf(allowedMetricNames); + } + + protected static Set loadAllowedMetricNames( + final boolean shouldFilterMetrics, + final ObjectMapper objectMapper, + final Optional metricSpecPath, + final String defaultMetricSpecPath, + final MetricAllowlistParser parser + ) + { + if (!shouldFilterMetrics) { + return Set.of(); + } + + return metricSpecPath + .map(path -> MetricAllowlistLoader.loadAllowlistFromFile(objectMapper, path, parser)) + .orElseGet(() -> MetricAllowlistLoader.loadAllowlistFromClasspath(objectMapper, defaultMetricSpecPath, parser)); + } + + @Override + public final void emit(final Event event) + { + preEmit(event); + if (shouldFilterMetrics && shouldFilterEvent(event)) { + return; + } + emitFilteredEvent(event); + } + + @Override + public boolean shouldFilterOutMetric(final String metricName) + { + return !allowedMetricNames.contains(metricName); + } + + protected boolean isShouldFilterMetrics() + { + return shouldFilterMetrics; + } + + protected Set getAllowedMetricNames() + { + return allowedMetricNames; + } + + /** + * Returns whether this event should be dropped before emission. + * + *

Implementations should apply emitter-specific event handling semantics here. + */ + protected abstract boolean shouldFilterEvent(Event event); + + /** + * Hook for pre-emission checks that must run before metric filtering. + */ + protected void preEmit(final Event event) + { + // default no-op + } + + /** + * Emits an event that already passed filtering (or is not a metric event). + */ + protected abstract void emitFilteredEvent(Event event); +} diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java index d17c2f99b6c0..21ab6feb378b 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java @@ -29,6 +29,7 @@ public class BaseHttpEmittingConfig extends GlobalEmitterConfig { + public static final String DEFAULT_METRIC_SPEC_PATH = "defaultMetrics.json"; public static final long DEFAULT_FLUSH_MILLIS = 60 * 1000; public static final int DEFAULT_FLUSH_COUNTS = 500; diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java index 769615325d67..f7e19613da85 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/Emitters.java @@ -126,6 +126,18 @@ static Map makeHttpMap(Properties props) Float.parseFloat(props.getProperty("org.apache.druid.java.util.emitter.http.minHttpTimeoutMillis")) ); } + if (props.containsKey("org.apache.druid.java.util.emitter.shouldFilterMetrics")) { + httpMap.put( + "shouldFilterMetrics", + Boolean.parseBoolean(props.getProperty("org.apache.druid.java.util.emitter.shouldFilterMetrics")) + ); + } + if (props.containsKey("org.apache.druid.java.util.emitter.metricSpecPath")) { + httpMap.put( + "metricSpecPath", + props.getProperty("org.apache.druid.java.util.emitter.metricSpecPath") + ); + } return httpMap; } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java index ca74cc0b523e..369caf783711 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/GlobalEmitterConfig.java @@ -23,11 +23,28 @@ import java.util.Optional; +/** + * Shared metric-filtering configuration for emitter implementations. + */ public class GlobalEmitterConfig { + /** + * When true, only metrics listed in the allowed metrics configuration are emitted. + * If {@link #metricSpecPath} is null/empty, the bundled default allowlist + * (`defaultMetrics.json` on the classpath) is used. If a path is provided, + * it is loaded from that file instead. + * Defaults to false (emit all metrics). + */ @JsonProperty private boolean shouldFilterMetrics; + /** + * Optional path to a JSON file containing a JSON object keyed by allowed metric names, + * for example `{"query/time": [], "jvm/gc/cpu": []}`. + * Only used when {@link #shouldFilterMetrics} is true. + * If null or empty, the bundled default resource (`defaultMetrics.json`) is loaded + * from the classpath. + */ @JsonProperty private String metricSpecPath; diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java index 26df2c0f6cb4..e5d326d6d3ce 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java @@ -20,7 +20,6 @@ package org.apache.druid.java.util.emitter.core; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.commons.lang3.StringUtils; import org.apache.druid.metadata.PasswordProvider; import javax.validation.constraints.NotNull; @@ -54,7 +53,7 @@ public HttpEmitterConfig(BaseHttpEmittingConfig base, String recipientBaseUrl) this.httpTimeoutAllowanceFactor = base.httpTimeoutAllowanceFactor; this.minHttpTimeoutMillis = base.minHttpTimeoutMillis; this.setShouldFilterMetrics(base.isShouldFilterMetrics()); - this.setMetricSpecPath(base.getMetricSpecPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH)); + base.getMetricSpecPath().ifPresent(this::setMetricSpecPath); } public String getRecipientBaseUrl() diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java index 2748881ec9dc..9c83ab700048 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java @@ -47,7 +47,6 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayDeque; -import java.util.Set; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; @@ -61,7 +60,7 @@ import java.util.concurrent.locks.LockSupport; import java.util.zip.GZIPOutputStream; -public class HttpPostEmitter implements Flushable, Closeable, Emitter, MetricFilteringEmitter +public class HttpPostEmitter extends AbstractFilteringEmitter implements Flushable, Closeable { private static final int MAX_EVENT_SIZE = 1023 * 1024; // Set max size slightly less than 1M to allow for metadata @@ -102,8 +101,6 @@ public class HttpPostEmitter implements Flushable, Closeable, Emitter, MetricFil private final AsyncHttpClient client; private final ObjectMapper jsonMapper; private final String url; - private final boolean filterMetrics; - private final Set metricAllowlist; private final ConcurrentLinkedQueue buffersToReuse = new ConcurrentLinkedQueue<>(); /** @@ -156,6 +153,16 @@ public HttpPostEmitter(HttpEmitterConfig config, AsyncHttpClient client) public HttpPostEmitter(HttpEmitterConfig config, AsyncHttpClient client, ObjectMapper jsonMapper) { + super( + config.isShouldFilterMetrics(), + loadAllowedMetricNames( + config.isShouldFilterMetrics(), + jsonMapper, + config.getMetricSpecPath(), + BaseHttpEmittingConfig.DEFAULT_METRIC_SPEC_PATH, + MetricAllowlistParsers::parseMetricNameObject + ) + ); batchingStrategy = config.getBatchingStrategy(); final int batchOverhead = batchingStrategy.batchStartLength() + batchingStrategy.batchEndLength(); Preconditions.checkArgument( @@ -173,14 +180,6 @@ public HttpPostEmitter(HttpEmitterConfig config, AsyncHttpClient client, ObjectM this.largeEventThreshold = (bufferSize - batchOverhead - batchingStrategy.separatorLength()) / 2; this.client = client; this.jsonMapper = jsonMapper; - this.filterMetrics = config.isShouldFilterMetrics(); - this.metricAllowlist = this.filterMetrics - ? MetricAllowlistLoader.loadAllowlist( - this.jsonMapper, - config.getMetricSpecPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH), - getMetricAllowlistParser() - ) - : Set.of(); try { this.url = new URL(config.getRecipientBaseUrl()).toString(); } @@ -233,11 +232,15 @@ private boolean isTerminated() } @Override - public void emit(Event event) + protected boolean shouldFilterEvent(final Event event) + { + return event instanceof ServiceMetricEvent + && shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric()); + } + + @Override + protected void emitFilteredEvent(final Event event) { - if (event instanceof ServiceMetricEvent && shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric())) { - return; - } emitAndReturnBatch(event); } @@ -294,12 +297,6 @@ private byte[] eventToBytes(Event event) } } - @Override - public boolean shouldFilterOutMetric(String metricName) - { - return filterMetrics && !metricAllowlist.contains(metricName); - } - @Override public MetricAllowlistParser getMetricAllowlistParser() { diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java index ccec4500e75e..1267088e4028 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java @@ -19,13 +19,8 @@ package org.apache.druid.java.util.emitter.core; -/** - */ - import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; -import com.google.common.collect.ImmutableSet; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; @@ -33,20 +28,16 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.slf4j.MarkerFactory; +import java.util.Optional; import java.util.Set; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicBoolean; -/** - */ -public class LoggingEmitter implements Emitter, MetricFilteringEmitter +public class LoggingEmitter extends AbstractFilteringEmitter { private final Logger log; private final Level level; private final ObjectMapper jsonMapper; - private final boolean shouldFilterMetrics; - private final Set metricNames; - private final AtomicBoolean started = new AtomicBoolean(false); public LoggingEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper) @@ -56,7 +47,7 @@ public LoggingEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper) Level.toLevel(config.getLogLevel()), jsonMapper, config.isShouldFilterMetrics(), - config.getMetricSpecPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH) + config.getMetricSpecPath().orElse(null) ); } @@ -78,23 +69,22 @@ public LoggingEmitter( level, jsonMapper, shouldFilterMetrics, - shouldFilterMetrics - ? MetricAllowlistLoader.loadAllowlist( + loadAllowedMetricNames( + shouldFilterMetrics, jsonMapper, - Strings.isNullOrEmpty(metricSpecPath) ? MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH : metricSpecPath, + Strings.isNullOrEmpty(metricSpecPath) ? Optional.empty() : Optional.of(metricSpecPath), + LoggingEmitterConfig.DEFAULT_METRIC_SPEC_PATH, MetricAllowlistParsers::parseMetricNameObject ) - : ImmutableSet.of() ); } public LoggingEmitter(Logger log, Level level, ObjectMapper jsonMapper, boolean shouldFilterMetrics, Set metricNames) { + super(shouldFilterMetrics, metricNames); this.log = log; this.level = level; this.jsonMapper = jsonMapper; - this.shouldFilterMetrics = shouldFilterMetrics; - this.metricNames = Set.copyOf(metricNames); } @Override @@ -131,16 +121,25 @@ public void start() } @Override - public void emit(Event event) + protected boolean shouldFilterEvent(final Event event) + { + return event instanceof ServiceMetricEvent + && shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric()); + } + + @Override + protected void preEmit(final Event event) { synchronized (started) { if (!started.get()) { throw new RejectedExecutionException("Service not started."); } } - if (event instanceof ServiceMetricEvent && shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric())) { - return; - } + } + + @Override + protected void emitFilteredEvent(final Event event) + { try { switch (level) { case TRACE: @@ -216,28 +215,16 @@ public String toString() return "LoggingEmitter{" + "log=" + log + ", level=" + level + - ", shouldFilterMetrics=" + shouldFilterMetrics + + ", shouldFilterMetrics=" + isShouldFilterMetrics() + '}'; } - @Override - public boolean shouldFilterOutMetric(String metricName) - { - return shouldFilterMetrics && !metricNames.contains(metricName); - } - @Override public MetricAllowlistParser getMetricAllowlistParser() { return MetricAllowlistParsers::parseMetricNameObject; } - @VisibleForTesting - Set getMetricNames() - { - return metricNames; - } - public enum Level { TRACE, diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java index 53ccc8ada77c..c09e630e5d3a 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitterConfig.java @@ -27,6 +27,8 @@ */ public class LoggingEmitterConfig extends GlobalEmitterConfig { + public static final String DEFAULT_METRIC_SPEC_PATH = "defaultMetrics.json"; + @NotNull @JsonProperty private String loggerClass = LoggingEmitter.class.getName(); diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistLoader.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistLoader.java index bcbd17358c8e..5da08c3bdf72 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistLoader.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistLoader.java @@ -21,9 +21,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.error.DruidException; +import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; @@ -33,36 +33,72 @@ public final class MetricAllowlistLoader { - public static final String DEFAULT_METRIC_ALLOWLIST_PATH = "defaultLoggingMetrics.json"; - - public static Set loadAllowlist( - ObjectMapper mapper, - String allowlistPath, - MetricAllowlistParser parser + public static Set loadAllowlistFromFile( + final ObjectMapper mapper, + final String allowlistPath, + final MetricAllowlistParser parser ) { - try (final InputStream is = openAllowlistFile(allowlistPath)) { - return parser.parse(mapper.readTree(is), allowlistPath); + validateAllowlistPath(allowlistPath); + + final File allowlistFile = new File(allowlistPath); + + try (final InputStream is = new FileInputStream(allowlistFile)) { + return parseAllowlist(mapper, is, allowlistFile.getPath(), parser); + } + catch (FileNotFoundException e) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.NOT_FOUND) + .build("Metric spec file path [%s] was not found.", allowlistFile.getPath()); } catch (IOException e) { - throw new ISE(e, "Failed to parse metric allowlist file [%s]", allowlistPath); + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Failed to parse metric spec file path [%s].", allowlistFile.getPath()); } } - private static InputStream openAllowlistFile(String allowlistPath) + public static Set loadAllowlistFromClasspath( + final ObjectMapper mapper, + final String resourcePath, + final MetricAllowlistParser parser + ) { - if (Strings.isNullOrEmpty(allowlistPath)) { - throw new IAE("Metric allowlist file path is empty"); + validateAllowlistPath(resourcePath); + + final InputStream classpathInputStream = MetricAllowlistLoader.class.getClassLoader().getResourceAsStream(resourcePath); + if (classpathInputStream == null) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.NOT_FOUND) + .build("Metric spec file path [%s] was not found.", resourcePath); } - try { - return new FileInputStream(allowlistPath); + + try (final InputStream is = classpathInputStream) { + return parseAllowlist(mapper, is, resourcePath, parser); } - catch (FileNotFoundException e) { - final InputStream classpathInputStream = MetricAllowlistLoader.class.getClassLoader().getResourceAsStream(allowlistPath); - if (classpathInputStream != null) { - return classpathInputStream; - } - throw new IAE(e, "Metric allowlist file [%s] not found", allowlistPath); + catch (IOException e) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Failed to parse metric spec file path [%s].", resourcePath); + } + } + + private static Set parseAllowlist( + final ObjectMapper mapper, + final InputStream inputStream, + final String source, + final MetricAllowlistParser parser + ) throws IOException + { + return parser.parse(mapper.readTree(inputStream), source); + } + + private static void validateAllowlistPath(final String allowlistPath) + { + if (Strings.isNullOrEmpty(allowlistPath)) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Metric spec file path was empty, value [%s].", allowlistPath); } } } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParser.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParser.java index 59617f482894..6d9cc6828f30 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParser.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParser.java @@ -23,7 +23,20 @@ import java.util.Set; +/** + * Parses a metric specification document into a set of metric names that can be used for filtering. + * + *

The expected JSON shape is parser-dependent and is implemented by each parser implementation. + */ public interface MetricAllowlistParser { + /** + * Parses metric configuration from the provided JSON node. + * + * @param metricConfig root JSON node of the metric specification + * @param source source identifier used for error reporting (for example, a path) + * + * @return metric names extracted from the configuration + */ Set parse(JsonNode metricConfig, String source); } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsers.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsers.java index de73e7b4a348..b3195b4350aa 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsers.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsers.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.ImmutableSet; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.error.DruidException; import java.util.Set; @@ -31,7 +31,9 @@ public final class MetricAllowlistParsers public static Set parseMetricNameObject(JsonNode metricConfig, String source) { if (!metricConfig.isObject()) { - throw new ISE("Metric allowlist file [%s] must be a JSON object of metric names", source); + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Metric allowlist file [%s] must be a JSON object of metric names.", source); } final ImmutableSet.Builder metricNames = ImmutableSet.builder(); metricConfig.fieldNames().forEachRemaining(metricNames::add); diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricFilteringEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricFilteringEmitter.java index c2f7a9382b86..6b186048bd85 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricFilteringEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/MetricFilteringEmitter.java @@ -19,9 +19,22 @@ package org.apache.druid.java.util.emitter.core; +/** + * Contract for emitters that can filter metric events based on metric name. + */ public interface MetricFilteringEmitter { + /** + * Returns whether a metric should be filtered out (not emitted). + * + * @param metricName metric name from {@link org.apache.druid.java.util.emitter.service.ServiceMetricEvent} + * + * @return {@code true} when the metric should be dropped + */ boolean shouldFilterOutMetric(String metricName); + /** + * Returns the parser used to load metric names from the configured metric specification. + */ MetricAllowlistParser getMetricAllowlistParser(); } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java index 36c523152ae4..497ddc9c8dca 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java @@ -38,7 +38,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.function.BiConsumer; -public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter, MetricFilteringEmitter +public class ParametrizedUriEmitter extends AbstractFilteringEmitter implements Flushable, Closeable { private static final Logger log = new Logger(ParametrizedUriEmitter.class); private static final Set ONLY_FEED_PARAM = ImmutableSet.of("feed"); @@ -69,8 +69,6 @@ private static UriExtractor makeUriExtractor(ParametrizedUriEmitterConfig config private final AsyncHttpClient client; private final ObjectMapper jsonMapper; private final ParametrizedUriEmitterConfig config; - private final boolean filterMetrics; - private final Set metricAllowlist; public ParametrizedUriEmitter( ParametrizedUriEmitterConfig config, @@ -88,18 +86,20 @@ public ParametrizedUriEmitter( UriExtractor uriExtractor ) { + super( + config.isShouldFilterMetrics(), + loadAllowedMetricNames( + config.isShouldFilterMetrics(), + jsonMapper, + config.getMetricSpecPath(), + ParametrizedUriEmitterConfig.DEFAULT_METRIC_SPEC_PATH, + MetricAllowlistParsers::parseMetricNameObject + ) + ); this.config = config; this.client = client; this.jsonMapper = jsonMapper; this.uriExtractor = uriExtractor; - this.filterMetrics = config.isShouldFilterMetrics(); - this.metricAllowlist = this.filterMetrics - ? MetricAllowlistLoader.loadAllowlist( - this.jsonMapper, - config.getMetricSpecPath().orElse(MetricAllowlistLoader.DEFAULT_METRIC_ALLOWLIST_PATH), - getMetricAllowlistParser() - ) - : Set.of(); } @Override @@ -126,11 +126,15 @@ public void start() } @Override - public void emit(Event event) + protected boolean shouldFilterEvent(final Event event) + { + return event instanceof ServiceMetricEvent + && shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric()); + } + + @Override + protected void emitFilteredEvent(final Event event) { - if (event instanceof ServiceMetricEvent && shouldFilterOutMetric(((ServiceMetricEvent) event).getMetric())) { - return; - } try { URI uri = uriExtractor.apply(event); // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed. @@ -221,12 +225,6 @@ public String toString() '}'; } - @Override - public boolean shouldFilterOutMetric(String metricName) - { - return filterMetrics && !metricAllowlist.contains(metricName); - } - @Override public MetricAllowlistParser getMetricAllowlistParser() { diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java index 46a2977cac30..49ce61504448 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfig.java @@ -25,6 +25,7 @@ public class ParametrizedUriEmitterConfig extends GlobalEmitterConfig { + public static final String DEFAULT_METRIC_SPEC_PATH = BaseHttpEmittingConfig.DEFAULT_METRIC_SPEC_PATH; private static final BaseHttpEmittingConfig DEFAULT_HTTP_EMITTING_CONFIG = new BaseHttpEmittingConfig(); @NotNull diff --git a/processing/src/main/resources/defaultLoggingMetrics.json b/processing/src/main/resources/defaultLoggingMetrics.json deleted file mode 100644 index 433f49e002b1..000000000000 --- a/processing/src/main/resources/defaultLoggingMetrics.json +++ /dev/null @@ -1,308 +0,0 @@ -{ - "cgroup/cpu/cores_quota": [], - "cgroup/cpu/shares": [], - "cgroup/cpu/usage/sys/percentage": [], - "cgroup/cpu/usage/total/percentage": [], - "cgroup/cpu/usage/user/percentage": [], - "cgroup/cpuset/cpu_count": [], - "cgroup/cpuset/effective_cpu_count": [], - "cgroup/cpuset/effective_mems_count": [], - "cgroup/cpuset/mems_count": [], - "cgroup/disk/read/count": [], - "cgroup/disk/read/size": [], - "cgroup/disk/write/count": [], - "cgroup/disk/write/size": [], - "cgroup/memory/limit/bytes": [], - "cgroup/memory/usage/bytes": [], - "compact/segmentAnalyzer/fetchAndProcessMillis": [], - "compact/task/count": [], - "compactTask/availableSlot/count": [], - "compactTask/maxSlot/count": [], - "coordinator/global/time": [], - "coordinator/time": [], - "groupBy/maxMergeDictionarySize": [], - "groupBy/maxSpilledBytes": [], - "groupBy/mergeDictionarySize": [], - "groupBy/spilledBytes": [], - "groupBy/spilledQueries": [], - "ingest/count": [], - "ingest/events/avgMessageGap": [], - "ingest/events/duplicate": [], - "ingest/events/maxMessageGap": [], - "ingest/events/messageGap": [], - "ingest/events/minMessageGap": [], - "ingest/events/processed": [], - "ingest/events/processedWithError": [], - "ingest/events/thrownAway": [], - "ingest/events/unparseable": [], - "ingest/handoff/count": [], - "ingest/handoff/failed": [], - "ingest/handoff/time": [], - "ingest/input/bytes": [], - "ingest/kafka/avgLag": [], - "ingest/kafka/avgLag/time": [], - "ingest/kafka/fetchOffsets/time": [], - "ingest/kafka/lag": [], - "ingest/kafka/lag/time": [], - "ingest/kafka/maxLag": [], - "ingest/kafka/maxLag/time": [], - "ingest/kafka/partitionLag": [], - "ingest/kinesis/avgLag/time": [], - "ingest/kinesis/fetchOffsets/time": [], - "ingest/kinesis/lag/time": [], - "ingest/kinesis/maxLag/time": [], - "ingest/kinesis/partitionLag/time": [], - "ingest/merge/cpu": [], - "ingest/merge/time": [], - "ingest/notices/queueSize": [], - "ingest/notices/time": [], - "ingest/pause/time": [], - "ingest/persists/backPressure": [], - "ingest/persists/count": [], - "ingest/persists/cpu": [], - "ingest/persists/failed": [], - "ingest/persists/time": [], - "ingest/rows/output": [], - "ingest/segments/count": [], - "ingest/shuffle/bytes": [], - "ingest/shuffle/requests": [], - "ingest/sink/count": [], - "ingest/tombstones/count": [], - "interval/compacted/count": [], - "interval/skipCompact/count": [], - "interval/waitCompact/count": [], - "jetty/numOpenConnections": [], - "jetty/threadPool/busy": [], - "jetty/threadPool/idle": [], - "jetty/threadPool/isLowOnThreads": [], - "jetty/threadPool/max": [], - "jetty/threadPool/min": [], - "jetty/threadPool/queueSize": [], - "jetty/threadPool/ready": [], - "jetty/threadPool/total": [], - "jetty/threadPool/utilizationRate": [], - "jetty/threadPool/utilized": [], - "jvm/bufferpool/capacity": [], - "jvm/bufferpool/count": [], - "jvm/bufferpool/used": [], - "jvm/gc/count": [], - "jvm/gc/cpu": [], - "jvm/mem/committed": [], - "jvm/mem/init": [], - "jvm/mem/max": [], - "jvm/mem/used": [], - "jvm/pool/committed": [], - "jvm/pool/init": [], - "jvm/pool/max": [], - "jvm/pool/used": [], - "kill/eligibleUnusedSegments/count": [], - "kill/pendingSegments/count": [], - "kill/task/count": [], - "killTask/availableSlot/count": [], - "killTask/maxSlot/count": [], - "mergeBuffer/acquisitionTimeNs": [], - "mergeBuffer/maxAcquisitionTimeNs": [], - "mergeBuffer/pendingRequests": [], - "mergeBuffer/queries": [], - "mergeBuffer/used": [], - "metadata/kill/audit/count": [], - "metadata/kill/compaction/count": [], - "metadata/kill/datasource/count": [], - "metadata/kill/rule/count": [], - "metadata/kill/supervisor/count": [], - "metadatacache/init/time": [], - "metadatacache/schemaPoll/time": [], - "query/byteLimit/exceeded/count": [], - "query/bytes": [], - "query/cache/memcached/delta": [], - "query/cache/memcached/total": [], - "query/count": [], - "query/cpu/time": [], - "query/failed/count": [], - "query/interrupted/count": [], - "query/node/bytes": [], - "query/node/time": [], - "query/node/ttfb": [], - "query/priority": [], - "query/resultCache/hit": [], - "query/rowLimit/exceeded/count": [], - "query/segment/time": [], - "query/segmentAndCache/time": [], - "query/segments/count": [], - "query/success/count": [], - "query/time": [], - "query/timeout/count": [], - "query/wait/time": [], - "s3/upload/part/queueSize": [], - "s3/upload/part/queuedTime": [], - "s3/upload/part/time": [], - "s3/upload/total/bytes": [], - "s3/upload/total/time": [], - "segment/added/bytes": [], - "segment/assignSkipped/count": [], - "segment/assigned/count": [], - "segment/availableDeepStorageOnly/count": [], - "segment/compacted/bytes": [], - "segment/compacted/count": [], - "segment/count": [], - "segment/deleted/count": [], - "segment/dropQueue/count": [], - "segment/dropSkipped/count": [], - "segment/dropped/count": [], - "segment/kill/jobsProcessed/count": [], - "segment/kill/queueProcess/time": [], - "segment/kill/queueReset/time": [], - "segment/kill/skippedIntervals/count": [], - "segment/kill/unusedIntervals/count": [], - "segment/killed/deepStorage/count": [], - "segment/killed/metadataStore/count": [], - "segment/loadQueue/assigned": [], - "segment/loadQueue/cancelled": [], - "segment/loadQueue/count": [], - "segment/loadQueue/failed": [], - "segment/loadQueue/size": [], - "segment/loadQueue/success": [], - "segment/loading/rateKbps": [], - "segment/max": [], - "segment/metadataCache/dataSource/deleted": [], - "segment/metadataCache/deleted": [], - "segment/metadataCache/interval/count": [], - "segment/metadataCache/pending/count": [], - "segment/metadataCache/pending/deleted": [], - "segment/metadataCache/pending/skipped": [], - "segment/metadataCache/pending/updated": [], - "segment/metadataCache/skipped": [], - "segment/metadataCache/sync/time": [], - "segment/metadataCache/transactions/readOnly": [], - "segment/metadataCache/transactions/readWrite": [], - "segment/metadataCache/transactions/writeOnly": [], - "segment/metadataCache/used/count": [], - "segment/metadataCache/used/stale": [], - "segment/metadataCache/used/updated": [], - "segment/moveSkipped/count": [], - "segment/moved/bytes": [], - "segment/moved/count": [], - "segment/nuked/bytes": [], - "segment/overShadowed/count": [], - "segment/pending/count": [], - "segment/pendingDelete": [], - "segment/rowCount/avg": [], - "segment/rowCount/range/count": [], - "segment/scan/active": [], - "segment/scan/pending": [], - "segment/schemaCache/backfill/count": [], - "segment/schemaCache/dataSource/removed": [], - "segment/schemaCache/deepStorageOnly/count": [], - "segment/schemaCache/deepStorageOnly/refresh/time": [], - "segment/schemaCache/pendingBackfill/count": [], - "segment/schemaCache/poll/count": [], - "segment/schemaCache/poll/failed": [], - "segment/schemaCache/realtime/count": [], - "segment/schemaCache/refresh/count": [], - "segment/schemaCache/refresh/time": [], - "segment/schemaCache/refreshSkipped/count": [], - "segment/schemaCache/rowSignature/changed": [], - "segment/schemaCache/rowSignature/column/count": [], - "segment/schemaCache/used/count": [], - "segment/schemaCache/usedFingerprint/count": [], - "segment/size": [], - "segment/skipCompact/bytes": [], - "segment/skipCompact/count": [], - "segment/unavailable/count": [], - "segment/underReplicated/count": [], - "segment/unneeded/count": [], - "segment/unneededEternityTombstone/count": [], - "segment/used": [], - "segment/used/count": [], - "segment/used/deepStorageOnly/count": [], - "segment/usedPercent": [], - "segment/waitCompact/bytes": [], - "segment/waitCompact/count": [], - "serverview/init/time": [], - "serverview/sync/healthy": [], - "serverview/sync/unstableTime": [], - "sqlQuery/bytes": [], - "sqlQuery/planningTimeMs": [], - "sqlQuery/time": [], - "subquery/byteLimit/count": [], - "subquery/bytes": [], - "subquery/fallback/count": [], - "subquery/fallback/insufficientType/count": [], - "subquery/fallback/unknownReason/count": [], - "subquery/rowLimit/count": [], - "subquery/rows": [], - "sys/cpu": [], - "sys/disk/queue": [], - "sys/disk/read/count": [], - "sys/disk/read/size": [], - "sys/disk/transferTime": [], - "sys/disk/write/count": [], - "sys/disk/write/size": [], - "sys/fs/files/count": [], - "sys/fs/files/free": [], - "sys/fs/max": [], - "sys/fs/used": [], - "sys/mem/free": [], - "sys/mem/max": [], - "sys/mem/used": [], - "sys/net/read/dropped": [], - "sys/net/read/errors": [], - "sys/net/read/packets": [], - "sys/net/read/size": [], - "sys/net/write/collisions": [], - "sys/net/write/errors": [], - "sys/net/write/packets": [], - "sys/net/write/size": [], - "sys/storage/used": [], - "sys/swap/free": [], - "sys/swap/max": [], - "sys/swap/pageIn": [], - "sys/swap/pageOut": [], - "sys/tcpv4/activeOpens": [], - "sys/tcpv4/attemptFails": [], - "sys/tcpv4/estabResets": [], - "sys/tcpv4/in/errs": [], - "sys/tcpv4/in/segs": [], - "sys/tcpv4/out/rsts": [], - "sys/tcpv4/out/segs": [], - "sys/tcpv4/passiveOpens": [], - "sys/tcpv4/retrans/segs": [], - "sys/uptime": [], - "task/action/batch/attempts": [], - "task/action/batch/queueTime": [], - "task/action/batch/runTime": [], - "task/action/batch/size": [], - "task/action/failed/count": [], - "task/action/run/time": [], - "task/action/success/count": [], - "task/autoScaler/requiredCount": [], - "task/autoScaler/scaleActionTime": [], - "task/failed/count": [], - "task/pending/count": [], - "task/pending/time": [], - "task/run/time": [], - "task/running/count": [], - "task/segmentAvailability/wait/time": [], - "task/success/count": [], - "task/waiting/count": [], - "task/waiting/time": [], - "taskSlot/blacklisted/count": [], - "taskSlot/idle/count": [], - "taskSlot/lazy/count": [], - "taskSlot/total/count": [], - "taskSlot/used/count": [], - "tier/historical/count": [], - "tier/replication/factor": [], - "tier/required/capacity": [], - "tier/total/capacity": [], - "worker/task/assigned/count": [], - "worker/task/completed/count": [], - "worker/task/failed/count": [], - "worker/task/running/count": [], - "worker/task/success/count": [], - "worker/taskSlot/idle/count": [], - "worker/taskSlot/total/count": [], - "worker/taskSlot/used/count": [], - "zk/connected": [], - "zk/reconnect/time": [] -} diff --git a/processing/src/main/resources/defaultMetrics.json b/processing/src/main/resources/defaultMetrics.json index ea6c2fe6b5ff..4aeff3bfca97 100644 --- a/processing/src/main/resources/defaultMetrics.json +++ b/processing/src/main/resources/defaultMetrics.json @@ -1,4 +1,19 @@ { + "cgroup/cpu/cores_quota": [], + "cgroup/cpu/shares": [], + "cgroup/cpu/usage/sys/percentage": [], + "cgroup/cpu/usage/total/percentage": [], + "cgroup/cpu/usage/user/percentage": [], + "cgroup/cpuset/cpu_count": [], + "cgroup/cpuset/effective_cpu_count": [], + "cgroup/cpuset/effective_mems_count": [], + "cgroup/cpuset/mems_count": [], + "cgroup/disk/read/count": [], + "cgroup/disk/read/size": [], + "cgroup/disk/write/count": [], + "cgroup/disk/write/size": [], + "cgroup/memory/limit/bytes": [], + "cgroup/memory/usage/bytes": [], "compact/segmentAnalyzer/fetchAndProcessMillis": [], "compact/task/count": [], "compactTask/availableSlot/count": [], @@ -11,8 +26,11 @@ "groupBy/spilledBytes": [], "groupBy/spilledQueries": [], "ingest/count": [], + "ingest/events/avgMessageGap": [], "ingest/events/duplicate": [], + "ingest/events/maxMessageGap": [], "ingest/events/messageGap": [], + "ingest/events/minMessageGap": [], "ingest/events/processed": [], "ingest/events/processedWithError": [], "ingest/events/thrownAway": [], @@ -22,9 +40,18 @@ "ingest/handoff/time": [], "ingest/input/bytes": [], "ingest/kafka/avgLag": [], + "ingest/kafka/avgLag/time": [], + "ingest/kafka/fetchOffsets/time": [], "ingest/kafka/lag": [], + "ingest/kafka/lag/time": [], "ingest/kafka/maxLag": [], + "ingest/kafka/maxLag/time": [], "ingest/kafka/partitionLag": [], + "ingest/kinesis/avgLag/time": [], + "ingest/kinesis/fetchOffsets/time": [], + "ingest/kinesis/lag/time": [], + "ingest/kinesis/maxLag/time": [], + "ingest/kinesis/partitionLag/time": [], "ingest/merge/cpu": [], "ingest/merge/time": [], "ingest/notices/queueSize": [], @@ -37,6 +64,8 @@ "ingest/persists/time": [], "ingest/rows/output": [], "ingest/segments/count": [], + "ingest/shuffle/bytes": [], + "ingest/shuffle/requests": [], "ingest/sink/count": [], "ingest/tombstones/count": [], "interval/compacted/count": [], @@ -66,18 +95,6 @@ "jvm/pool/init": [], "jvm/pool/max": [], "jvm/pool/used": [], - "kafka/consumer/bytesConsumed": [], - "kafka/consumer/fetch": [], - "kafka/consumer/fetchLatencyAvg": [], - "kafka/consumer/fetchLatencyMax": [], - "kafka/consumer/fetchRate": [], - "kafka/consumer/fetchSizeAvg": [], - "kafka/consumer/fetchSizeMax": [], - "kafka/consumer/incomingBytes": [], - "kafka/consumer/outgoingBytes": [], - "kafka/consumer/recordsConsumed": [], - "kafka/consumer/recordsLag": [], - "kafka/consumer/recordsPerRequestAvg": [], "kill/eligibleUnusedSegments/count": [], "kill/pendingSegments/count": [], "kill/task/count": [], @@ -93,38 +110,12 @@ "metadata/kill/datasource/count": [], "metadata/kill/rule/count": [], "metadata/kill/supervisor/count": [], - "metadatacache/backfill/count": [], "metadatacache/init/time": [], - "metadatacache/refresh/count": [], - "metadatacache/refresh/time": [], - "metadatacache/schemaPoll/count": [], - "metadatacache/schemaPoll/failed": [], "metadatacache/schemaPoll/time": [], + "query/byteLimit/exceeded/count": [], "query/bytes": [], - "query/cache/delta/averageBytes": [], - "query/cache/delta/errors": [], - "query/cache/delta/evictions": [], - "query/cache/delta/hitRate": [], - "query/cache/delta/hits": [], - "query/cache/delta/misses": [], - "query/cache/delta/numEntries": [], - "query/cache/delta/put/error": [], - "query/cache/delta/put/ok": [], - "query/cache/delta/put/oversized": [], - "query/cache/delta/sizeBytes": [], - "query/cache/delta/timeouts": [], - "query/cache/total/averageBytes": [], - "query/cache/total/errors": [], - "query/cache/total/evictions": [], - "query/cache/total/hitRate": [], - "query/cache/total/hits": [], - "query/cache/total/misses": [], - "query/cache/total/numEntries": [], - "query/cache/total/put/error": [], - "query/cache/total/put/ok": [], - "query/cache/total/put/oversized": [], - "query/cache/total/sizeBytes": [], - "query/cache/total/timeouts": [], + "query/cache/memcached/delta": [], + "query/cache/memcached/total": [], "query/count": [], "query/cpu/time": [], "query/failed/count": [], @@ -132,17 +123,21 @@ "query/node/bytes": [], "query/node/time": [], "query/node/ttfb": [], + "query/priority": [], + "query/resultCache/hit": [], + "query/rowLimit/exceeded/count": [], "query/segment/time": [], "query/segmentAndCache/time": [], + "query/segments/count": [], "query/success/count": [], "query/time": [], "query/timeout/count": [], "query/wait/time": [], - "schemacache/finalizedSchemaPayload/count": [], - "schemacache/finalizedSegmentMetadata/count": [], - "schemacache/inTransitSMQPublishedResults/count": [], - "schemacache/inTransitSMQResults/count": [], - "schemacache/realtime/count": [], + "s3/upload/part/queueSize": [], + "s3/upload/part/queuedTime": [], + "s3/upload/part/time": [], + "s3/upload/total/bytes": [], + "s3/upload/total/time": [], "segment/added/bytes": [], "segment/assignSkipped/count": [], "segment/assigned/count": [], @@ -154,21 +149,62 @@ "segment/dropQueue/count": [], "segment/dropSkipped/count": [], "segment/dropped/count": [], + "segment/kill/jobsProcessed/count": [], + "segment/kill/queueProcess/time": [], + "segment/kill/queueReset/time": [], + "segment/kill/skippedIntervals/count": [], + "segment/kill/unusedIntervals/count": [], + "segment/killed/deepStorage/count": [], + "segment/killed/metadataStore/count": [], "segment/loadQueue/assigned": [], "segment/loadQueue/cancelled": [], "segment/loadQueue/count": [], "segment/loadQueue/failed": [], "segment/loadQueue/size": [], "segment/loadQueue/success": [], + "segment/loading/rateKbps": [], "segment/max": [], + "segment/metadataCache/dataSource/deleted": [], + "segment/metadataCache/deleted": [], + "segment/metadataCache/interval/count": [], + "segment/metadataCache/pending/count": [], + "segment/metadataCache/pending/deleted": [], + "segment/metadataCache/pending/skipped": [], + "segment/metadataCache/pending/updated": [], + "segment/metadataCache/skipped": [], + "segment/metadataCache/sync/time": [], + "segment/metadataCache/transactions/readOnly": [], + "segment/metadataCache/transactions/readWrite": [], + "segment/metadataCache/transactions/writeOnly": [], + "segment/metadataCache/used/count": [], + "segment/metadataCache/used/stale": [], + "segment/metadataCache/used/updated": [], "segment/moveSkipped/count": [], "segment/moved/bytes": [], "segment/moved/count": [], "segment/nuked/bytes": [], "segment/overShadowed/count": [], + "segment/pending/count": [], "segment/pendingDelete": [], + "segment/rowCount/avg": [], + "segment/rowCount/range/count": [], "segment/scan/active": [], "segment/scan/pending": [], + "segment/schemaCache/backfill/count": [], + "segment/schemaCache/dataSource/removed": [], + "segment/schemaCache/deepStorageOnly/count": [], + "segment/schemaCache/deepStorageOnly/refresh/time": [], + "segment/schemaCache/pendingBackfill/count": [], + "segment/schemaCache/poll/count": [], + "segment/schemaCache/poll/failed": [], + "segment/schemaCache/realtime/count": [], + "segment/schemaCache/refresh/count": [], + "segment/schemaCache/refresh/time": [], + "segment/schemaCache/refreshSkipped/count": [], + "segment/schemaCache/rowSignature/changed": [], + "segment/schemaCache/rowSignature/column/count": [], + "segment/schemaCache/used/count": [], + "segment/schemaCache/usedFingerprint/count": [], "segment/size": [], "segment/skipCompact/bytes": [], "segment/skipCompact/count": [], @@ -177,16 +213,24 @@ "segment/unneeded/count": [], "segment/unneededEternityTombstone/count": [], "segment/used": [], + "segment/used/count": [], + "segment/used/deepStorageOnly/count": [], "segment/usedPercent": [], "segment/waitCompact/bytes": [], "segment/waitCompact/count": [], "serverview/init/time": [], "serverview/sync/healthy": [], "serverview/sync/unstableTime": [], - "service/heartbeat": [], "sqlQuery/bytes": [], "sqlQuery/planningTimeMs": [], "sqlQuery/time": [], + "subquery/byteLimit/count": [], + "subquery/bytes": [], + "subquery/fallback/count": [], + "subquery/fallback/insufficientType/count": [], + "subquery/fallback/unknownReason/count": [], + "subquery/rowLimit/count": [], + "subquery/rows": [], "sys/cpu": [], "sys/disk/queue": [], "sys/disk/read/count": [], @@ -232,6 +276,7 @@ "task/action/run/time": [], "task/action/success/count": [], "task/autoScaler/requiredCount": [], + "task/autoScaler/scaleActionTime": [], "task/failed/count": [], "task/pending/count": [], "task/pending/time": [], @@ -240,10 +285,24 @@ "task/segmentAvailability/wait/time": [], "task/success/count": [], "task/waiting/count": [], + "task/waiting/time": [], + "taskSlot/blacklisted/count": [], + "taskSlot/idle/count": [], + "taskSlot/lazy/count": [], + "taskSlot/total/count": [], + "taskSlot/used/count": [], "tier/historical/count": [], "tier/replication/factor": [], "tier/required/capacity": [], "tier/total/capacity": [], + "worker/task/assigned/count": [], + "worker/task/completed/count": [], + "worker/task/failed/count": [], + "worker/task/running/count": [], + "worker/task/success/count": [], + "worker/taskSlot/idle/count": [], + "worker/taskSlot/total/count": [], + "worker/taskSlot/used/count": [], "zk/connected": [], "zk/reconnect/time": [] -} \ No newline at end of file +} diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfigTest.java index ebc213303db4..46e1e42699fb 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfigTest.java @@ -136,6 +136,21 @@ public void testSettingEverythingLegacy() Assert.assertEquals(100, config.getMinHttpTimeoutMillis()); } + @Test + public void testSettingMetricFilteringLegacy() + { + final Properties props = new Properties(); + props.setProperty("org.apache.druid.java.util.emitter.http.url", "http://example.com/"); + props.setProperty("org.apache.druid.java.util.emitter.shouldFilterMetrics", "true"); + props.setProperty("org.apache.druid.java.util.emitter.metricSpecPath", "/tmp/http-metrics.json"); + + final ObjectMapper objectMapper = new ObjectMapper(); + final HttpEmitterConfig config = objectMapper.convertValue(Emitters.makeHttpMap(props), HttpEmitterConfig.class); + + Assert.assertTrue(config.isShouldFilterMetrics()); + Assert.assertEquals("/tmp/http-metrics.json", config.getMetricSpecPath().orElse(null)); + } + @Test public void testMemoryLimits() { diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpPostEmitterTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpPostEmitterTest.java index 987de0b3c5e7..f8369176c635 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpPostEmitterTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpPostEmitterTest.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.primitives.Ints; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.java.util.emitter.service.UnitEvent; import org.asynchttpclient.ListenableFuture; import org.asynchttpclient.Request; import org.asynchttpclient.Response; @@ -30,11 +32,13 @@ import java.io.IOException; import java.lang.reflect.Field; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.atomic.AtomicReference; public class HttpPostEmitterTest { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() { @Override @@ -95,4 +99,45 @@ public void testRecoveryEmitAndReturnBatch() Assert.assertEquals(2, emitter.getTotalEmittedEvents()); } } + + @Test + public void testFilteringAllowsConfiguredMetricAndNonMetricEvent() throws Exception + { + final HttpEmitterConfig.Builder builder = new HttpEmitterConfig.Builder("http://foo.bar") + .setFlushMillis(100) + .setFlushCount(4) + .setFlushTimeout(BaseHttpEmittingConfig.TEST_FLUSH_TIMEOUT_MILLIS) + .setBatchingStrategy(BatchingStrategy.ONLY_EVENTS) + .setMaxBatchSize(1024 * 1024) + .setBatchQueueSizeLimit(1000); + builder.setShouldFilterMetrics(true); + + final HttpEmitterConfig config = builder.build(); + final List payloads = new ArrayList<>(); + httpClient.setGoHandler( + new GoHandler() + { + @Override + protected ListenableFuture go(final Request request) + { + payloads.add(StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString()); + return GoHandlers.immediateFuture(EmitterTest.okResponse()); + } + } + ); + + try (final HttpPostEmitter emitter = new HttpPostEmitter(config, httpClient, new ObjectMapper())) { + emitter.start(); + emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("some/unlisted/metric", 200).build("test", "localhost")); + emitter.emit(new UnitEvent("alerts", 1)); + emitter.flush(); + + final String payload = String.join("", payloads); + Assert.assertTrue(payload.contains("\"metric\":\"query/time\"")); + Assert.assertTrue(payload.contains("\"feed\":\"alerts\"")); + Assert.assertFalse(payload.contains("\"metric\":\"some/unlisted/metric\"")); + Assert.assertEquals(2, emitter.getTotalEmittedEvents()); + } + } } diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java index 34c353998573..e266572c1a7e 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterAllowlistConfigTest.java @@ -40,8 +40,8 @@ public void testUsesDefaultAllowlistWhenCustomPathIsNotProvided() final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); Assert.assertNotNull(emitter); - Assert.assertTrue(emitter.getMetricNames().contains("jvm/gc/cpu")); - Assert.assertTrue(emitter.getMetricNames().contains("query/time")); + Assert.assertTrue(emitter.getAllowedMetricNames().contains("jvm/gc/cpu")); + Assert.assertTrue(emitter.getAllowedMetricNames().contains("query/time")); } @Test @@ -57,9 +57,9 @@ public void testReadsCustomAllowlistAsMetricObject() throws IOException final LoggingEmitter emitter = new LoggingEmitter(config, new ObjectMapper()); Assert.assertNotNull(emitter); - Assert.assertEquals(2, emitter.getMetricNames().size()); - Assert.assertTrue(emitter.getMetricNames().contains("jvm/gc/cpu")); - Assert.assertTrue(emitter.getMetricNames().contains("jvm/gc/count")); - Assert.assertFalse(emitter.getMetricNames().contains("query/time")); + Assert.assertEquals(2, emitter.getAllowedMetricNames().size()); + Assert.assertTrue(emitter.getAllowedMetricNames().contains("jvm/gc/cpu")); + Assert.assertTrue(emitter.getAllowedMetricNames().contains("jvm/gc/count")); + Assert.assertFalse(emitter.getAllowedMetricNames().contains("query/time")); } } diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsersTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsersTest.java new file mode 100644 index 000000000000..eea9300e6a4f --- /dev/null +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/MetricAllowlistParsersTest.java @@ -0,0 +1,63 @@ +/* + * 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.java.util.emitter.core; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.error.DruidException; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Set; + +public class MetricAllowlistParsersTest +{ + private static final ObjectMapper MAPPER = new ObjectMapper(); + + @Test + public void testParseMetricNameObject() + throws Exception + { + final JsonNode jsonNode = MAPPER.readTree("{\"query/time\":[],\"jvm/gc/cpu\":[]}"); + + final Set metricNames = MetricAllowlistParsers.parseMetricNameObject(jsonNode, "test-source"); + + Assert.assertEquals(2, metricNames.size()); + Assert.assertTrue(metricNames.contains("query/time")); + Assert.assertTrue(metricNames.contains("jvm/gc/cpu")); + } + + @Test + public void testParseMetricNameObjectRequiresJsonObject() + throws Exception + { + final JsonNode jsonNode = MAPPER.readTree("[\"query/time\"]"); + + final DruidException exception = Assert.assertThrows( + DruidException.class, + () -> MetricAllowlistParsers.parseMetricNameObject(jsonNode, "test-source") + ); + + Assert.assertEquals( + "Metric allowlist file [test-source] must be a JSON object of metric names.", + exception.getMessage() + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterTest.java index c5dc26432fe0..b41e5faee0cf 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterTest.java @@ -20,10 +20,12 @@ package org.apache.druid.java.util.emitter.core; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.emitter.service.UnitEvent; import org.asynchttpclient.ListenableFuture; import org.asynchttpclient.Request; @@ -62,6 +64,11 @@ public void tearDown() } private Emitter parametrizedEmmiter(String uriPattern) throws Exception + { + return parametrizedEmmiter(uriPattern, ImmutableMap.of()); + } + + private Emitter parametrizedEmmiter(String uriPattern, Map extraProps) throws Exception { final Properties props = new Properties(); props.setProperty("org.apache.druid.java.util.emitter.type", "parametrized"); @@ -70,6 +77,9 @@ private Emitter parametrizedEmmiter(String uriPattern) throws Exception "org.apache.druid.java.util.emitter.httpEmitting.flushTimeOut", String.valueOf(BaseHttpEmittingConfig.TEST_FLUSH_TIMEOUT_MILLIS) ); + for (Map.Entry entry : extraProps.entrySet()) { + props.setProperty(entry.getKey(), entry.getValue()); + } lifecycle = new Lifecycle(); Emitter emitter = Emitters.create(props, httpClient, lifecycle); Assert.assertEquals(ParametrizedUriEmitter.class, emitter.getClass()); @@ -219,4 +229,43 @@ public void failEmitMalformedEvent() throws Exception ); } } + + @Test + public void testFilteringAllowsConfiguredMetricAndNonMetricEvent() throws Exception + { + final Emitter emitter = parametrizedEmmiter( + "http://example.com/{feed}", + ImmutableMap.of( + "org.apache.druid.java.util.emitter.shouldFilterMetrics", "true" + ) + ); + + final Map requests = new HashMap<>(); + httpClient.setGoHandler( + new GoHandler() + { + @Override + protected ListenableFuture go(final Request request) throws Exception + { + requests.put( + request.getUrl(), + JSON_MAPPER.readTree(StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString()) + ); + return GoHandlers.immediateFuture(EmitterTest.okResponse()); + } + }.times(2) + ); + + emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("some/unlisted/metric", 200).build("test", "localhost")); + emitter.emit(new UnitEvent("alerts", 1)); + emitter.flush(); + + Assert.assertTrue(httpClient.succeeded()); + final JsonNode metricsPayload = requests.get("http://example.com/metrics"); + Assert.assertNotNull(metricsPayload); + Assert.assertEquals(1, metricsPayload.size()); + Assert.assertEquals("query/time", metricsPayload.get(0).get("metric").asText()); + Assert.assertNotNull(requests.get("http://example.com/alerts")); + } } From 91567b70b1af8369bb508f5fd9c945487bd87b40 Mon Sep 17 00:00:00 2001 From: Misha Date: Tue, 10 Mar 2026 17:30:09 +0100 Subject: [PATCH 09/12] cleanup --- .../druid/emitter/prometheus/MetricsTest.java | 19 ++----------------- .../util/emitter/core/HttpEmitterConfig.java | 2 +- 2 files changed, 3 insertions(+), 18 deletions(-) diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java index 489901bb34db..572aceb884a3 100644 --- a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java @@ -25,9 +25,6 @@ import org.junit.Assert; import org.junit.Test; -import java.net.URISyntaxException; -import java.nio.file.Path; -import java.nio.file.Paths; import java.util.HashMap; import java.util.Map; @@ -120,7 +117,7 @@ public void testMetricsConfigurationWithUnSupportedType() PrometheusEmitterConfig config = new PrometheusEmitterConfig( null, "test_5", - resolveResourcePath("defaultInvalidMetricsTest.json"), + "src/test/resources/defaultInvalidMetricsTest.json", null, null, true, @@ -142,7 +139,7 @@ public void testMetricsConfigurationWithTimerHistogramBuckets() PrometheusEmitterConfig config = new PrometheusEmitterConfig( null, "test_6", - resolveResourcePath("defaultMetricsTest.json"), + "src/test/resources/defaultMetricsTest.json", null, null, true, @@ -164,16 +161,4 @@ public void testMetricsConfigurationWithTimerHistogramBuckets() double[] expectedHistogramBuckets = {10.0, 30.0, 60.0, 120.0, 200.0, 300.0}; Assert.assertArrayEquals(expectedHistogramBuckets, dimensionsAndCollector.getHistogramBuckets(), 0.0); } - - private static String resolveResourcePath(final String resourceName) - { - try { - final Path resourcePath = Paths.get(MetricsTest.class.getClassLoader().getResource(resourceName).toURI()); - return resourcePath.toString(); - } - catch (URISyntaxException e) { - throw new RuntimeException(e); - } - } - } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java index e5d326d6d3ce..43446288ba01 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfig.java @@ -53,7 +53,7 @@ public HttpEmitterConfig(BaseHttpEmittingConfig base, String recipientBaseUrl) this.httpTimeoutAllowanceFactor = base.httpTimeoutAllowanceFactor; this.minHttpTimeoutMillis = base.minHttpTimeoutMillis; this.setShouldFilterMetrics(base.isShouldFilterMetrics()); - base.getMetricSpecPath().ifPresent(this::setMetricSpecPath); + this.setMetricSpecPath(base.getMetricSpecPath().orElse(null)); } public String getRecipientBaseUrl() From 9a75e8d49432158d5d9e011951a4a97cc52fd252 Mon Sep 17 00:00:00 2001 From: Misha Date: Wed, 11 Mar 2026 11:09:51 +0100 Subject: [PATCH 10/12] removed redundand event param from preEmit --- .../java/util/emitter/core/AbstractFilteringEmitter.java | 4 ++-- .../apache/druid/java/util/emitter/core/LoggingEmitter.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/AbstractFilteringEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/AbstractFilteringEmitter.java index f2cf4fadd4b1..3534095c5ab5 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/AbstractFilteringEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/AbstractFilteringEmitter.java @@ -61,7 +61,7 @@ protected static Set loadAllowedMetricNames( @Override public final void emit(final Event event) { - preEmit(event); + preEmit(); if (shouldFilterMetrics && shouldFilterEvent(event)) { return; } @@ -94,7 +94,7 @@ protected Set getAllowedMetricNames() /** * Hook for pre-emission checks that must run before metric filtering. */ - protected void preEmit(final Event event) + protected void preEmit() { // default no-op } diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java index 1267088e4028..466431525f3c 100644 --- a/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java +++ b/processing/src/main/java/org/apache/druid/java/util/emitter/core/LoggingEmitter.java @@ -128,7 +128,7 @@ protected boolean shouldFilterEvent(final Event event) } @Override - protected void preEmit(final Event event) + protected void preEmit() { synchronized (started) { if (!started.get()) { From c41f754ef7ac5f8b8a9b0571fa7748b7a40cb258 Mon Sep 17 00:00:00 2001 From: Misha Date: Wed, 11 Mar 2026 11:17:17 +0100 Subject: [PATCH 11/12] fix test --- .../util/emitter/core/LoggingEmitterTest.java | 89 ++++++++----------- 1 file changed, 35 insertions(+), 54 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java index b29cf44ef41b..23d55d4b9b8b 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/LoggingEmitterTest.java @@ -21,10 +21,10 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.emitter.service.UnitEvent; -import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -47,7 +47,6 @@ public class LoggingEmitterTest private List serializedObjects; private ObjectMapper trackingMapper; - private LoggingEmitter emitter; @Before public void setUp() @@ -71,7 +70,7 @@ public String writeValueAsString(Object value) throws JsonProcessingException private LoggingEmitter createEmitter(boolean shouldFilterMetrics, String allowedMetricsPath) { - emitter = new LoggingEmitter( + final LoggingEmitter emitter = new LoggingEmitter( new Logger(LoggingEmitter.class), LoggingEmitter.Level.WARN, trackingMapper, @@ -82,28 +81,19 @@ private LoggingEmitter createEmitter(boolean shouldFilterMetrics, String allowed return emitter; } - @After - public void tearDown() - { - if (emitter != null) { - emitter.close(); - emitter = null; - } - } - /** * Without filtering enabled, the emitter should log all events (backward compatibility). */ @Test public void testEmitAllWhenFilteringDisabled() { - createEmitter(false, null); - - emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); - emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/mem/used", 512).build("test", "localhost")); - emitter.emit(ServiceMetricEvent.builder().setMetric("some/random/metric", 1).build("test", "localhost")); + try (LoggingEmitter emitter = createEmitter(false, null)) { + emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/mem/used", 512).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("some/random/metric", 1).build("test", "localhost")); - Assert.assertEquals("All events should be serialized (logged)", 3, serializedObjects.size()); + Assert.assertEquals("All events should be serialized (logged)", 3, serializedObjects.size()); + } } /** @@ -114,14 +104,12 @@ public void testEmitAllWhenFilteringDisabled() @Test public void testFilterWithDefaultResource() { - createEmitter(true, null); - - // "query/time" is in the default allowed metrics list - emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); - // "some/unlisted/metric" is NOT in the default list - emitter.emit(ServiceMetricEvent.builder().setMetric("some/unlisted/metric", 1).build("test", "localhost")); + try (LoggingEmitter emitter = createEmitter(true, null)) { + emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("some/unlisted/metric", 1).build("test", "localhost")); - Assert.assertEquals("Only the allowed metric should be serialized", 1, serializedObjects.size()); + Assert.assertEquals("Only the allowed metric should be serialized", 1, serializedObjects.size()); + } } /** @@ -131,13 +119,13 @@ public void testFilterWithDefaultResource() public void testFilterWithCustomFilePath() throws IOException { final File allowFile = createAllowlistFile("{\"query/time\": [], \"query/bytes\": []}"); - createEmitter(true, allowFile.getAbsolutePath()); - - emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); - emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/mem/used", 512).build("test", "localhost")); - emitter.emit(ServiceMetricEvent.builder().setMetric("query/bytes", 2048).build("test", "localhost")); + try (LoggingEmitter emitter = createEmitter(true, allowFile.getAbsolutePath())) { + emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/mem/used", 512).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("query/bytes", 2048).build("test", "localhost")); - Assert.assertEquals("Only allowed metrics should be serialized", 2, serializedObjects.size()); + Assert.assertEquals("Only allowed metrics should be serialized", 2, serializedObjects.size()); + } } /** @@ -148,12 +136,11 @@ public void testFilterWithCustomFilePath() throws IOException public void testNonMetricEventsAlwaysPassThrough() throws IOException { final File allowFile = createAllowlistFile("{\"query/time\": []}"); - createEmitter(true, allowFile.getAbsolutePath()); - - // This is NOT a ServiceMetricEvent, so it should bypass the allowlist filter - emitter.emit(new UnitEvent("alerts", 42)); + try (LoggingEmitter emitter = createEmitter(true, allowFile.getAbsolutePath())) { + emitter.emit(new UnitEvent("alerts", 42)); - Assert.assertEquals("Non-metric events should bypass the allowlist filter", 1, serializedObjects.size()); + Assert.assertEquals("Non-metric events should bypass the allowlist filter", 1, serializedObjects.size()); + } } /** @@ -161,15 +148,9 @@ public void testNonMetricEventsAlwaysPassThrough() throws IOException * to the default classpath resource and emits successfully. */ @Test - public void testMissingCustomPathFallsBackToDefault() + public void testMissingCustomPathThrows() { - createEmitter(true, "/nonexistent/path/to/allowedMetrics.json"); - - // Fallback to default should allow "query/time" (in default list) and drop unlisted metrics - emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); - emitter.emit(ServiceMetricEvent.builder().setMetric("some/unlisted/metric", 1).build("test", "localhost")); - - Assert.assertEquals("Fallback to default should allow listed metrics only", 1, serializedObjects.size()); + Assert.assertThrows(DruidException.class, () -> createEmitter(true, "/nonexistent/path/to/allowedMetrics.json")); } /** @@ -179,12 +160,12 @@ public void testMissingCustomPathFallsBackToDefault() public void testEmptyAllowlistBlocksAllMetrics() throws IOException { final File allowFile = createAllowlistFile("{}"); - createEmitter(true, allowFile.getAbsolutePath()); + try (LoggingEmitter emitter = createEmitter(true, allowFile.getAbsolutePath())) { + emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); + emitter.emit(new UnitEvent("alerts", 42)); - emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); - emitter.emit(new UnitEvent("alerts", 42)); - - Assert.assertEquals("Only non-metric event should pass through", 1, serializedObjects.size()); + Assert.assertEquals("Only non-metric event should pass through", 1, serializedObjects.size()); + } } /** @@ -194,12 +175,12 @@ public void testEmptyAllowlistBlocksAllMetrics() throws IOException public void testFilterDisabledIgnoresPath() throws IOException { final File allowFile = createAllowlistFile("{\"query/time\": []}"); - createEmitter(false, allowFile.getAbsolutePath()); + try (LoggingEmitter emitter = createEmitter(false, allowFile.getAbsolutePath())) { + emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); + emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/mem/used", 512).build("test", "localhost")); - emitter.emit(ServiceMetricEvent.builder().setMetric("query/time", 100).build("test", "localhost")); - emitter.emit(ServiceMetricEvent.builder().setMetric("jvm/mem/used", 512).build("test", "localhost")); - - Assert.assertEquals("All events should pass when filtering is disabled", 2, serializedObjects.size()); + Assert.assertEquals("All events should pass when filtering is disabled", 2, serializedObjects.size()); + } } private File createAllowlistFile(String jsonContent) throws IOException From 0631abadc192b18b7dcd5cf65b96457aeab8d856 Mon Sep 17 00:00:00 2001 From: Misha Date: Wed, 11 Mar 2026 13:06:48 +0100 Subject: [PATCH 12/12] fix resources --- processing/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/processing/pom.xml b/processing/pom.xml index 3bf60b04387d..38c507ba85b4 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -533,6 +533,9 @@ src/main/resources + + defaultMetrics.json +