From aae2554b3fbb809ca7afbd90ee89dcf0ad4a8565 Mon Sep 17 00:00:00 2001 From: Gianluca Graziadei Date: Sun, 3 May 2026 15:38:03 +0200 Subject: [PATCH 1/8] init --- conf/defaults.yaml | 47 +- .../src/jvm/org/apache/storm/Config.java | 115 ++-- .../org/apache/storm/metrics2/EWMAGauge.java | 72 +++ .../apache/storm/metrics2/TaskMetrics.java | 83 ++- .../org/apache/storm/utils/ConfigUtils.java | 24 + .../storm/validation/ConfigValidation.java | 17 + .../org/apache/storm/TestConfigValidate.java | 26 + .../apache/storm/metrics2/EWMAGaugeTest.java | 346 +++++++++++++ .../storm/metrics2/TaskMetricsTest.java | 490 ++++++++++++++++++ 9 files changed, 1145 insertions(+), 75 deletions(-) create mode 100644 storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java create mode 100644 storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java create mode 100644 storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 4368099725c..4b0313d1d6f 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -25,7 +25,7 @@ java.library.path: "/usr/local/lib:/opt/local/lib:/usr/lib:/usr/lib64" storm.local.dir: "storm-local" storm.log4j2.conf.dir: "log4j2" storm.zookeeper.servers: - - "localhost" + - "localhost" storm.zookeeper.port: 2181 storm.zookeeper.root: "/storm" storm.zookeeper.session.timeout: 20000 @@ -52,7 +52,7 @@ storm.nimbus.retry.intervalceiling.millis: 60000 storm.nimbus.zookeeper.acls.check: true storm.nimbus.zookeeper.acls.fixup: true -storm.auth.simple-white-list.users: [] +storm.auth.simple-white-list.users: [ ] storm.cluster.state.store: "org.apache.storm.cluster.ZKStateStorageFactory" storm.meta.serialization.delegate: "org.apache.storm.serialization.GzipThriftSerializationDelegate" storm.codedistributor.class: "org.apache.storm.codedistributor.LocalFileSystemCodeDistributor" @@ -62,7 +62,7 @@ storm.health.check.timeout.ms: 5000 storm.disable.symlinks: false ### nimbus.* configs are for the master -nimbus.seeds : ["localhost"] +nimbus.seeds: [ "localhost" ] nimbus.thrift.port: 6627 nimbus.thrift.threads: 64 nimbus.thrift.max_buffer_size: 1048576 @@ -163,10 +163,10 @@ storm.blobstore.acl.validation.enabled: false ### supervisor.* configs are for node supervisors # Define the amount of workers that can be run on this machine. Each worker is assigned a port to use for communication supervisor.slots.ports: - - 6700 - - 6701 - - 6702 - - 6703 + - 6700 + - 6701 + - 6702 + - 6703 supervisor.childopts: "-Xmx256m" supervisor.run.worker.as.user: false #how long supervisor will wait to ensure that a worker process is started @@ -184,8 +184,8 @@ supervisor.worker.heartbeats.max.timeout.secs: 600 #For topology configurable heartbeat timeout, maximum allowed heartbeat timeout. worker.max.timeout.secs: 600 supervisor.enable: true -supervisor.supervisors: [] -supervisor.supervisors.commands: [] +supervisor.supervisors: [ ] +supervisor.supervisors.commands: [ ] supervisor.memory.capacity.mb: 4096.0 #By convention 1 cpu core should be about 100, but this can be adjusted if needed # using 100 makes it simple to set the desired value to the capacity measurement @@ -278,6 +278,7 @@ topology.max.task.parallelism: null topology.max.spout.pending: null # ideally should be larger than topology.producer.batch.size. (esp. if topology.batch.flush.interval.millis=0) topology.state.synchronization.timeout.secs: 60 topology.stats.sample.rate: 0.05 +topology.stats.ewma.enable: false topology.builtin.metrics.bucket.size.secs: 60 topology.fall.back.on.java.serialization: false topology.worker.childopts: null @@ -287,16 +288,16 @@ topology.worker.shared.thread.pool.size: 4 # Spout Wait Strategy - employed when there is no data to produce topology.spout.wait.strategy: "org.apache.storm.policy.WaitStrategyProgressive" -topology.spout.wait.park.microsec : 100 # park time for org.apache.storm.policy.WaitStrategyPark. Busy spins if set to 0. +topology.spout.wait.park.microsec: 100 # park time for org.apache.storm.policy.WaitStrategyPark. Busy spins if set to 0. topology.spout.wait.progressive.level1.count: 0 # number of iterations to spend in level 1 [no sleep] of WaitStrategyProgressive, before progressing to level 2 topology.spout.wait.progressive.level2.count: 0 # number of iterations to spend in level 2 [parkNanos(1)] of WaitStrategyProgressive, before progressing to level 3 topology.spout.wait.progressive.level3.sleep.millis: 1 # sleep duration for idling iterations in level 3 of WaitStrategyProgressive # Bolt Wait Strategy - employed when there is no data in its receive buffer to process -topology.bolt.wait.strategy : "org.apache.storm.policy.WaitStrategyProgressive" +topology.bolt.wait.strategy: "org.apache.storm.policy.WaitStrategyProgressive" -topology.bolt.wait.park.microsec : 100 # park time for org.apache.storm.policy.WaitStrategyPark. Busy spins if set to 0. +topology.bolt.wait.park.microsec: 100 # park time for org.apache.storm.policy.WaitStrategyPark. Busy spins if set to 0. topology.bolt.wait.progressive.level1.count: 1 # number of iterations to spend in level 1 [no sleep] of WaitStrategyProgressive, before progressing to level 2 topology.bolt.wait.progressive.level2.count: 1000 # number of iterations to spend in level 2 [parkNanos(1)] of WaitStrategyProgressive, before progressing to level 3 @@ -363,7 +364,7 @@ blacklist.scheduler.assume.supervisor.bad.based.on.bad.slot: true dev.zookeeper.path: "/tmp/dev-storm-zookeeper" -pacemaker.servers: [] +pacemaker.servers: [ ] pacemaker.port: 6699 pacemaker.base.threads: 10 pacemaker.max.threads: 50 @@ -371,12 +372,12 @@ pacemaker.client.max.threads: 2 pacemaker.thread.timeout: 10 pacemaker.childopts: "-Xmx1024m" pacemaker.auth.method: "NONE" -pacemaker.kerberos.users: [] +pacemaker.kerberos.users: [ ] pacemaker.thrift.message.size.max: 10485760 #default storm daemon metrics reporter plugins storm.daemon.metrics.reporter.plugins: - - "org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter" + - "org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter" storm.daemon.metrics.reporter.interval.secs: 10 storm.metricstore.class: "org.apache.storm.metricstore.rocksdb.RocksDbStore" @@ -399,8 +400,8 @@ storm.cgroup.inherit.cpuset.configs: false # Configs for CGroup support storm.cgroup.hierarchy.dir: "/cgroup/storm_resources" storm.cgroup.resources: - - "cpu" - - "memory" + - "cpu" + - "memory" storm.cgroup.hierarchy.name: "storm" storm.supervisor.cgroup.rootdir: "storm" storm.cgroup.cgexec.cmd: "/bin/cgexec" @@ -419,12 +420,12 @@ storm.worker.min.cpu.pcore.percent: 0.0 storm.topology.classpath.beginning.enabled: false worker.metrics: - "CGroupMemory": "org.apache.storm.metrics2.cgroup.CGroupMemoryUsage" - "CGroupMemoryLimit": "org.apache.storm.metrics2.cgroup.CGroupMemoryLimit" - "CGroupCpu": "org.apache.storm.metrics2.cgroup.CGroupCpu" - "CGroupCpuGuarantee": "org.apache.storm.metrics2.cgroup.CGroupCpuGuarantee" - "CGroupCpuGuaranteeByCfsQuota": "org.apache.storm.metrics2.cgroup.CGroupCpuGuaranteeByCfsQuota" - "CGroupCpuStat": "org.apache.storm.metrics2.cgroup.CGroupCpuStat" + "CGroupMemory": "org.apache.storm.metrics2.cgroup.CGroupMemoryUsage" + "CGroupMemoryLimit": "org.apache.storm.metrics2.cgroup.CGroupMemoryLimit" + "CGroupCpu": "org.apache.storm.metrics2.cgroup.CGroupCpu" + "CGroupCpuGuarantee": "org.apache.storm.metrics2.cgroup.CGroupCpuGuarantee" + "CGroupCpuGuaranteeByCfsQuota": "org.apache.storm.metrics2.cgroup.CGroupCpuGuaranteeByCfsQuota" + "CGroupCpuStat": "org.apache.storm.metrics2.cgroup.CGroupCpuStat" # The number of buckets for running statistics num.stat.buckets: 20 diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java index e332b726b28..09b0790475b 100644 --- a/storm-client/src/jvm/org/apache/storm/Config.java +++ b/storm-client/src/jvm/org/apache/storm/Config.java @@ -19,12 +19,14 @@ package org.apache.storm; import com.esotericsoftware.kryo.Serializer; + import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; + import org.apache.storm.metric.IEventLogger; import org.apache.storm.policy.IWaitStrategy; import org.apache.storm.serialization.IKryoDecorator; @@ -255,14 +257,14 @@ public class Config extends HashMap { */ @IsPositiveNumber(includeZero = true) public static final String TOPOLOGY_METRICS_CONSUMER_RESOURCES_ONHEAP_MEMORY_MB = - "topology.metrics.consumer.resources.onheap.memory.mb"; + "topology.metrics.consumer.resources.onheap.memory.mb"; /** * The maximum amount of memory an instance of a metrics consumer will take off heap. This enables the scheduler to allocate slots on * machines with enough available memory. A default value will be set for this config if user does not override */ @IsPositiveNumber(includeZero = true) public static final String TOPOLOGY_METRICS_CONSUMER_RESOURCES_OFFHEAP_MEMORY_MB = - "topology.metrics.consumer.resources.offheap.memory.mb"; + "topology.metrics.consumer.resources.offheap.memory.mb"; /** * The config indicates the percentage of cpu for a core an instance(executor) of a metrics consumer will use. Assuming the a core value * to be 100, a value of 10 indicates 10% of the core. The P in PCORE represents the term "physical". A default value will be set for @@ -348,15 +350,16 @@ public class Config extends HashMap { *

* *

comp-1 cannot exist on same worker as comp-2 or comp-3, and at most "2" comp-1 on same node

+ * *

comp-2 and comp-4 cannot be on same worker (missing comp-1 is implied from comp-1 constraint)

* - *

- * { "comp-1": { "maxNodeCoLocationCnt": 2, "incompatibleComponents": ["comp-2", "comp-3" ] }, - * "comp-2": { "incompatibleComponents": [ "comp-4" ] } - * } - *

+ *

+ * { "comp-1": { "maxNodeCoLocationCnt": 2, "incompatibleComponents": ["comp-2", "comp-3" ] }, + * "comp-2": { "incompatibleComponents": [ "comp-4" ] } + * } + *

*/ - @IsExactlyOneOf(valueValidatorClasses = { ListOfListOfStringValidator.class, RasConstraintsTypeValidator.class }) + @IsExactlyOneOf(valueValidatorClasses = {ListOfListOfStringValidator.class, RasConstraintsTypeValidator.class}) public static final String TOPOLOGY_RAS_CONSTRAINTS = "topology.ras.constraints"; /** @@ -424,17 +427,17 @@ public class Config extends HashMap { * *

* 1. If not setting this variable or setting it as null, - * a. If RAS is not used: - * Nimbus will set it to {@link Config#TOPOLOGY_WORKERS}. - * b. If RAS is used: - * Nimbus will set it to (the estimate number of workers * {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER}). - * {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER} is default to be 1 if not set. + * a. If RAS is not used: + * Nimbus will set it to {@link Config#TOPOLOGY_WORKERS}. + * b. If RAS is used: + * Nimbus will set it to (the estimate number of workers * {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER}). + * {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER} is default to be 1 if not set. * 2. If this variable is set to 0, - * then Storm will immediately ack tuples as soon as they come off the spout, - * effectively disabling reliability. + * then Storm will immediately ack tuples as soon as they come off the spout, + * effectively disabling reliability. * 3. If this variable is set to a positive integer, - * Storm will not honor {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER} setting. - * Instead, nimbus will set it as (this variable / estimate num of workers). + * Storm will not honor {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER} setting. + * Instead, nimbus will set it as (this variable / estimate num of workers). *

*/ @IsInteger @@ -448,7 +451,7 @@ public class Config extends HashMap { * This setting is RAS specific. * If {@link Config#TOPOLOGY_ACKER_EXECUTORS} is not configured, * this setting will be used to calculate {@link Config#TOPOLOGY_ACKER_EXECUTORS}. - * + *

* If {@link Config#TOPOLOGY_ACKER_EXECUTORS} is configured, * nimbus will ignore this and set it as ({@link Config#TOPOLOGY_ACKER_EXECUTORS} / estimate num of workers). *

@@ -465,7 +468,7 @@ public class Config extends HashMap { *

Note that EventLoggerBolt takes care of all the implementations of IEventLogger, hence registering many * implementations (especially they're implemented as 'blocking' manner) would slow down overall topology. */ - @IsListEntryCustom(entryValidatorClasses = { EventLoggerRegistryValidator.class }) + @IsListEntryCustom(entryValidatorClasses = {EventLoggerRegistryValidator.class}) public static final String TOPOLOGY_EVENT_LOGGER_REGISTER = "topology.event.logger.register"; /** * How many executors to spawn for event logger. @@ -543,7 +546,7 @@ public class Config extends HashMap { * it's parallelism is configurable. */ - @IsListEntryCustom(entryValidatorClasses = { MetricRegistryValidator.class }) + @IsListEntryCustom(entryValidatorClasses = {MetricRegistryValidator.class}) public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register"; /** * Enable tracking of network message byte counts per source-destination task. This is off by default as it creates tasks^2 metric @@ -596,6 +599,20 @@ public class Config extends HashMap { */ @IsPositiveNumber public static final String TOPOLOGY_STATS_SAMPLE_RATE = "topology.stats.sample.rate"; + /** + * Enabling jitter streaming calculation (RFC 1889a). + * + * @see RFC 1889 Appendix A.8 + */ + @IsBoolean + public static final String TOPOLOGY_STATS_EWMA_ENABLE = "topology.stats.ewma.enable"; + /** + * The smoothing factor (alpha) used for exponential jitter calculation (RFC 1889a). + * + * @see RFC 1889 Appendix A.8 + */ + @CustomValidator(validatorClass = ConfigValidation.EWMASmoothingFactorValidator.class) + public static final String TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR = "topology.stats.ewma.smoothing_factor"; /** * The time period that builtin metrics data in bucketed into. */ @@ -833,14 +850,14 @@ public class Config extends HashMap { * Topology central logging sensitivity to determine who has access to logs in central logging system. The possible values are: S0 - * Public (open to all users on grid) S1 - Restricted S2 - Confidential S3 - Secret (default.) */ - @IsString(acceptedValues = { "S0", "S1", "S2", "S3" }) + @IsString(acceptedValues = {"S0", "S1", "S2", "S3"}) public static final String TOPOLOGY_LOGGING_SENSITIVITY = "topology.logging.sensitivity"; /** * Log file the user can use to configure Log4j2. * Can be a resource in the jar (specified with classpath:/path/to/resource) or a file. * This configuration is applied in addition to the regular worker log4j2 configuration. * The configs are merged according to the rules here: - * https://logging.apache.org/log4j/2.x/manual/configuration.html#CompositeConfiguration + * https://logging.apache.org/log4j/2.x/manual/configuration.html#CompositeConfiguration */ @IsString public static final String TOPOLOGY_LOGGING_CONFIG_FILE = "topology.logging.config"; @@ -884,7 +901,8 @@ public class Config extends HashMap { * Alternatively set {@code storm.scheduler} to {@code org.apache.storm.scheduler.resource.ResourceAwareScheduler} * using {@link Config#TOPOLOGY_SCHEDULER_STRATEGY} set to * {@code org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy} - * */ + * + */ @IsInteger @IsPositiveNumber public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines"; @@ -977,7 +995,7 @@ public class Config extends HashMap { @NotNull @IsPositiveNumber(includeZero = true) public static final String TOPOLOGY_BACKPRESSURE_WAIT_PROGRESSIVE_LEVEL3_SLEEP_MILLIS = - "topology.backpressure.wait.progressive.level3.sleep.millis"; + "topology.backpressure.wait.progressive.level3.sleep.millis"; /** * Configures steps used to determine progression to the next level of wait .. if using WaitStrategyProgressive for BackPressure. */ @@ -1434,22 +1452,34 @@ public class Config extends HashMap { @IsString public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME = "storm.zookeeper.topology.auth.scheme"; - /** Enable SSL/TLS for ZooKeeper client connection. */ + /** + * Enable SSL/TLS for ZooKeeper client connection. + */ @IsBoolean public static final String ZK_SSL_ENABLE = "storm.zookeeper.ssl.enable"; - /** Keystore location for ZooKeeper client connection over SSL. */ + /** + * Keystore location for ZooKeeper client connection over SSL. + */ @IsString public static final String STORM_ZOOKEEPER_SSL_KEYSTORE_PATH = "storm.zookeeper.ssl.keystore.path"; - /** Keystore password for ZooKeeper client connection over SSL. */ + /** + * Keystore password for ZooKeeper client connection over SSL. + */ @IsString public static final String STORM_ZOOKEEPER_SSL_KEYSTORE_PASSWORD = "storm.zookeeper.ssl.keystore.password"; - /** Truststore location for ZooKeeper client connection over SSL. */ + /** + * Truststore location for ZooKeeper client connection over SSL. + */ @IsString public static final String STORM_ZOOKEEPER_SSL_TRUSTSTORE_PATH = "storm.zookeeper.ssl.truststore.path"; - /** Truststore password for ZooKeeper client connection over SSL. */ + /** + * Truststore password for ZooKeeper client connection over SSL. + */ @IsString public static final String STORM_ZOOKEEPER_SSL_TRUSTSTORE_PASSWORD = "storm.zookeeper.ssl.truststore.password"; - /** Enable or disable hostname verification.*/ + /** + * Enable or disable hostname verification. + */ @IsBoolean public static final String STORM_ZOOKEEPER_SSL_HOSTNAME_VERIFICATION = "storm.zookeeper.ssl.hostnameVerification"; /** @@ -1462,13 +1492,13 @@ public class Config extends HashMap { /** * Configure the topology metrics reporters to be used on workers. */ - @IsListEntryCustom(entryValidatorClasses = { MetricReportersValidator.class }) + @IsListEntryCustom(entryValidatorClasses = {MetricReportersValidator.class}) public static final String TOPOLOGY_METRICS_REPORTERS = "topology.metrics.reporters"; /** * A list of system metrics reporters that will get added to each topology. */ - @IsListEntryCustom(entryValidatorClasses = { MetricReportersValidator.class }) + @IsListEntryCustom(entryValidatorClasses = {MetricReportersValidator.class}) public static final String STORM_TOPOLOGY_METRICS_SYSTEM_REPORTERS = "storm.topology.metrics.system.reporters"; /** @@ -1476,7 +1506,7 @@ public class Config extends HashMap { * Use {@link Config#TOPOLOGY_METRICS_REPORTERS} instead. */ @Deprecated(forRemoval = true, since = "2.0.0") - @IsListEntryCustom(entryValidatorClasses = { MetricReportersValidator.class }) + @IsListEntryCustom(entryValidatorClasses = {MetricReportersValidator.class}) public static final String STORM_METRICS_REPORTERS = "storm.metrics.reporters"; /** @@ -1511,6 +1541,7 @@ public class Config extends HashMap { public static final String BLOBSTORE_HDFS_PRINCIPAL = "blobstore.hdfs.principal"; /** * keytab for nimbus/supervisor to use to access secure hdfs for the blobstore. + * * @Deprecated Use {@link Config#STORM_HDFS_LOGIN_KEYTAB} instead. */ @Deprecated @@ -1830,7 +1861,7 @@ public class Config extends HashMap { public static final String STORM_MESSAGING_NETTY_TLS_SSL_PROTOCOLS = "storm.messaging.netty.tls.ssl.protocols"; /** - /** + * /** * Netty based messaging: The number of milliseconds that a Netty client will retry flushing messages that are already * buffered to be sent. */ @@ -1924,8 +1955,8 @@ public class Config extends HashMap { /** * Impersonation user ACL config entries. */ - @IsMapEntryCustom(keyValidatorClasses = { ConfigValidation.StringValidator.class }, - valueValidatorClasses = { ConfigValidation.ImpersonationAclUserEntryValidator.class }) + @IsMapEntryCustom(keyValidatorClasses = {ConfigValidation.StringValidator.class}, + valueValidatorClasses = {ConfigValidation.ImpersonationAclUserEntryValidator.class}) public static final String NIMBUS_IMPERSONATION_ACL = "nimbus.impersonation.acl"; /** * A whitelist of the RAS scheduler strategies allowed by nimbus. Should be a list of fully-qualified class names or null to allow all. @@ -2385,7 +2416,7 @@ public void setTopologyComponentWorkerConstraints(String component1, String comp if (component1 != null && component2 != null) { List constraintPair = Arrays.asList(component1, component2); List> constraints = (List>) computeIfAbsent(Config.TOPOLOGY_RAS_CONSTRAINTS, - (k) -> new ArrayList<>(1)); + (k) -> new ArrayList<>(1)); constraints.add(constraintPair); } } @@ -2430,6 +2461,7 @@ public static String getBlobstoreHDFSPrincipal(Map conf) throws UnknownHostExcep /** * Get the hostname substituted hdfs principal. + * * @param conf the storm Configuration * @return the principal * @throws UnknownHostException on UnknowHostException @@ -2445,12 +2477,12 @@ public static String getHdfsPrincipal(Map conf) throws UnknownHo ret = hdfsPrincipal; } else if (hdfsPrincipal == null) { LOG.warn("{} is used as the hdfs principal. Please use {} instead", - Config.BLOBSTORE_HDFS_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL); + Config.BLOBSTORE_HDFS_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL); ret = blobstorePrincipal; } else { //both not null; LOG.warn("Both {} and {} are set. Use {} only.", - Config.BLOBSTORE_HDFS_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL); + Config.BLOBSTORE_HDFS_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL); ret = hdfsPrincipal; } return substituteHostnameInPrincipal(ret); @@ -2458,6 +2490,7 @@ public static String getHdfsPrincipal(Map conf) throws UnknownHo /** * Get the hdfs keytab. + * * @param conf the storm Configuration * @return the keytab */ @@ -2472,12 +2505,12 @@ public static String getHdfsKeytab(Map conf) { ret = hdfsKeyTab; } else if (hdfsKeyTab == null) { LOG.warn("{} is used as the hdfs keytab. Please use {} instead", - Config.BLOBSTORE_HDFS_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB); + Config.BLOBSTORE_HDFS_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB); ret = blobstoreKeyTab; } else { //both not null; LOG.warn("Both {} and {} are set. Use {} only.", - Config.BLOBSTORE_HDFS_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB); + Config.BLOBSTORE_HDFS_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB); ret = hdfsKeyTab; } return ret; diff --git a/storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java b/storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java new file mode 100644 index 00000000000..c16d65f9d55 --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java @@ -0,0 +1,72 @@ +/* + * 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.storm.metrics2; + +import com.codahale.metrics.Gauge; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.LongAdder; + +public class EWMAGauge implements Gauge { + + public static final double RFC1889_ALPHA = 1.0 / 16.0; + private static final long UNSEEDED = Long.MIN_VALUE; + private static final int MAX_WINDOW_SIZE = 10; + + private final double alpha; + private double jitter = 0.0; + private final AtomicLong lastTransit = new AtomicLong(UNSEEDED); + private final LongAdder deviationSum = new LongAdder(); + private final LongAdder deviationCount = new LongAdder(); + + EWMAGauge(double alpha) { + if (alpha <= 0.0 || alpha >= 1.0 || Double.isNaN(alpha)) { + throw new IllegalArgumentException( + "alpha must be in (0, 1), got: " + alpha); + } + this.alpha = alpha; + } + + EWMAGauge() { + this(RFC1889_ALPHA); + } + + public void addValue(long transitMs) { + if (transitMs < 0) { + return; + } + if (lastTransit.compareAndSet(UNSEEDED, transitMs)) { + return; + } + long prev = lastTransit.getAndSet(transitMs); + if (prev == UNSEEDED) { + return; + } + deviationSum.add(Math.abs(transitMs - prev)); + deviationCount.increment(); + if (deviationCount.longValue() >= MAX_WINDOW_SIZE) { + getValue(); + } + } + + @Override + public synchronized Double getValue() { + long sum = deviationSum.sumThenReset(); + long count = deviationCount.sumThenReset(); + if (count > 0) { + double meanDeviation = (double) sum / count; + jitter += (meanDeviation - jitter) * alpha; + } + return jitter; + } +} \ No newline at end of file diff --git a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java index 0ac3a5e9492..9a041c3bbb4 100644 --- a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java +++ b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java @@ -12,10 +12,12 @@ package org.apache.storm.metrics2; -import com.codahale.metrics.Counter; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.Supplier; + +import com.codahale.metrics.Gauge; import org.apache.storm.task.WorkerTopologyContext; import org.apache.storm.utils.ConfigUtils; import org.apache.storm.utils.Utils; @@ -27,12 +29,15 @@ public class TaskMetrics { private static final String METRIC_NAME_TRANSFERRED = "__transfer-count"; private static final String METRIC_NAME_EXECUTED = "__execute-count"; private static final String METRIC_NAME_PROCESS_LATENCY = "__process-latency"; + private static final String METRIC_NAME_PROCESS_RFC_1889a_JITTER = "__process-rfc1889a-jitter"; private static final String METRIC_NAME_COMPLETE_LATENCY = "__complete-latency"; + private static final String METRIC_NAME_COMPLETE_RFC_1889a_JITTER = "__complete-rfc1889a-jitter"; private static final String METRIC_NAME_EXECUTE_LATENCY = "__execute-latency"; + private static final String METRIC_NAME_EXECUTE_RFC_1889a_JITTER = "__execute-rfc1889a-jitter"; private static final String METRIC_NAME_CAPACITY = "__capacity"; private final ConcurrentMap rateCounters = new ConcurrentHashMap<>(); - private final ConcurrentMap gauges = new ConcurrentHashMap<>(); + private final ConcurrentMap gauges = new ConcurrentHashMap<>(); private final String topologyId; private final String componentId; @@ -40,6 +45,8 @@ public class TaskMetrics { private final Integer workerPort; private final StormMetricRegistry metricRegistry; private final int samplingRate; + private final double ewmaSmoothingFactor; + private final boolean ewmaEnable; public TaskMetrics(WorkerTopologyContext context, String componentId, Integer taskid, @@ -50,6 +57,8 @@ public TaskMetrics(WorkerTopologyContext context, String componentId, Integer ta this.taskId = taskid; this.workerPort = context.getThisWorkerPort(); this.samplingRate = ConfigUtils.samplingRate(topoConf); + this.ewmaSmoothingFactor = ConfigUtils.ewmaSmoothingFactor(topoConf); + this.ewmaEnable = ConfigUtils.ewmaEnable(topoConf); } public void setCapacity(double capacity) { @@ -67,6 +76,12 @@ public void spoutAckedTuple(String streamId, long latencyMs) { metricName = METRIC_NAME_COMPLETE_LATENCY + "-" + streamId; RollingAverageGauge gauge = this.getRollingAverageGauge(metricName, streamId); gauge.addValue(latencyMs); + + if(this.ewmaEnable) { + metricName = METRIC_NAME_COMPLETE_RFC_1889a_JITTER + "-" + streamId; + EWMAGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, streamId); + ewmaGauge.addValue(latencyMs); + } } public void boltAckedTuple(String sourceComponentId, String sourceStreamId, long latencyMs) { @@ -78,6 +93,12 @@ public void boltAckedTuple(String sourceComponentId, String sourceStreamId, long metricName = METRIC_NAME_PROCESS_LATENCY + "-" + key; RollingAverageGauge gauge = this.getRollingAverageGauge(metricName, sourceStreamId); gauge.addValue(latencyMs); + + if(this.ewmaEnable) { + metricName = METRIC_NAME_PROCESS_RFC_1889a_JITTER + "-" + key; + EWMAGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId); + ewmaGauge.addValue(latencyMs); + } } public void spoutFailedTuple(String streamId) { @@ -117,6 +138,12 @@ public void boltExecuteTuple(String sourceComponentId, String sourceStreamId, lo metricName = METRIC_NAME_EXECUTE_LATENCY + "-" + key; RollingAverageGauge gauge = this.getRollingAverageGauge(metricName, sourceStreamId); gauge.addValue(latencyMs); + + if(this.ewmaEnable) { + metricName = METRIC_NAME_EXECUTE_RFC_1889a_JITTER + "-" + key; + EWMAGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId); + ewmaGauge.addValue(latencyMs); + } } private RateCounter getRateCounter(String metricName, String streamId) { @@ -135,18 +162,52 @@ private RateCounter getRateCounter(String metricName, String streamId) { } private RollingAverageGauge getRollingAverageGauge(String metricName, String streamId) { - RollingAverageGauge gauge = this.gauges.get(metricName); - if (gauge == null) { + return getOrCreateGauge(metricName, streamId, RollingAverageGauge.class, RollingAverageGauge::new); + } + + private EWMAGauge getExponentialWeightedMobileAverageGauge(String metricName, String streamId) { + return getOrCreateGauge(metricName, streamId, EWMAGauge.class, () -> new EWMAGauge(ewmaSmoothingFactor)); + } + + private > G getOrCreateGauge( + String metricName, + String streamId, + Class gaugeClass, + Supplier factory) { + + Object existing = this.gauges.get(metricName); + if (existing == null) { synchronized (this) { - gauge = this.gauges.get(metricName); - if (gauge == null) { - gauge = new RollingAverageGauge(); - metricRegistry.gauge(metricName, gauge, this.topologyId, this.componentId, - streamId, this.taskId, this.workerPort); - this.gauges.put(metricName, gauge); + existing = this.gauges.get(metricName); + if (existing == null) { + G created = factory.get(); + registerGauge(metricName, streamId, created); + this.gauges.put(metricName, created); + return created; } } } - return gauge; + + if (!gaugeClass.isInstance(existing)) { + throw new IllegalStateException( + "Metric '" + metricName + "' is registered as " + + existing.getClass().getName() + + " but expected " + gaugeClass.getName()); + } + + return gaugeClass.cast(existing); + } + + /** Safe cast: G is bounded by Gauge in the signature of getOrCreateGauge, + so every instance of G is by definition a Gauge. + The cast to raw Gauge is required because metricRegistry.gauge() does not + accept Gauge the wildcard is not compatible with the type parameter T + expected by the external API. Type-safety is guaranteed by the bound + > declared at the call site. **/ + @SuppressWarnings({"unchecked", "rawtypes"}) + private void registerGauge(String metricName, String streamId, Gauge gauge) { + metricRegistry.gauge(metricName, (Gauge) gauge, this.topologyId, + this.componentId, streamId, this.taskId, this.workerPort); } + } diff --git a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java index 9357cc562cf..1bcf7c6a579 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java +++ b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java @@ -35,6 +35,8 @@ import org.apache.storm.validation.ConfigValidation; import org.apache.storm.validation.ConfigValidationAnnotations; +import static org.apache.storm.metrics2.EWMAGauge.RFC1889_ALPHA; + public class ConfigUtils { public static final String FILE_SEPARATOR = File.separator; public static final String STORM_HOME = "storm.home"; @@ -175,6 +177,28 @@ public static int samplingRate(Map conf) { throw new IllegalArgumentException("Illegal topology.stats.sample.rate in conf: " + rate); } + public static double ewmaSmoothingFactor(Map conf) { + Object value = conf.get(Config.TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR); + if (value == null) { + return RFC1889_ALPHA; + } + double alpha = ObjectReader.getDouble(value); + if (alpha > 0.0 && alpha < 1.0) { + return alpha; + } + throw new IllegalArgumentException( + "Illegal " + Config.TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR + + " in conf: " + alpha + " must be in (0, 1)"); + } + + public static boolean ewmaEnable(Map conf) { + Object value = conf.get(Config.TOPOLOGY_STATS_EWMA_ENABLE); + if (value == null) { + return false; + } + return ObjectReader.getBoolean(value, false); + } + public static BooleanSupplier mkStatsSampler(Map conf) { return evenSampler(samplingRate(conf)); } diff --git a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java index 4175ee9f4a7..36ab2b1661a 100644 --- a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java +++ b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java @@ -849,6 +849,23 @@ public void validateField(String name, Object o) { } } + public static class EWMASmoothingFactorValidator extends Validator { + @Override + public void validateField(String name, Object o) { + if (o == null){ + return; + } + if (o instanceof Number) { + double alpha = ((Number) o).doubleValue(); + if (alpha > 0.0 && alpha < 1.0) { + return; + } + } + throw new IllegalArgumentException( + "Field " + name + " must be a number in the open interval (0, 1), got: " + o); + } + } + public static class CustomIsExactlyOneOfValidators extends Validator { private Class[] subValidators; private List validatorClassNames; diff --git a/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java b/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java index 3c335308da5..58668db1db6 100644 --- a/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java +++ b/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java @@ -206,6 +206,32 @@ public void testTopologyStatsSampleRateIsFloat() { ConfigValidation.validateFields(conf); } + @Test + public void testTopologyStatsEwmaEnableIsBoolean() { + Map conf = new HashMap<>(); + // optional configuration + ConfigValidation.validateFields(conf); + conf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, true); + ConfigValidation.validateFields(conf); + conf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, false); + ConfigValidation.validateFields(conf); + } + + @Test + public void testTopologyStatsEwmaSmoothingFactorCustomValidator() { + Map conf = new HashMap<>(); + // optional configuration + ConfigValidation.validateFields(conf); + conf.put(Config.TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR, 0.1); + ConfigValidation.validateFields(conf); + conf.put(Config.TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR, 0.9); + ConfigValidation.validateFields(conf); + for (double notAllowedValue : new double[]{0.0, -0.1, 1.9}) { + conf.put(Config.TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR, notAllowedValue); + assertThrows(IllegalArgumentException.class, () -> ConfigValidation.validateFields(conf)); + } + } + @Test public void testWorkerChildoptsIsStringOrStringList() { Map conf = new HashMap<>(); diff --git a/storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java b/storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java new file mode 100644 index 00000000000..1287ee43bcc --- /dev/null +++ b/storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java @@ -0,0 +1,346 @@ +/* + * 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.storm.metrics2; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class EWMAGaugeTest { + + private static final double DELTA = 1e-9; + + @Nested + @DisplayName("Construction") + class ConstructionTest { + + @Test + @DisplayName("Default constructor uses RFC 1889 alpha (1/16)") + void defaultAlpha() { + EWMAGauge gauge = new EWMAGauge(); + gauge.addValue(0L); + gauge.addValue(16L); // D = 16 ; J = 0 + (16 - 0) * (1/16) = 1.0 + assertEquals(1.0, gauge.getValue(), DELTA); + } + + @Test + @DisplayName("Invalid alpha values throw IllegalArgumentException") + void invalidAlphaThrows() { + double[] invalidAlphas = { + 0.0, 1.0, -0.1, 1.1, + Double.NaN, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY + }; + for (double alpha : invalidAlphas) { + assertThrows(IllegalArgumentException.class, + () -> new EWMAGauge(alpha), + "Expected IllegalArgumentException for alpha=" + alpha); + } + } + + @Test + @DisplayName("Valid alpha boundary values are accepted") + void validAlphaAccepted() { + double[] validAlphas = {0.001, 0.0625, 0.5, 0.999}; + for (double alpha : validAlphas) { + assertNotNull(new EWMAGauge(alpha), + "Expected no exception for alpha=" + alpha); + } + } + } + + + @Nested + @DisplayName("Cold-start semantics") + class ColdStartTest { + + private EWMAGauge gauge; + + @BeforeEach + void setUp() { + gauge = new EWMAGauge(); + } + + @Test + @DisplayName("getValue() returns 0.0 before any sample") + void noSamples() { + assertEquals(0.0, gauge.getValue(), DELTA); + } + + @Test + @DisplayName("getValue() returns 0.0 after exactly one sample (seed only)") + void oneSample() { + gauge.addValue(100L); + assertEquals(0.0, gauge.getValue(), DELTA); + } + + } + + @Nested + @DisplayName("EWMA formula RFC 1889 §A.8") + class FormulaTest { + + @Test + @DisplayName("Single update: J = 0 + (D - 0) * alpha") + void singleDeviation() { + EWMAGauge gauge = new EWMAGauge(0.5); + gauge.addValue(0L); + gauge.addValue(10L); + assertEquals(5.0, gauge.getValue(), DELTA); + } + + @Test + @DisplayName("Manual step-by-step verification against reference values") + void manualSteps() { + EWMAGauge gauge = new EWMAGauge(0.5); + + gauge.addValue(0L); // seed + + // Step 1: transit=10, prev=0, D=10, J = 0 + (10-0) * 0.5 = 5.0 + gauge.addValue(10L); + assertEquals(5.0, gauge.getValue(), DELTA, "Step 1"); + + // Step 2: transit=0, prev=10, D=10, J = 5.0 + (10-5.0) * 0.5 = 7.5 + gauge.addValue(0L); + assertEquals(7.5, gauge.getValue(), DELTA, "Step 2"); + + // Step 3: transit=10, prev=0, D=10, J = 7.5 + (10-7.5) * 0.5 = 8.75 + gauge.addValue(10L); + assertEquals(8.75, gauge.getValue(), DELTA, "Step 3"); + } + + @Test + @DisplayName("Zero deviation decays jitter toward zero") + void zeroDeviationDecays() { + EWMAGauge gauge = new EWMAGauge(0.5); + gauge.addValue(0L); + gauge.addValue(10L); + double afterFirst = gauge.getValue(); + + gauge.addValue(10L); + assertEquals(afterFirst * 0.5, gauge.getValue(), DELTA); + } + + } + + + @Nested + @DisplayName("Negative value guard") + class NegativeValueTest { + + @Test + @DisplayName("Negative transit values are silently ignored before seed") + void negativeIgnoredBeforeSeed() { + EWMAGauge gauge = new EWMAGauge(); + gauge.addValue(-1L); + gauge.addValue(-100L); + assertEquals(0.0, gauge.getValue(), DELTA); + } + + @Test + @DisplayName("Negative value after seed does not corrupt lastTransit") + void negativeAfterSeedIgnored() { + EWMAGauge gauge = new EWMAGauge(0.5); + gauge.addValue(10L); + gauge.addValue(-5L); + gauge.addValue(20L); + assertEquals(5.0, gauge.getValue(), DELTA); + } + } + + + @Nested + @DisplayName("getValue() preserves EWMA across calls") + class GetValueIdempotentTest { + + @Test + @DisplayName("Repeated getValue() without new samples returns same estimate") + void repeatedGetValueStable() { + EWMAGauge gauge = new EWMAGauge(0.5); + gauge.addValue(0L); + gauge.addValue(10L); + double first = gauge.getValue(); + + assertEquals(first, gauge.getValue(), DELTA, "Second call"); + assertEquals(first, gauge.getValue(), DELTA, "Third call"); + } + + @Test + @DisplayName("EWMA accumulates correctly across multiple reporting windows") + void acrossReportingWindows() { + EWMAGauge gauge = new EWMAGauge(0.5); + gauge.addValue(0L); + + gauge.addValue(10L); + assertEquals(5.0, gauge.getValue(), DELTA, "Window 1"); + + gauge.addValue(0L); + assertEquals(7.5, gauge.getValue(), DELTA, "Window 2"); + + gauge.addValue(10L); + assertEquals(8.75, gauge.getValue(), DELTA, "Window 3"); + } + } + + @Nested + @DisplayName("thread safe") + class ConcurrencyTest { + + @Test + @DisplayName("Concurrent addValue() calls do not corrupt state") + void concurrentAddValue() throws InterruptedException { + EWMAGauge gauge = new EWMAGauge(); + int threads = 8; + int samplesPerThread = 10_000; + CountDownLatch ready = new CountDownLatch(threads); + CountDownLatch start = new CountDownLatch(1); + ExecutorService pool = Executors.newFixedThreadPool(threads); + + for (int t = 0; t < threads; t++) { + final long base = t * 10L; + pool.submit(() -> { + ready.countDown(); + try { + start.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + for (int i = 0; i < samplesPerThread; i++) { + gauge.addValue(base + (i % 10)); + } + }); + } + + ready.await(); + start.countDown(); + pool.shutdown(); + assertTrue(pool.awaitTermination(10, TimeUnit.SECONDS), + "Executor did not terminate — possible deadlock"); + + double value = gauge.getValue(); + assertTrue(value >= 0.0,"Jitter must be non-negative, got: " + value); + assertTrue(Double.isFinite(value), "Jitter must be finite, got: " + value); + } + + @Test + @DisplayName("Concurrent getValue() and addValue() do not deadlock") + void concurrentGetAndAdd() throws Exception { + EWMAGauge gauge = new EWMAGauge(); + ExecutorService pool = Executors.newFixedThreadPool(2); + CountDownLatch done = new CountDownLatch(2); + + Future writer = pool.submit(() -> { + for (int i = 0; i < 50_000; i++) { + gauge.addValue(i % 100); + } + done.countDown(); + }); + + Future reader = pool.submit(() -> { + for (int i = 0; i < 1_000; i++) { + double v = gauge.getValue(); + assertTrue(v >= 0.0 && Double.isFinite(v), + "getValue() returned invalid result: " + v); + } + done.countDown(); + }); + + assertTrue(done.await(10, TimeUnit.SECONDS), + "Test did not complete within timeout possible deadlock"); + + writer.get(); + reader.get(); + pool.shutdown(); + } + + @Test + @DisplayName("Only one thread seeds lastTransit all same value gives zero jitter") + void seedRace() throws InterruptedException { + EWMAGauge gauge = new EWMAGauge(); + int threads = 16; + CountDownLatch ready = new CountDownLatch(threads); + CountDownLatch start = new CountDownLatch(1); + ExecutorService pool = Executors.newFixedThreadPool(threads); + + for (int t = 0; t < threads; t++) { + pool.submit(() -> { + ready.countDown(); + try { + start.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + gauge.addValue(42L); + }); + } + + ready.await(); + start.countDown(); + pool.shutdown(); + assertTrue(pool.awaitTermination(5, TimeUnit.SECONDS), + "Executor did not terminate possible deadlock"); + + assertEquals(0.0, gauge.getValue(), DELTA); + } + } + + @Nested + @DisplayName("Edge cases") + class EdgeCaseTest { + + @Test + @DisplayName("Long.MAX_VALUE transit does not overflow deviation") + void maxLongTransit() { + EWMAGauge gauge = new EWMAGauge(0.5); + gauge.addValue(0L); + gauge.addValue(Long.MAX_VALUE); + double value = gauge.getValue(); + assertTrue(value > 0.0,"Jitter should be positive"); + assertTrue(Double.isFinite(value),"Jitter should be finite"); + } + + @Test + @DisplayName("Zero transit time is valid and produces zero deviation") + void zeroTransit() { + EWMAGauge gauge = new EWMAGauge(0.5); + gauge.addValue(0L); + gauge.addValue(0L); + assertEquals(0.0, gauge.getValue(), DELTA); + } + + @Test + @DisplayName("Large number of samples does not overflow LongAdder") + void manySamples() { + EWMAGauge gauge = new EWMAGauge(); + gauge.addValue(0L); + for (int i = 1; i <= 100_000; i++) { + gauge.addValue(i % 2 == 0 ? 0L : 10L); + } + double value = gauge.getValue(); + assertTrue(value > 0.0,"Jitter should be positive after many samples"); + assertTrue(value <= 10.0,"Jitter cannot exceed max deviation of 10"); + assertTrue(Double.isFinite(value),"Jitter must be finite"); + } + } +} \ No newline at end of file diff --git a/storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java b/storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java new file mode 100644 index 00000000000..b2c504a3621 --- /dev/null +++ b/storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java @@ -0,0 +1,490 @@ +/* + * 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.storm.metrics2; + +import com.codahale.metrics.Gauge; +import org.apache.storm.task.WorkerTopologyContext; +import org.apache.storm.utils.ConfigUtils; +import org.apache.storm.utils.Utils; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.*; +import static org.mockito.Mockito.*; + +@ExtendWith(MockitoExtension.class) +class TaskMetricsTest { + + private static final String TOPOLOGY_ID = "test-topology-1"; + private static final String COMPONENT_ID = "test-bolt"; + private static final Integer TASK_ID = 42; + private static final Integer WORKER_PORT = 6700; + private static final String STREAM_ID = "default"; + private static final String SOURCE_COMP = "source-spout"; + private static final int SAMPLING_RATE = 1; + private static final double EWMA_FACTOR = 0.3; + + @Mock private WorkerTopologyContext context; + @Mock private StormMetricRegistry metricRegistry; + @Mock private RateCounter rateCounter; + @Mock private RollingAverageGauge rollingAverageGauge; + @Mock private EWMAGauge ewmaGauge; + + private Map topoConf; + + private TaskMetrics buildTaskMetrics(boolean ewmaEnabled) { + try (MockedStatic cfgUtils = mockStatic(ConfigUtils.class)) { + cfgUtils.when(() -> ConfigUtils.samplingRate(topoConf)).thenReturn(SAMPLING_RATE); + cfgUtils.when(() -> ConfigUtils.ewmaSmoothingFactor(topoConf)).thenReturn(EWMA_FACTOR); + cfgUtils.when(() -> ConfigUtils.ewmaEnable(topoConf)).thenReturn(ewmaEnabled); + + return new TaskMetrics(context, COMPONENT_ID, TASK_ID, metricRegistry, topoConf); + } + } + + @BeforeEach + void setUp() { + when(context.getStormId()).thenReturn(TOPOLOGY_ID); + when(context.getThisWorkerPort()).thenReturn(WORKER_PORT); + + topoConf = new HashMap<>(); + + when(metricRegistry.rateCounter(anyString(), anyString(), anyString(), + anyInt(), anyInt(), anyString())).thenReturn(rateCounter); + } + + @Test + void spoutAckedTuple_incrementsAckCounter() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.spoutAckedTuple(STREAM_ID, 100L); + + verify(rateCounter).inc(SAMPLING_RATE); + } + + @Test + void spoutAckedTuple_registersCompleteLatencyGauge() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.spoutAckedTuple(STREAM_ID, 200L); + + verify(metricRegistry, atLeastOnce()).gauge( + contains("__complete-latency"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void spoutAckedTuple_withEwmaEnabled_registersJitterGauge() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(true); + + tm.spoutAckedTuple(STREAM_ID, 150L); + + verify(metricRegistry, atLeastOnce()).gauge( + contains("__complete-rfc1889a-jitter"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void spoutAckedTuple_withEwmaDisabled_doesNotRegisterJitterGauge() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.spoutAckedTuple(STREAM_ID, 150L); + + verify(metricRegistry, never()).gauge( + contains("__complete-rfc1889a-jitter"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void boltAckedTuple_incrementsAckCounter() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.boltAckedTuple(SOURCE_COMP, STREAM_ID, 50L); + + verify(rateCounter).inc(SAMPLING_RATE); + } + + @Test + void boltAckedTuple_registersProcessLatencyGauge() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.boltAckedTuple(SOURCE_COMP, STREAM_ID, 50L); + + verify(metricRegistry, atLeastOnce()).gauge( + contains("__process-latency"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void boltAckedTuple_withEwmaEnabled_registersJitterGauge() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(true); + + tm.boltAckedTuple(SOURCE_COMP, STREAM_ID, 75L); + + verify(metricRegistry, atLeastOnce()).gauge( + contains("__process-rfc1889a-jitter"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void boltAckedTuple_metricKeyIncludesSourceComponentAndStream() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.boltAckedTuple(SOURCE_COMP, STREAM_ID, 50L); + + verify(metricRegistry).rateCounter( + contains(SOURCE_COMP + ":" + STREAM_ID), + anyString(), anyString(), anyInt(), anyInt(), anyString()); + } + + @Test + void spoutFailedTuple_incrementsFailCounter() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.spoutFailedTuple(STREAM_ID); + + verify(rateCounter).inc(SAMPLING_RATE); + } + + @Test + void spoutFailedTuple_usesCorrectMetricName() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.spoutFailedTuple(STREAM_ID); + + verify(metricRegistry).rateCounter( + eq("__fail-count-" + STREAM_ID), + eq(TOPOLOGY_ID), eq(COMPONENT_ID), eq(TASK_ID), eq(WORKER_PORT), eq(STREAM_ID)); + } + + @Test + void boltFailedTuple_incrementsFailCounter() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.boltFailedTuple(SOURCE_COMP, STREAM_ID); + + verify(rateCounter).inc(SAMPLING_RATE); + } + + @Test + void boltFailedTuple_metricKeyIncludesSourceComponentAndStream() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.boltFailedTuple(SOURCE_COMP, STREAM_ID); + + verify(metricRegistry).rateCounter( + eq("__fail-count-" + SOURCE_COMP + ":" + STREAM_ID), + anyString(), anyString(), anyInt(), anyInt(), anyString()); + } + + @Test + void emittedTuple_incrementsEmitCounter() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.emittedTuple(STREAM_ID); + + verify(rateCounter).inc(SAMPLING_RATE); + } + + @Test + void emittedTuple_usesCorrectMetricName() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.emittedTuple(STREAM_ID); + + verify(metricRegistry).rateCounter( + eq("__emit-count-" + STREAM_ID), + eq(TOPOLOGY_ID), eq(COMPONENT_ID), eq(TASK_ID), eq(WORKER_PORT), eq(STREAM_ID)); + } + + @Test + void transferredTuples_incrementsByAmountTimesSamplingRate() { + TaskMetrics tm = buildTaskMetrics(false); + int amount = 5; + + tm.transferredTuples(STREAM_ID, amount); + + verify(rateCounter).inc(amount * SAMPLING_RATE); + } + + @Test + void transferredTuples_usesCorrectMetricName() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.transferredTuples(STREAM_ID, 3); + + verify(metricRegistry).rateCounter( + eq("__transfer-count-" + STREAM_ID), + anyString(), anyString(), anyInt(), anyInt(), anyString()); + } + + @Test + void boltExecuteTuple_incrementsExecuteCounter() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.boltExecuteTuple(SOURCE_COMP, STREAM_ID, 30L); + + verify(rateCounter).inc(SAMPLING_RATE); + } + + @Test + void boltExecuteTuple_registersExecuteLatencyGauge() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.boltExecuteTuple(SOURCE_COMP, STREAM_ID, 30L); + + verify(metricRegistry, atLeastOnce()).gauge( + contains("__execute-latency"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void boltExecuteTuple_withEwmaEnabled_registersJitterGauge() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(true); + + tm.boltExecuteTuple(SOURCE_COMP, STREAM_ID, 30L); + + verify(metricRegistry, atLeastOnce()).gauge( + contains("__execute-rfc1889a-jitter"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void boltExecuteTuple_withEwmaDisabled_doesNotRegisterJitterGauge() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.boltExecuteTuple(SOURCE_COMP, STREAM_ID, 30L); + + verify(metricRegistry, never()).gauge( + contains("__execute-rfc1889a-jitter"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void differentStreams_produceSeparateRateCounters() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.emittedTuple("stream-A"); + tm.emittedTuple("stream-B"); + + verify(metricRegistry).rateCounter( + eq("__emit-count-stream-A"), + anyString(), anyString(), anyInt(), anyInt(), eq("stream-A")); + verify(metricRegistry).rateCounter( + eq("__emit-count-stream-B"), + anyString(), anyString(), anyInt(), anyInt(), eq("stream-B")); + } + + @Test + void rateCounter_registeredOnlyOnceForSameMetricName() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.emittedTuple(STREAM_ID); + tm.emittedTuple(STREAM_ID); + tm.emittedTuple(STREAM_ID); + + verify(metricRegistry, times(1)).rateCounter( + eq("__emit-count-" + STREAM_ID), + anyString(), anyString(), anyInt(), anyInt(), anyString()); + } + + @Test + void gauge_registeredOnlyOnceForSameMetricName() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.spoutAckedTuple(STREAM_ID, 10L); + tm.spoutAckedTuple(STREAM_ID, 20L); + tm.spoutAckedTuple(STREAM_ID, 30L); + + verify(metricRegistry, times(1)).gauge( + contains("__complete-latency"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void concurrentEmittedTuple_registersRateCounterExactlyOnce() throws InterruptedException { + TaskMetrics tm = buildTaskMetrics(false); + int threadCount = 20; + CountDownLatch ready = new CountDownLatch(threadCount); + CountDownLatch start = new CountDownLatch(1); + CountDownLatch done = new CountDownLatch(threadCount); + + ExecutorService pool = Executors.newFixedThreadPool(threadCount); + for (int i = 0; i < threadCount; i++) { + pool.submit(() -> { + ready.countDown(); + try { start.await(); } catch (InterruptedException ignored) {} + tm.emittedTuple(STREAM_ID); + done.countDown(); + }); + } + + ready.await(); + start.countDown(); + assertTrue(done.await(5, TimeUnit.SECONDS)); + pool.shutdown(); + + verify(metricRegistry, times(1)).rateCounter( + eq("__emit-count-" + STREAM_ID), + anyString(), anyString(), anyInt(), anyInt(), anyString()); + } + + @Test + void concurrentSpoutAckedTuple_registersGaugeExactlyOnce() throws InterruptedException { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + int threadCount = 20; + CountDownLatch ready = new CountDownLatch(threadCount); + CountDownLatch start = new CountDownLatch(1); + CountDownLatch done = new CountDownLatch(threadCount); + + ExecutorService pool = Executors.newFixedThreadPool(threadCount); + for (int i = 0; i < threadCount; i++) { + pool.submit(() -> { + ready.countDown(); + try { start.await(); } catch (InterruptedException ignored) {} + tm.spoutAckedTuple(STREAM_ID, 100L); + done.countDown(); + }); + } + + ready.await(); + start.countDown(); + assertTrue(done.await(5, TimeUnit.SECONDS)); + pool.shutdown(); + + verify(metricRegistry, times(1)).gauge( + contains("__complete-latency"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void getOrCreateGauge_sameTypeReusedWithoutThrowing() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + + tm.spoutAckedTuple(STREAM_ID, 10L); + tm.spoutAckedTuple(STREAM_ID, 20L); + + verify(metricRegistry, times(1)).gauge( + contains("__complete-latency"), any(Gauge.class), + anyString(), anyString(), anyString(), anyInt(), anyInt()); + } + + @Test + void boltAckedTuple_metricNameFormat() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + String expectedKey = SOURCE_COMP + ":" + STREAM_ID; + + tm.boltAckedTuple(SOURCE_COMP, STREAM_ID, 10L); + + verify(metricRegistry).rateCounter( + eq("__ack-count-" + expectedKey), + eq(TOPOLOGY_ID), eq(COMPONENT_ID), eq(TASK_ID), eq(WORKER_PORT), eq(STREAM_ID)); + } + + @Test + void boltExecuteTuple_metricNameFormat() { + when(metricRegistry.gauge(anyString(), any(Gauge.class), anyString(), + anyString(), anyString(), anyInt(), anyInt())).thenReturn(null); + TaskMetrics tm = buildTaskMetrics(false); + String expectedKey = SOURCE_COMP + ":" + STREAM_ID; + + tm.boltExecuteTuple(SOURCE_COMP, STREAM_ID, 10L); + + verify(metricRegistry).rateCounter( + eq("__execute-count-" + expectedKey), + eq(TOPOLOGY_ID), eq(COMPONENT_ID), eq(TASK_ID), eq(WORKER_PORT), eq(STREAM_ID)); + } + + @Test + void boltFailedTuple_metricNameFormat() { + TaskMetrics tm = buildTaskMetrics(false); + String expectedKey = SOURCE_COMP + ":" + STREAM_ID; + + tm.boltFailedTuple(SOURCE_COMP, STREAM_ID); + + verify(metricRegistry).rateCounter( + eq("__fail-count-" + expectedKey), + eq(TOPOLOGY_ID), eq(COMPONENT_ID), eq(TASK_ID), eq(WORKER_PORT), eq(STREAM_ID)); + } + + @Test + void contextFields_propagatedCorrectlyToRegistry() { + TaskMetrics tm = buildTaskMetrics(false); + + tm.emittedTuple(STREAM_ID); + + ArgumentCaptor topoCaptor = ArgumentCaptor.forClass(String.class); + ArgumentCaptor compCaptor = ArgumentCaptor.forClass(String.class); + ArgumentCaptor taskCaptor = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor portCaptor = ArgumentCaptor.forClass(Integer.class); + + verify(metricRegistry).rateCounter( + anyString(), + topoCaptor.capture(), compCaptor.capture(), + taskCaptor.capture(), portCaptor.capture(), + anyString()); + + assertEquals(TOPOLOGY_ID, topoCaptor.getValue()); + assertEquals(COMPONENT_ID, compCaptor.getValue()); + assertEquals(TASK_ID, taskCaptor.getValue()); + assertEquals(WORKER_PORT, portCaptor.getValue()); + } +} \ No newline at end of file From 2aad8a2547a64db0778554bb01b8f73f8996bbc1 Mon Sep 17 00:00:00 2001 From: Gianluca Graziadei Date: Sun, 3 May 2026 19:05:57 +0200 Subject: [PATCH 2/8] allocate suppliers once --- .../apache/storm/metrics2/TaskMetrics.java | 36 +++++++++++-------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java index 9a041c3bbb4..e9062172f70 100644 --- a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java +++ b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java @@ -2,9 +2,9 @@ * 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. @@ -38,6 +38,9 @@ public class TaskMetrics { private final ConcurrentMap rateCounters = new ConcurrentHashMap<>(); private final ConcurrentMap gauges = new ConcurrentHashMap<>(); + // Gauge supplier singleton factories + private final Supplier ewmaGaugeFactory; + private final Supplier rollingAverageGaugeFactory; private final String topologyId; private final String componentId; @@ -45,7 +48,6 @@ public class TaskMetrics { private final Integer workerPort; private final StormMetricRegistry metricRegistry; private final int samplingRate; - private final double ewmaSmoothingFactor; private final boolean ewmaEnable; @@ -57,8 +59,10 @@ public TaskMetrics(WorkerTopologyContext context, String componentId, Integer ta this.taskId = taskid; this.workerPort = context.getThisWorkerPort(); this.samplingRate = ConfigUtils.samplingRate(topoConf); - this.ewmaSmoothingFactor = ConfigUtils.ewmaSmoothingFactor(topoConf); + double ewmaSmoothingFactor = ConfigUtils.ewmaSmoothingFactor(topoConf); this.ewmaEnable = ConfigUtils.ewmaEnable(topoConf); + this.rollingAverageGaugeFactory = RollingAverageGauge::new; + this.ewmaGaugeFactory = () -> new EWMAGauge(ewmaSmoothingFactor); } public void setCapacity(double capacity) { @@ -77,7 +81,7 @@ public void spoutAckedTuple(String streamId, long latencyMs) { RollingAverageGauge gauge = this.getRollingAverageGauge(metricName, streamId); gauge.addValue(latencyMs); - if(this.ewmaEnable) { + if (this.ewmaEnable) { metricName = METRIC_NAME_COMPLETE_RFC_1889a_JITTER + "-" + streamId; EWMAGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, streamId); ewmaGauge.addValue(latencyMs); @@ -94,7 +98,7 @@ public void boltAckedTuple(String sourceComponentId, String sourceStreamId, long RollingAverageGauge gauge = this.getRollingAverageGauge(metricName, sourceStreamId); gauge.addValue(latencyMs); - if(this.ewmaEnable) { + if (this.ewmaEnable) { metricName = METRIC_NAME_PROCESS_RFC_1889a_JITTER + "-" + key; EWMAGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId); ewmaGauge.addValue(latencyMs); @@ -139,7 +143,7 @@ public void boltExecuteTuple(String sourceComponentId, String sourceStreamId, lo RollingAverageGauge gauge = this.getRollingAverageGauge(metricName, sourceStreamId); gauge.addValue(latencyMs); - if(this.ewmaEnable) { + if (this.ewmaEnable) { metricName = METRIC_NAME_EXECUTE_RFC_1889a_JITTER + "-" + key; EWMAGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId); ewmaGauge.addValue(latencyMs); @@ -162,11 +166,11 @@ private RateCounter getRateCounter(String metricName, String streamId) { } private RollingAverageGauge getRollingAverageGauge(String metricName, String streamId) { - return getOrCreateGauge(metricName, streamId, RollingAverageGauge.class, RollingAverageGauge::new); + return getOrCreateGauge(metricName, streamId, RollingAverageGauge.class, this.rollingAverageGaugeFactory); } private EWMAGauge getExponentialWeightedMobileAverageGauge(String metricName, String streamId) { - return getOrCreateGauge(metricName, streamId, EWMAGauge.class, () -> new EWMAGauge(ewmaSmoothingFactor)); + return getOrCreateGauge(metricName, streamId, EWMAGauge.class, this.ewmaGaugeFactory); } private > G getOrCreateGauge( @@ -198,12 +202,14 @@ private > G getOrCreateGauge( return gaugeClass.cast(existing); } - /** Safe cast: G is bounded by Gauge in the signature of getOrCreateGauge, - so every instance of G is by definition a Gauge. - The cast to raw Gauge is required because metricRegistry.gauge() does not - accept Gauge the wildcard is not compatible with the type parameter T - expected by the external API. Type-safety is guaranteed by the bound - > declared at the call site. **/ + /** + * Safe cast: G is bounded by Gauge in the signature of getOrCreateGauge, + * so every instance of G is by definition a Gauge. + * The cast to raw Gauge is required because metricRegistry.gauge() does not + * accept Gauge the wildcard is not compatible with the type parameter T + * expected by the external API. Type-safety is guaranteed by the bound + * > declared at the call site. + **/ @SuppressWarnings({"unchecked", "rawtypes"}) private void registerGauge(String metricName, String streamId, Gauge gauge) { metricRegistry.gauge(metricName, (Gauge) gauge, this.topologyId, From 5cb1936ebc7be6e7136c273ccd7882fe5548b140 Mon Sep 17 00:00:00 2001 From: Gianluca Graziadei Date: Tue, 5 May 2026 17:01:46 +0200 Subject: [PATCH 3/8] improve jitter definition --- .../org/apache/storm/metrics2/EWMAGauge.java | 63 ++++++++++++------- .../org/apache/storm/utils/ConfigUtils.java | 17 +++-- .../apache/storm/metrics2/EWMAGaugeTest.java | 24 +++---- 3 files changed, 62 insertions(+), 42 deletions(-) diff --git a/storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java b/storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java index c16d65f9d55..f55eff736f7 100644 --- a/storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java +++ b/storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java @@ -9,25 +9,30 @@ * 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.storm.metrics2; import com.codahale.metrics.Gauge; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.LongAdder; +import static org.apache.storm.utils.ConfigUtils.RFC1889_ALPHA; + +/** + * Lock-free jitter estimator following RFC 1889 Section 6.3.1. + *

+ * The jitter accumulator is stored as raw IEEE 754 bits in an AtomicLong + * so that CAS can be used without locks. + *

+ * Thread safety: addValue is lock-free; getValue is wait-free. + */ public class EWMAGauge implements Gauge { - public static final double RFC1889_ALPHA = 1.0 / 16.0; private static final long UNSEEDED = Long.MIN_VALUE; - private static final int MAX_WINDOW_SIZE = 10; + private static final long ZERO_BITS = Double.doubleToLongBits(0.0); - private final double alpha; - private double jitter = 0.0; private final AtomicLong lastTransit = new AtomicLong(UNSEEDED); - private final LongAdder deviationSum = new LongAdder(); - private final LongAdder deviationCount = new LongAdder(); + private final AtomicLong jitterBits = new AtomicLong(ZERO_BITS); + private final double alpha; EWMAGauge(double alpha) { if (alpha <= 0.0 || alpha >= 1.0 || Double.isNaN(alpha)) { @@ -38,35 +43,51 @@ public class EWMAGauge implements Gauge { } EWMAGauge() { - this(RFC1889_ALPHA); + this(RFC1889_ALPHA); // 1.0 / 16.0 } + /** + * Update the jitter estimate. + * + * @param transitMs transit time for this tuple: {@code arrival - timestamp} + * Negative values are silently ignored. + */ public void addValue(long transitMs) { if (transitMs < 0) { return; } + + // Seed on the very first packet: store transit, nothing to diff against yet. if (lastTransit.compareAndSet(UNSEEDED, transitMs)) { return; } + long prev = lastTransit.getAndSet(transitMs); if (prev == UNSEEDED) { + // Lost a race during seeding; prev is not a real transit value. return; } - deviationSum.add(Math.abs(transitMs - prev)); - deviationCount.increment(); - if (deviationCount.longValue() >= MAX_WINDOW_SIZE) { - getValue(); + + double d = Math.abs(transitMs - prev); + + if (d <= 0) { + return; } + + long currentBits, updatedBits; + do { + currentBits = jitterBits.get(); + double currentJitter = Double.longBitsToDouble(currentBits); + double updatedJitter = currentJitter + alpha * (d - currentJitter); + updatedBits = Double.doubleToLongBits(updatedJitter); + } while (!jitterBits.compareAndSet(currentBits, updatedBits)); } + /** + * Returns the current jitter estimate in timestamp units. + */ @Override - public synchronized Double getValue() { - long sum = deviationSum.sumThenReset(); - long count = deviationCount.sumThenReset(); - if (count > 0) { - double meanDeviation = (double) sum / count; - jitter += (meanDeviation - jitter) * alpha; - } - return jitter; + public Double getValue() { + return Double.longBitsToDouble(jitterBits.get()); } } \ No newline at end of file diff --git a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java index 1bcf7c6a579..9f6d5285542 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java +++ b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java @@ -2,9 +2,9 @@ * 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. @@ -35,17 +35,17 @@ import org.apache.storm.validation.ConfigValidation; import org.apache.storm.validation.ConfigValidationAnnotations; -import static org.apache.storm.metrics2.EWMAGauge.RFC1889_ALPHA; public class ConfigUtils { public static final String FILE_SEPARATOR = File.separator; public static final String STORM_HOME = "storm.home"; public static final String RESOURCES_SUBDIR = "resources"; + public static final double RFC1889_ALPHA = 1.0 / 16.0; private static final Set passwordConfigKeys = new HashSet<>(); static { - for (Class clazz: ConfigValidation.getConfigClasses()) { + for (Class clazz : ConfigValidation.getConfigClasses()) { for (Field field : clazz.getFields()) { for (Annotation annotation : field.getAnnotations()) { boolean isPassword = annotation.annotationType().getName().equals( @@ -338,7 +338,7 @@ public static Map readSupervisorStormConfGivenPath(Map overrideLoginConfigWithSystemProperty( - Map conf) { // note that we delete the return value + Map conf) { // note that we delete the return value String loginConfFile = System.getProperty("java.security.auth.login.config"); if (loginConfFile != null) { conf.put("java.security.auth.login.config", loginConfFile); @@ -437,7 +437,6 @@ public static File getWorkerDirFromRoot(String logRoot, String id, Integer port) * @param name - the config key * @param conf - the config map * @return - the config value converted to a List <String> if found, otherwise null. - * * @throws IllegalArgumentException if conf is null * @throws NullPointerException if name is null and the conf map doesn't support null keys */ @@ -451,9 +450,9 @@ public static List getValueAsList(String name, Map conf) listValue = null; } else if (value instanceof Collection) { listValue = ((Collection) value) - .stream() - .map(ObjectReader::getString) - .collect(Collectors.toList()); + .stream() + .map(ObjectReader::getString) + .collect(Collectors.toList()); } else { listValue = Arrays.asList(ObjectReader.getString(value).split("\\s+")); } diff --git a/storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java b/storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java index 1287ee43bcc..6bf53ff0784 100644 --- a/storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java +++ b/storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java @@ -134,12 +134,12 @@ void manualSteps() { @DisplayName("Zero deviation decays jitter toward zero") void zeroDeviationDecays() { EWMAGauge gauge = new EWMAGauge(0.5); - gauge.addValue(0L); - gauge.addValue(10L); + gauge.addValue(0L); // 0 + gauge.addValue(10L); // 0 + 5*alpha = 2.5 double afterFirst = gauge.getValue(); - gauge.addValue(10L); - assertEquals(afterFirst * 0.5, gauge.getValue(), DELTA); + gauge.addValue(10L); // 2.5 + 0*alpha = 2.5 + assertEquals(afterFirst, gauge.getValue(), DELTA); } } @@ -193,10 +193,10 @@ void acrossReportingWindows() { gauge.addValue(0L); gauge.addValue(10L); - assertEquals(5.0, gauge.getValue(), DELTA, "Window 1"); + assertEquals(5.0, gauge.getValue(), DELTA, "Window 1"); gauge.addValue(0L); - assertEquals(7.5, gauge.getValue(), DELTA, "Window 2"); + assertEquals(7.5, gauge.getValue(), DELTA, "Window 2"); gauge.addValue(10L); assertEquals(8.75, gauge.getValue(), DELTA, "Window 3"); @@ -239,7 +239,7 @@ void concurrentAddValue() throws InterruptedException { "Executor did not terminate — possible deadlock"); double value = gauge.getValue(); - assertTrue(value >= 0.0,"Jitter must be non-negative, got: " + value); + assertTrue(value >= 0.0, "Jitter must be non-negative, got: " + value); assertTrue(Double.isFinite(value), "Jitter must be finite, got: " + value); } @@ -316,8 +316,8 @@ void maxLongTransit() { gauge.addValue(0L); gauge.addValue(Long.MAX_VALUE); double value = gauge.getValue(); - assertTrue(value > 0.0,"Jitter should be positive"); - assertTrue(Double.isFinite(value),"Jitter should be finite"); + assertTrue(value > 0.0, "Jitter should be positive"); + assertTrue(Double.isFinite(value), "Jitter should be finite"); } @Test @@ -338,9 +338,9 @@ void manySamples() { gauge.addValue(i % 2 == 0 ? 0L : 10L); } double value = gauge.getValue(); - assertTrue(value > 0.0,"Jitter should be positive after many samples"); - assertTrue(value <= 10.0,"Jitter cannot exceed max deviation of 10"); - assertTrue(Double.isFinite(value),"Jitter must be finite"); + assertTrue(value > 0.0, "Jitter should be positive after many samples"); + assertTrue(value <= 10.0, "Jitter cannot exceed max deviation of 10"); + assertTrue(Double.isFinite(value), "Jitter must be finite"); } } } \ No newline at end of file From 5aa82648e5538ae5b3df8d74f3249434d1b24bb1 Mon Sep 17 00:00:00 2001 From: Gianluca Graziadei Date: Tue, 5 May 2026 18:26:32 +0200 Subject: [PATCH 4/8] add documentation --- docs/Metrics.md | 314 ++++++++++++++++++++++++++++++++++-------------- 1 file changed, 223 insertions(+), 91 deletions(-) diff --git a/docs/Metrics.md b/docs/Metrics.md index 4a2db2728b7..c3cff6eba91 100644 --- a/docs/Metrics.md +++ b/docs/Metrics.md @@ -3,38 +3,58 @@ title: Storm Metrics layout: documentation documentation: true --- + Storm exposes a metrics interface to report summary statistics across the full topology. -The numbers you see on the UI come from some of these built in metrics, but are reported through the worker heartbeats instead of through the IMetricsConsumer described below. +The numbers you see on the UI come from some of these built in metrics, but are reported through the worker heartbeats +instead of through the IMetricsConsumer described below. If you are looking for cluster wide monitoring please see [Cluster Metrics](ClusterMetrics.html). ### Metric Types -Metrics have to implement [`IMetric`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java) which contains just one method, `getValueAndReset` -- do any remaining work to find the summary value, and reset back to an initial state. For example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values back to zero. +Metrics have to implement [ +`IMetric`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java) which contains just one +method, `getValueAndReset` -- do any remaining work to find the summary value, and reset back to an initial state. For +example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values +back to zero. Storm gives you these metric types: -* [AssignableMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java) -- set the metric to the explicit value you supply. Useful if it's an external value or in the case that you are already calculating the summary statistic yourself. -* [CombinedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java) -- generic interface for metrics that can be updated associatively. -* [CountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java) -- a running total of the supplied values. Call `incr()` to increment by one, `incrBy(n)` to add/subtract the given number. - - [MultiCountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java) -- a hashmap of count metrics. +* [AssignableMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java) -- + set the metric to the explicit value you supply. Useful if it's an external value or in the case that you are already + calculating the summary statistic yourself. +* [CombinedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java) -- + generic interface for metrics that can be updated associatively. +* [CountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java) -- a running + total of the supplied values. Call `incr()` to increment by one, `incrBy(n)` to add/subtract the given number. + - [MultiCountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java) -- + a hashmap of count metrics. * [ReducedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/ReducedMetric.java) - - [MeanReducer]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java) -- track a running average of values given to its `reduce()` method. (It accepts `Double`, `Integer` or `Long` values, and maintains the internal average as a `Double`.) Despite his reputation, the MeanReducer is actually a pretty nice guy in person. - - [MultiReducedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java) -- a hashmap of reduced metrics. - -Be aware that even though `getValueAndReset` can return an object returning any object makes it very difficult for an `IMetricsConsumer` to know how to translate it into something usable. Also note that because it is sent to the `IMetricsConsumer` as a part of a tuple the values returned need to be able to be [serialized](Serialization.html) by your topology. + - [MeanReducer]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java) -- track a + running average of values given to its `reduce()` method. (It accepts `Double`, `Integer` or `Long` values, and + maintains the internal average as a `Double`.) Despite his reputation, the MeanReducer is actually a pretty nice + guy in person. + - [MultiReducedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java) -- + a hashmap of reduced metrics. + +Be aware that even though `getValueAndReset` can return an object returning any object makes it very difficult for an +`IMetricsConsumer` to know how to translate it into something usable. Also note that because it is sent to the +`IMetricsConsumer` as a part of a tuple the values returned need to be able to be [serialized](Serialization.html) by +your topology. ### Metrics Consumer -You can listen and handle the topology metrics via registering Metrics Consumer to your topology. +You can listen and handle the topology metrics via registering Metrics Consumer to your topology. To register metrics consumer to your topology, add to your topology's configuration like: ```java -conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class, 1); +conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer .class, 1); ``` -You can refer [Config#registerMetricsConsumer](javadocs/org/apache/storm/Config.html#registerMetricsConsumer-java.lang.Class-) and overloaded methods from javadoc. +You can +refer [Config#registerMetricsConsumer](javadocs/org/apache/storm/Config.html#registerMetricsConsumer-java.lang.Class-) +and overloaded methods from javadoc. Otherwise edit the storm.yaml config file: @@ -47,66 +67,92 @@ topology.metrics.consumer.register: argument: "http://example.com:8080/metrics/my-topology/" ``` -Storm adds a MetricsConsumerBolt to your topology for each class in the `topology.metrics.consumer.register` list. Each MetricsConsumerBolt subscribes to receive metrics from all tasks in the topology. The parallelism for each Bolt is set to `parallelism.hint` and `component id` for that Bolt is set to `__metrics_`. If you register the same class name more than once, postfix `#` is appended to component id. +Storm adds a MetricsConsumerBolt to your topology for each class in the `topology.metrics.consumer.register` list. Each +MetricsConsumerBolt subscribes to receive metrics from all tasks in the topology. The parallelism for each Bolt is set +to `parallelism.hint` and `component id` for that Bolt is set to `__metrics_`. If you +register the same class name more than once, postfix `#` is appended to component id. Storm provides some built-in metrics consumers for you to try out to see which metrics are provided in your topology. -* [`LoggingMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java) -- listens for all metrics and dumps them to log file with TSV (Tab Separated Values). -* [`HttpForwardingMetricsConsumer`]({{page.git-blob-base}}/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java) -- listens for all metrics and POSTs them serialized to a configured URL via HTTP. Storm also provides [`HttpForwardingMetricsServer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsServer.java) as abstract class so you can extend this class and run as a HTTP server, and handle metrics sent by HttpForwardingMetricsConsumer. - -Also, Storm exposes the interface [`IMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java) for implementing Metrics Consumer so you can create custom metrics consumers and attach to their topologies, or use other great implementation of Metrics Consumers provided by Storm community. Some of examples are [versign/storm-graphite](https://github.com/verisign/storm-graphite), and [storm-metrics-statsd](https://github.com/endgameinc/storm-metrics-statsd). - -When you implement your own metrics consumer, `argument` is passed to Object when [IMetricsConsumer#prepare](javadocs/org/apache/storm/metric/api/IMetricsConsumer.html#prepare-java.util.Map-java.lang.Object-org.apache.storm.task.TopologyContext-org.apache.storm.task.IErrorReporter-) is called, so you need to infer the Java type of configured value on yaml, and do explicit type casting. - -Please keep in mind that MetricsConsumerBolt is just a kind of Bolt, so whole throughput of the topology will go down when registered metrics consumers cannot keep up handling incoming metrics, so you may want to take care of those Bolts like normal Bolts. One of idea to avoid this is making your implementation of Metrics Consumer as `non-blocking` fashion. - +* [ + `LoggingMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java) -- + listens for all metrics and dumps them to log file with TSV (Tab Separated Values). +* [ + `HttpForwardingMetricsConsumer`]({{page.git-blob-base}}/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java) -- + listens for all metrics and POSTs them serialized to a configured URL via HTTP. Storm also provides [ + `HttpForwardingMetricsServer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsServer.java) + as abstract class so you can extend this class and run as a HTTP server, and handle metrics sent by + HttpForwardingMetricsConsumer. + +Also, Storm exposes the interface [ +`IMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java) for +implementing Metrics Consumer so you can create custom metrics consumers and attach to their topologies, or use other +great implementation of Metrics Consumers provided by Storm community. Some of examples +are [versign/storm-graphite](https://github.com/verisign/storm-graphite), +and [storm-metrics-statsd](https://github.com/endgameinc/storm-metrics-statsd). + +When you implement your own metrics consumer, `argument` is passed to Object +when [IMetricsConsumer#prepare](javadocs/org/apache/storm/metric/api/IMetricsConsumer.html#prepare-java.util.Map-java.lang.Object-org.apache.storm.task.TopologyContext-org.apache.storm.task.IErrorReporter-) +is called, so you need to infer the Java type of configured value on yaml, and do explicit type casting. + +Please keep in mind that MetricsConsumerBolt is just a kind of Bolt, so whole throughput of the topology will go down +when registered metrics consumers cannot keep up handling incoming metrics, so you may want to take care of those Bolts +like normal Bolts. One of idea to avoid this is making your implementation of Metrics Consumer as `non-blocking` +fashion. ### Build your own metric (task level) -You can measure your own metric by registering `IMetric` to Metric Registry. +You can measure your own metric by registering `IMetric` to Metric Registry. -Suppose we would like to measure execution count of Bolt#execute. Let's start with defining metric instance. CountMetric seems to fit our use case. +Suppose we would like to measure execution count of Bolt#execute. Let's start with defining metric instance. CountMetric +seems to fit our use case. ```java private transient CountMetric countMetric; ``` -Notice we define it as transient. IMertic is not Serializable so we defined as transient to avoid any serialization issues. +Notice we define it as transient. IMertic is not Serializable so we defined as transient to avoid any serialization +issues. Next, let's initialize and register the metric instance. ```java + @Override public void prepare(Map conf, TopologyContext context, OutputCollector collector) { - // other initialization here. - countMetric = new CountMetric(); - context.registerMetric("execute_count", countMetric, 60); + // other initialization here. + countMetric = new CountMetric(); + context.registerMetric("execute_count", countMetric, 60); } ``` -The meaning of first and second parameters are straightforward, metric name and instance of IMetric. Third parameter of [TopologyContext#registerMetric](javadocs/org/apache/storm/task/TopologyContext.html#registerMetric-java.lang.String-T-int-) is the period (seconds) to publish and reset the metric. +The meaning of first and second parameters are straightforward, metric name and instance of IMetric. Third parameter +of [TopologyContext#registerMetric](javadocs/org/apache/storm/task/TopologyContext.html#registerMetric-java.lang.String-T-int-) +is the period (seconds) to publish and reset the metric. Last, let's increment the value when Bolt.execute() is executed. ```java public void execute(Tuple input) { - countMetric.incr(); - // handle tuple here. + countMetric.incr(); + // handle tuple here. } ``` Note that sample rate for topology metrics is not applied to custom metrics since we're calling incr() ourselves. -Done! `countMetric.getValueAndReset()` is called every 60 seconds as we registered as period, and pair of ("execute_count", value) will be pushed to MetricsConsumer. +Done! `countMetric.getValueAndReset()` is called every 60 seconds as we registered as period, and pair of (" +execute_count", value) will be pushed to MetricsConsumer. ### Build your own metrics (worker level) -You can register your own worker level metrics by adding them to `Config.WORKER_METRICS` for all workers in cluster, or `Config.TOPOLOGY_WORKER_METRICS` for all workers in specific topology. +You can register your own worker level metrics by adding them to `Config.WORKER_METRICS` for all workers in cluster, or +`Config.TOPOLOGY_WORKER_METRICS` for all workers in specific topology. For example, we can add `worker.metrics` to storm.yaml in cluster, ```yaml -worker.metrics: +worker.metrics: metricA: "aaa.bbb.ccc.ddd.MetricA" metricB: "aaa.bbb.ccc.ddd.MetricB" ... @@ -114,9 +160,10 @@ worker.metrics: or put `Map` (metric name, metric class name) with key `Config.TOPOLOGY_WORKER_METRICS` to config map. -There are some restrictions for worker level metric instances: +There are some restrictions for worker level metric instances: -A) Metrics for worker level should be kind of gauge since it is initialized and registered from SystemBolt and not exposed to user tasks. +A) Metrics for worker level should be kind of gauge since it is initialized and registered from SystemBolt and not +exposed to user tasks. B) Metrics will be initialized with default constructor, and no injection for configuration or object will be performed. @@ -124,22 +171,35 @@ C) Bucket size (seconds) for metrics is fixed to `Config.TOPOLOGY_BUILTIN_METRIC ### Builtin Metrics -The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) instrument Storm itself. +The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) +instrument Storm itself. -[BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java). +[BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) +sets up data structures for the built-in metrics, and facade methods that the other framework components can use to +update them. The metrics themselves are calculated in the calling code -- see for example [ +`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java). #### Reporting Rate -The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` config. If you set this too low it can overload the consumers, +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` +config. If you set this too low it can overload the consumers, so please use caution when modifying it. #### Tuple Counting Metrics -There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples. +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like +emitting, transferring, acking, and failing of tuples. -In general all of these tuple count metrics are randomly sub-sampled unless otherwise stated. This means that the counts you see both on the UI and from the built in metrics are not necessarily exact. In fact by default we sample only 5% of the events and estimate the total number of events from that. The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config. Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted in the same code path as tuples are processed). This is why we have a 5% sample rate as the default. +In general all of these tuple count metrics are randomly sub-sampled unless otherwise stated. This means that the counts +you see both on the UI and from the built in metrics are not necessarily exact. In fact by default we sample only 5% of +the events and estimate the total number of events from that. The sampling percentage is configurable per topology +through the `topology.stats.sample.rate` config. Setting it to 1.0 will make the counts exact, but be aware that the +more events we sample the slower your topology will run (as the metrics are counted in the same code path as tuples are +processed). This is why we have a 5% sample rate as the default. -The tuple counting metric names contain `"${stream_name}"` or `"${upstream_component}:${stream_name}"`. The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`). The latter is used for bolt metrics that deal with incoming tuples. +The tuple counting metric names contain `"${stream_name}"` or `"${upstream_component}:${stream_name}"`. The former is +used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`). The latter is used for +bolt metrics that deal with incoming tuples. So for a word count topology the count bolt might show something like the following for the `__ack-count` metric @@ -153,59 +213,104 @@ But the spout instead would show something like the following for the `__ack-cou "__ack-count-default": 12500 ``` - ##### `__ack-count` -For bolts it is the number of incoming tuples that had the `ack` method called on them. For spouts it is the number of tuples trees that were fully acked. See Guaranteeing Message Processing[](Guaranteeing-message-processing.html) for more information about what a tuple tree is. If acking is disabled this metric is still reported, but it is not really meaningful. +For bolts it is the number of incoming tuples that had the `ack` method called on them. For spouts it is the number of +tuples trees that were fully acked. See Guaranteeing Message Processing[](Guaranteeing-message-processing.html) for more +information about what a tuple tree is. If acking is disabled this metric is still reported, but it is not really +meaningful. ##### `__fail-count` -For bolts this is the number of incoming tuples that had the `fail` method called on them. For spouts this is the number of tuple trees that failed. Tuple trees may fail from timing out or because a bolt called fail on it. The two are not separated out by this metric. +For bolts this is the number of incoming tuples that had the `fail` method called on them. For spouts this is the number +of tuple trees that failed. Tuple trees may fail from timing out or because a bolt called fail on it. The two are not +separated out by this metric. ##### `__emit-count` -This is the total number of times the `emit` method was called to send a tuple. This is the same for both bolts and spouts. +This is the total number of times the `emit` method was called to send a tuple. This is the same for both bolts and +spouts. ##### `__transfer-count` -This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`. If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted. Similarly if there are multiple down stream consumers it may be a multiple of the number emitted. The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream. +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always +match `__emit_count`. If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were +emitted. Similarly if there are multiple down stream consumers it may be a multiple of the number emitted. The grouping +also can play a role if it sends the tuple to multiple instances of a single bolt down stream. ##### `__execute-count` -This count metric is bolt specific. It counts the number of times that a bolt's `execute` method was called. +This count metric is bolt specific. It counts the number of times that a bolt's `execute` method was called. #### Tuple Latency Metrics -Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts. These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well. In all cases the latency is measured in milliseconds. +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts. These follow the +same structure as the bolt/spout maps and are sub-sampled in the same way as well. In all cases the latency is measured +in milliseconds. ##### `__complete-latency` -The complete latency is just for spouts. It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted. If acking is disabled this metric is likely to be blank or 0 for all values, and should be ignored. +The complete latency is just for spouts. It is the average amount of time it took for `ack` or `fail` to be called for a +tuple after it was emitted. If acking is disabled this metric is likely to be blank or 0 for all values, and should be +ignored. + +##### `__complete-rfc1889a-jitter` + +This metric is specific to spouts. It measures the variation (jitter) in the total completion time (end-to-end latency) +of tuples, calculated using the exponentially weighted moving average (EWMA) algorithm as defined in RFC 1889 appendix +A. +While `__complete-latency` indicates the average amount of time it took for a tuple to be fully processed by the +topology (from emission to the final ack), the jitter metric quantifies the consistency of that process. If acking is +disabled, this metric is likely to be blank or 0 and should be ignored. ##### `__execute-latency` -This is just for bolts. It is the average amount of time that the bolt spent in the call to the `execute` method. The higher this gets, the lower the throughput of tuples per bolt instance. +This is just for bolts. It is the average amount of time that the bolt spent in the call to the `execute` method. The +higher this gets, the lower the throughput of tuples per bolt instance. + +##### `__execute-rfc1889a-jitter` + +This metric is specific to bolts. It measures the variation (jitter) in the time spent within the execute method, +calculated using the exponentially weighted moving average (EWMA) algorithm as defined in RFC 1889 appendix A. +While `__execute-latency` provides the average time spent in the execute call, the jitter metric quantifies the +predictability of that execution time. It is a critical indicator of computational "smoothness". ##### `__process-latency` -This is also just for bolts. It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt. If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller. If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency. +This is also just for bolts. It is the average amount of time between when `execute` was called to start processing a +tuple, to when it was acked or failed by the bolt. If your bolt is a very simple bolt and the processing is synchronous +then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being +slightly smaller. If you are doing a join or have asynchronous processing then it may take a while for a tuple to be +acked so the process latency would be higher than the execute latency. -##### `__skipped-max-spout-ms` +##### `__process-rfc1889a-jitter` + +This metric is specific to bolts. It measures the variation (jitter) in the process latency, calculated using the +exponentially weighted moving average (EWMA) algorithm as defined in RFC 1889 appendix A. +While `__process-latency` provides the average time a tuple spends being processed, the jitter metric quantifies the +stability of that processing time. It helps identify "noisy" execution environments where processing times fluctuate +significantly, even if the average remains within acceptable limits. -This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. +##### `__skipped-max-spout-ms` +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still +outstanding. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. ##### `__skipped-backpressure-ms` -This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. This is similar to skipped-throttle-ms in Storm 1.x. +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the +topology were too full. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. +This is similar to skipped-throttle-ms in Storm 1.x. ##### `__backpressure-last-overflow-count` -This metric indicates the overflow count last time BP status was sent, with a minimum value of 1 if a task has backpressure on. +This metric indicates the overflow count last time BP status was sent, with a minimum value of 1 if a task has +backpressure on. ##### `skipped-inactive-ms` -This metric records how much time a spout was idle because the topology was deactivated. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. +This metric records how much time a spout was idle because the topology was deactivated. This is the total time in +milliseconds, not the average amount of time and is not sub-sampled. #### Error Reporting Metrics @@ -213,13 +318,16 @@ Storm also collects error reporting metrics for bolts and spouts. ##### `__reported-error-count` -This metric records how many errors were reported by a spout/bolt. It is the total number of times the `reportError` method was called. +This metric records how many errors were reported by a spout/bolt. It is the total number of times the `reportError` +method was called. #### Queue Metrics -Each bolt or spout instance in a topology has a receive queue. Each worker also has a worker transfer queue for sending messages to other workers. All of these have metrics that are reported. +Each bolt or spout instance in a topology has a receive queue. Each worker also has a worker transfer queue for sending +messages to other workers. All of these have metrics that are reported. -The receive queue metrics are reported under the `receive_queue` name. The metrics for the queue that sends messages to other workers is under the `worker-transfer-queue` metric name for the system bolt (`__system`). +The receive queue metrics are reported under the `receive_queue` name. The metrics for the queue that sends messages to +other workers is under the `worker-transfer-queue` metric name for the system bolt (`__system`). These queues report the following metrics: @@ -236,23 +344,33 @@ These queues report the following metrics: } ``` -`arrival_rate_secs` is an estimation of the number of tuples that are inserted into the queue in one second, although it is actually the dequeue rate. -The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each tuple sits in the queue before it is processed. +`arrival_rate_secs` is an estimation of the number of tuples that are inserted into the queue in one second, although it +is actually the dequeue rate. +The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each tuple sits in +the queue before it is processed. -The queue has a set maximum number of entries. If the regular queue fills up an overflow queue takes over. The number of tuples stored in this overflow section are represented by the `overflow` metric. Note that an overflow queue is only used for executors to receive tuples from remote workers. It doesn't apply to intra-worker tuple transfer. +The queue has a set maximum number of entries. If the regular queue fills up an overflow queue takes over. The number of +tuples stored in this overflow section are represented by the `overflow` metric. Note that an overflow queue is only +used for executors to receive tuples from remote workers. It doesn't apply to intra-worker tuple transfer. -`capacity` is the maximum number of entries in the queue. `population` is the number of entries currently filled in the queue. 'pct_full' tracks the percentage of capacity in use. +`capacity` is the maximum number of entries in the queue. `population` is the number of entries currently filled in the +queue. 'pct_full' tracks the percentage of capacity in use. -'insert_failures' tracks the number of failures inserting into the queue. 'dropped_messages' tracks messages dropped due to the overflow queue being full. +'insert_failures' tracks the number of failures inserting into the queue. 'dropped_messages' tracks messages dropped due +to the overflow queue being full. #### System Bolt (Worker) Metrics -The System Bolt `__system` provides lots of metrics for different worker wide things. The one metric not described here is the `__transfer` queue metric, because it fits with the other disruptor metrics described above. +The System Bolt `__system` provides lots of metrics for different worker wide things. The one metric not described here +is the `__transfer` queue metric, because it fits with the other disruptor metrics described above. -Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for it. +Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for +it. ##### Receive (NettyServer) -`__recv-iconnection` reports stats for the netty server on the worker. This is what gets messages from other workers. It is of the form + +`__recv-iconnection` reports stats for the netty server on the worker. This is what gets messages from other workers. It +is of the form ``` { @@ -263,51 +381,65 @@ Be aware that the `__system` bolt is an actual bolt so regular bolt metrics desc } ``` -`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the bolts/spouts. That is no longer the case and the value can be ignored. -`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this worker. +`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the +bolts/spouts. That is no longer the case and the value can be ignored. +`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this +worker. ##### Send (Netty Client) -The `__send-iconnection` metrics report information about all of the clients for this worker. They are named __send-iconnection-METRIC_TYPE-HOST:PORT for a given Client that is -connected to a worker with the given host/port. These metrics can be disabled by setting topology.enable.send.iconnection.metrics to false. +The `__send-iconnection` metrics report information about all of the clients for this worker. They are named __ +send-iconnection-METRIC_TYPE-HOST:PORT for a given Client that is +connected to a worker with the given host/port. These metrics can be disabled by setting +topology.enable.send.iconnection.metrics to false. The metric types reported for each client are: - * `reconnects` the number of reconnections that have happened. - * `pending` the number of messages that have not been sent. (This corresponds to messages, not tuples) - * `sent` the number of messages that have been sent. (This is messages not tuples) - * `lostOnSend`. This is the number of messages that were lost because of connection issues. (This is messages not tuples). +* `reconnects` the number of reconnections that have happened. +* `pending` the number of messages that have not been sent. (This corresponds to messages, not tuples) +* `sent` the number of messages that have been sent. (This is messages not tuples) +* `lostOnSend`. This is the number of messages that were lost because of connection issues. (This is messages not + tuples). ##### JVM Memory -JVM memory usage is reported through `memory.non-heap` for off heap memory, `memory.heap` for on heap memory and `memory.total` for combined values. These values come from the [MemoryUsage](https://docs.oracle.com/javase/8/docs/api/index.html?java/lang/management/MemoryUsage.html) mxbean. Each of the metrics are reported as a map with the following keys, and values returned by the corresponding java code. +JVM memory usage is reported through `memory.non-heap` for off heap memory, `memory.heap` for on heap memory and +`memory.total` for combined values. These values come from +the [MemoryUsage](https://docs.oracle.com/javase/8/docs/api/index.html?java/lang/management/MemoryUsage.html) mxbean. +Each of the metrics are reported as a map with the following keys, and values returned by the corresponding java code. -| Key | Corresponding Code | -|--------|--------------------| -| `max` | `memUsage.getMax()` | -| `committed` | `memUsage.getCommitted()` | -| `init` | `memUsage.getInit()` | -| `used` | `memUsage.getUsed()` | -| `usage` | `Ratio.of(memUsage.getUsed(), memUsage.getMax())` | +| Key | Corresponding Code | +|-------------|---------------------------------------------------| +| `max` | `memUsage.getMax()` | +| `committed` | `memUsage.getCommitted()` | +| `init` | `memUsage.getInit()` | +| `used` | `memUsage.getUsed()` | +| `usage` | `Ratio.of(memUsage.getUsed(), memUsage.getMax())` | ##### JVM Garbage Collection -The exact GC metric name depends on the garbage collector that your worker uses. The data is all collected from `ManagementFactory.getGarbageCollectorMXBeans()` and the name of the metrics is `"GC"` followed by the name of the returned bean with white space removed. The reported metrics are just +The exact GC metric name depends on the garbage collector that your worker uses. The data is all collected from +`ManagementFactory.getGarbageCollectorMXBeans()` and the name of the metrics is `"GC"` followed by the name of the +returned bean with white space removed. The reported metrics are just * `count` the number of gc events that happened and -* `time` the total number of milliseconds that were spent doing gc. +* `time` the total number of milliseconds that were spent doing gc. -Please refer to the [JVM documentation](https://docs.oracle.com/javase/8/docs/api/java/lang/management/ManagementFactory.html#getGarbageCollectorMXBeans--) for more details. +Please refer to +the [JVM documentation](https://docs.oracle.com/javase/8/docs/api/java/lang/management/ManagementFactory.html#getGarbageCollectorMXBeans--) +for more details. ##### JVM Misc -* There are metrics prefixed with `threads` providing the number of threads, daemon threads, blocked and deadlocked threads. +* There are metrics prefixed with `threads` providing the number of threads, daemon threads, blocked and deadlocked + threads. ##### Other worker metrics * `doHeartbeat-calls` is a meter that indicates the rate the worker is performing heartbeats. -* `newWorkerEvent` is 1 when a worker is first started and 0 all other times. This can be used to tell when a worker has crashed and is restarted. +* `newWorkerEvent` is 1 when a worker is first started and 0 all other times. This can be used to tell when a worker has + crashed and is restarted. * `startTimeSecs` is when the worker started in seconds since the epoch * `uptimeSecs` reports the number of seconds the worker has been up for -* `workerCpuUsage` reports the cpu usage of the worker as a percentage of cores. 1.0 indicates 1 cpu core. +* `workerCpuUsage` reports the cpu usage of the worker as a percentage of cores. 1.0 indicates 1 cpu core. From 946715db3d4f775969d99ad51cd9281972354186 Mon Sep 17 00:00:00 2001 From: Gianluca Graziadei Date: Tue, 5 May 2026 19:02:30 +0200 Subject: [PATCH 5/8] format --- conf/defaults.yaml | 46 ++++---- docs/Metrics.md | 277 ++++++++++++++------------------------------- 2 files changed, 111 insertions(+), 212 deletions(-) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 4b0313d1d6f..d92da7b4658 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -25,7 +25,7 @@ java.library.path: "/usr/local/lib:/opt/local/lib:/usr/lib:/usr/lib64" storm.local.dir: "storm-local" storm.log4j2.conf.dir: "log4j2" storm.zookeeper.servers: - - "localhost" + - "localhost" storm.zookeeper.port: 2181 storm.zookeeper.root: "/storm" storm.zookeeper.session.timeout: 20000 @@ -52,7 +52,7 @@ storm.nimbus.retry.intervalceiling.millis: 60000 storm.nimbus.zookeeper.acls.check: true storm.nimbus.zookeeper.acls.fixup: true -storm.auth.simple-white-list.users: [ ] +storm.auth.simple-white-list.users: [] storm.cluster.state.store: "org.apache.storm.cluster.ZKStateStorageFactory" storm.meta.serialization.delegate: "org.apache.storm.serialization.GzipThriftSerializationDelegate" storm.codedistributor.class: "org.apache.storm.codedistributor.LocalFileSystemCodeDistributor" @@ -62,7 +62,7 @@ storm.health.check.timeout.ms: 5000 storm.disable.symlinks: false ### nimbus.* configs are for the master -nimbus.seeds: [ "localhost" ] +nimbus.seeds : ["localhost"] nimbus.thrift.port: 6627 nimbus.thrift.threads: 64 nimbus.thrift.max_buffer_size: 1048576 @@ -163,10 +163,10 @@ storm.blobstore.acl.validation.enabled: false ### supervisor.* configs are for node supervisors # Define the amount of workers that can be run on this machine. Each worker is assigned a port to use for communication supervisor.slots.ports: - - 6700 - - 6701 - - 6702 - - 6703 + - 6700 + - 6701 + - 6702 + - 6703 supervisor.childopts: "-Xmx256m" supervisor.run.worker.as.user: false #how long supervisor will wait to ensure that a worker process is started @@ -184,8 +184,8 @@ supervisor.worker.heartbeats.max.timeout.secs: 600 #For topology configurable heartbeat timeout, maximum allowed heartbeat timeout. worker.max.timeout.secs: 600 supervisor.enable: true -supervisor.supervisors: [ ] -supervisor.supervisors.commands: [ ] +supervisor.supervisors: [] +supervisor.supervisors.commands: [] supervisor.memory.capacity.mb: 4096.0 #By convention 1 cpu core should be about 100, but this can be adjusted if needed # using 100 makes it simple to set the desired value to the capacity measurement @@ -288,16 +288,16 @@ topology.worker.shared.thread.pool.size: 4 # Spout Wait Strategy - employed when there is no data to produce topology.spout.wait.strategy: "org.apache.storm.policy.WaitStrategyProgressive" -topology.spout.wait.park.microsec: 100 # park time for org.apache.storm.policy.WaitStrategyPark. Busy spins if set to 0. +topology.spout.wait.park.microsec : 100 # park time for org.apache.storm.policy.WaitStrategyPark. Busy spins if set to 0. topology.spout.wait.progressive.level1.count: 0 # number of iterations to spend in level 1 [no sleep] of WaitStrategyProgressive, before progressing to level 2 topology.spout.wait.progressive.level2.count: 0 # number of iterations to spend in level 2 [parkNanos(1)] of WaitStrategyProgressive, before progressing to level 3 topology.spout.wait.progressive.level3.sleep.millis: 1 # sleep duration for idling iterations in level 3 of WaitStrategyProgressive # Bolt Wait Strategy - employed when there is no data in its receive buffer to process -topology.bolt.wait.strategy: "org.apache.storm.policy.WaitStrategyProgressive" +topology.bolt.wait.strategy : "org.apache.storm.policy.WaitStrategyProgressive" -topology.bolt.wait.park.microsec: 100 # park time for org.apache.storm.policy.WaitStrategyPark. Busy spins if set to 0. +topology.bolt.wait.park.microsec : 100 # park time for org.apache.storm.policy.WaitStrategyPark. Busy spins if set to 0. topology.bolt.wait.progressive.level1.count: 1 # number of iterations to spend in level 1 [no sleep] of WaitStrategyProgressive, before progressing to level 2 topology.bolt.wait.progressive.level2.count: 1000 # number of iterations to spend in level 2 [parkNanos(1)] of WaitStrategyProgressive, before progressing to level 3 @@ -364,7 +364,7 @@ blacklist.scheduler.assume.supervisor.bad.based.on.bad.slot: true dev.zookeeper.path: "/tmp/dev-storm-zookeeper" -pacemaker.servers: [ ] +pacemaker.servers: [] pacemaker.port: 6699 pacemaker.base.threads: 10 pacemaker.max.threads: 50 @@ -372,12 +372,12 @@ pacemaker.client.max.threads: 2 pacemaker.thread.timeout: 10 pacemaker.childopts: "-Xmx1024m" pacemaker.auth.method: "NONE" -pacemaker.kerberos.users: [ ] +pacemaker.kerberos.users: [] pacemaker.thrift.message.size.max: 10485760 #default storm daemon metrics reporter plugins storm.daemon.metrics.reporter.plugins: - - "org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter" + - "org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter" storm.daemon.metrics.reporter.interval.secs: 10 storm.metricstore.class: "org.apache.storm.metricstore.rocksdb.RocksDbStore" @@ -400,8 +400,8 @@ storm.cgroup.inherit.cpuset.configs: false # Configs for CGroup support storm.cgroup.hierarchy.dir: "/cgroup/storm_resources" storm.cgroup.resources: - - "cpu" - - "memory" + - "cpu" + - "memory" storm.cgroup.hierarchy.name: "storm" storm.supervisor.cgroup.rootdir: "storm" storm.cgroup.cgexec.cmd: "/bin/cgexec" @@ -420,12 +420,12 @@ storm.worker.min.cpu.pcore.percent: 0.0 storm.topology.classpath.beginning.enabled: false worker.metrics: - "CGroupMemory": "org.apache.storm.metrics2.cgroup.CGroupMemoryUsage" - "CGroupMemoryLimit": "org.apache.storm.metrics2.cgroup.CGroupMemoryLimit" - "CGroupCpu": "org.apache.storm.metrics2.cgroup.CGroupCpu" - "CGroupCpuGuarantee": "org.apache.storm.metrics2.cgroup.CGroupCpuGuarantee" - "CGroupCpuGuaranteeByCfsQuota": "org.apache.storm.metrics2.cgroup.CGroupCpuGuaranteeByCfsQuota" - "CGroupCpuStat": "org.apache.storm.metrics2.cgroup.CGroupCpuStat" + "CGroupMemory": "org.apache.storm.metrics2.cgroup.CGroupMemoryUsage" + "CGroupMemoryLimit": "org.apache.storm.metrics2.cgroup.CGroupMemoryLimit" + "CGroupCpu": "org.apache.storm.metrics2.cgroup.CGroupCpu" + "CGroupCpuGuarantee": "org.apache.storm.metrics2.cgroup.CGroupCpuGuarantee" + "CGroupCpuGuaranteeByCfsQuota": "org.apache.storm.metrics2.cgroup.CGroupCpuGuaranteeByCfsQuota" + "CGroupCpuStat": "org.apache.storm.metrics2.cgroup.CGroupCpuStat" # The number of buckets for running statistics num.stat.buckets: 20 diff --git a/docs/Metrics.md b/docs/Metrics.md index c3cff6eba91..6c65c0419dc 100644 --- a/docs/Metrics.md +++ b/docs/Metrics.md @@ -3,58 +3,38 @@ title: Storm Metrics layout: documentation documentation: true --- - Storm exposes a metrics interface to report summary statistics across the full topology. -The numbers you see on the UI come from some of these built in metrics, but are reported through the worker heartbeats -instead of through the IMetricsConsumer described below. +The numbers you see on the UI come from some of these built in metrics, but are reported through the worker heartbeats instead of through the IMetricsConsumer described below. If you are looking for cluster wide monitoring please see [Cluster Metrics](ClusterMetrics.html). ### Metric Types -Metrics have to implement [ -`IMetric`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java) which contains just one -method, `getValueAndReset` -- do any remaining work to find the summary value, and reset back to an initial state. For -example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values -back to zero. +Metrics have to implement [`IMetric`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java) which contains just one method, `getValueAndReset` -- do any remaining work to find the summary value, and reset back to an initial state. For example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values back to zero. Storm gives you these metric types: -* [AssignableMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java) -- - set the metric to the explicit value you supply. Useful if it's an external value or in the case that you are already - calculating the summary statistic yourself. -* [CombinedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java) -- - generic interface for metrics that can be updated associatively. -* [CountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java) -- a running - total of the supplied values. Call `incr()` to increment by one, `incrBy(n)` to add/subtract the given number. - - [MultiCountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java) -- - a hashmap of count metrics. +* [AssignableMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java) -- set the metric to the explicit value you supply. Useful if it's an external value or in the case that you are already calculating the summary statistic yourself. +* [CombinedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java) -- generic interface for metrics that can be updated associatively. +* [CountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java) -- a running total of the supplied values. Call `incr()` to increment by one, `incrBy(n)` to add/subtract the given number. + - [MultiCountMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java) -- a hashmap of count metrics. * [ReducedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/ReducedMetric.java) - - [MeanReducer]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java) -- track a - running average of values given to its `reduce()` method. (It accepts `Double`, `Integer` or `Long` values, and - maintains the internal average as a `Double`.) Despite his reputation, the MeanReducer is actually a pretty nice - guy in person. - - [MultiReducedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java) -- - a hashmap of reduced metrics. - -Be aware that even though `getValueAndReset` can return an object returning any object makes it very difficult for an -`IMetricsConsumer` to know how to translate it into something usable. Also note that because it is sent to the -`IMetricsConsumer` as a part of a tuple the values returned need to be able to be [serialized](Serialization.html) by -your topology. + - [MeanReducer]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java) -- track a running average of values given to its `reduce()` method. (It accepts `Double`, `Integer` or `Long` values, and maintains the internal average as a `Double`.) Despite his reputation, the MeanReducer is actually a pretty nice guy in person. + - [MultiReducedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java) -- a hashmap of reduced metrics. + +Be aware that even though `getValueAndReset` can return an object returning any object makes it very difficult for an `IMetricsConsumer` to know how to translate it into something usable. Also note that because it is sent to the `IMetricsConsumer` as a part of a tuple the values returned need to be able to be [serialized](Serialization.html) by your topology. ### Metrics Consumer -You can listen and handle the topology metrics via registering Metrics Consumer to your topology. +You can listen and handle the topology metrics via registering Metrics Consumer to your topology. To register metrics consumer to your topology, add to your topology's configuration like: ```java -conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer .class, 1); +conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class, 1); ``` -You can -refer [Config#registerMetricsConsumer](javadocs/org/apache/storm/Config.html#registerMetricsConsumer-java.lang.Class-) -and overloaded methods from javadoc. +You can refer [Config#registerMetricsConsumer](javadocs/org/apache/storm/Config.html#registerMetricsConsumer-java.lang.Class-) and overloaded methods from javadoc. Otherwise edit the storm.yaml config file: @@ -67,92 +47,66 @@ topology.metrics.consumer.register: argument: "http://example.com:8080/metrics/my-topology/" ``` -Storm adds a MetricsConsumerBolt to your topology for each class in the `topology.metrics.consumer.register` list. Each -MetricsConsumerBolt subscribes to receive metrics from all tasks in the topology. The parallelism for each Bolt is set -to `parallelism.hint` and `component id` for that Bolt is set to `__metrics_`. If you -register the same class name more than once, postfix `#` is appended to component id. +Storm adds a MetricsConsumerBolt to your topology for each class in the `topology.metrics.consumer.register` list. Each MetricsConsumerBolt subscribes to receive metrics from all tasks in the topology. The parallelism for each Bolt is set to `parallelism.hint` and `component id` for that Bolt is set to `__metrics_`. If you register the same class name more than once, postfix `#` is appended to component id. Storm provides some built-in metrics consumers for you to try out to see which metrics are provided in your topology. -* [ - `LoggingMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java) -- - listens for all metrics and dumps them to log file with TSV (Tab Separated Values). -* [ - `HttpForwardingMetricsConsumer`]({{page.git-blob-base}}/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java) -- - listens for all metrics and POSTs them serialized to a configured URL via HTTP. Storm also provides [ - `HttpForwardingMetricsServer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsServer.java) - as abstract class so you can extend this class and run as a HTTP server, and handle metrics sent by - HttpForwardingMetricsConsumer. - -Also, Storm exposes the interface [ -`IMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java) for -implementing Metrics Consumer so you can create custom metrics consumers and attach to their topologies, or use other -great implementation of Metrics Consumers provided by Storm community. Some of examples -are [versign/storm-graphite](https://github.com/verisign/storm-graphite), -and [storm-metrics-statsd](https://github.com/endgameinc/storm-metrics-statsd). - -When you implement your own metrics consumer, `argument` is passed to Object -when [IMetricsConsumer#prepare](javadocs/org/apache/storm/metric/api/IMetricsConsumer.html#prepare-java.util.Map-java.lang.Object-org.apache.storm.task.TopologyContext-org.apache.storm.task.IErrorReporter-) -is called, so you need to infer the Java type of configured value on yaml, and do explicit type casting. - -Please keep in mind that MetricsConsumerBolt is just a kind of Bolt, so whole throughput of the topology will go down -when registered metrics consumers cannot keep up handling incoming metrics, so you may want to take care of those Bolts -like normal Bolts. One of idea to avoid this is making your implementation of Metrics Consumer as `non-blocking` -fashion. +* [`LoggingMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java) -- listens for all metrics and dumps them to log file with TSV (Tab Separated Values). +* [`HttpForwardingMetricsConsumer`]({{page.git-blob-base}}/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java) -- listens for all metrics and POSTs them serialized to a configured URL via HTTP. Storm also provides [`HttpForwardingMetricsServer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsServer.java) as abstract class so you can extend this class and run as a HTTP server, and handle metrics sent by HttpForwardingMetricsConsumer. + +Also, Storm exposes the interface [`IMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java) for implementing Metrics Consumer so you can create custom metrics consumers and attach to their topologies, or use other great implementation of Metrics Consumers provided by Storm community. Some of examples are [versign/storm-graphite](https://github.com/verisign/storm-graphite), and [storm-metrics-statsd](https://github.com/endgameinc/storm-metrics-statsd). + +When you implement your own metrics consumer, `argument` is passed to Object when [IMetricsConsumer#prepare](javadocs/org/apache/storm/metric/api/IMetricsConsumer.html#prepare-java.util.Map-java.lang.Object-org.apache.storm.task.TopologyContext-org.apache.storm.task.IErrorReporter-) is called, so you need to infer the Java type of configured value on yaml, and do explicit type casting. + +Please keep in mind that MetricsConsumerBolt is just a kind of Bolt, so whole throughput of the topology will go down when registered metrics consumers cannot keep up handling incoming metrics, so you may want to take care of those Bolts like normal Bolts. One of idea to avoid this is making your implementation of Metrics Consumer as `non-blocking` fashion. + ### Build your own metric (task level) -You can measure your own metric by registering `IMetric` to Metric Registry. +You can measure your own metric by registering `IMetric` to Metric Registry. -Suppose we would like to measure execution count of Bolt#execute. Let's start with defining metric instance. CountMetric -seems to fit our use case. +Suppose we would like to measure execution count of Bolt#execute. Let's start with defining metric instance. CountMetric seems to fit our use case. ```java private transient CountMetric countMetric; ``` -Notice we define it as transient. IMertic is not Serializable so we defined as transient to avoid any serialization -issues. +Notice we define it as transient. IMertic is not Serializable so we defined as transient to avoid any serialization issues. Next, let's initialize and register the metric instance. ```java - @Override public void prepare(Map conf, TopologyContext context, OutputCollector collector) { - // other initialization here. - countMetric = new CountMetric(); - context.registerMetric("execute_count", countMetric, 60); + // other initialization here. + countMetric = new CountMetric(); + context.registerMetric("execute_count", countMetric, 60); } ``` -The meaning of first and second parameters are straightforward, metric name and instance of IMetric. Third parameter -of [TopologyContext#registerMetric](javadocs/org/apache/storm/task/TopologyContext.html#registerMetric-java.lang.String-T-int-) -is the period (seconds) to publish and reset the metric. +The meaning of first and second parameters are straightforward, metric name and instance of IMetric. Third parameter of [TopologyContext#registerMetric](javadocs/org/apache/storm/task/TopologyContext.html#registerMetric-java.lang.String-T-int-) is the period (seconds) to publish and reset the metric. Last, let's increment the value when Bolt.execute() is executed. ```java public void execute(Tuple input) { - countMetric.incr(); - // handle tuple here. + countMetric.incr(); + // handle tuple here. } ``` Note that sample rate for topology metrics is not applied to custom metrics since we're calling incr() ourselves. -Done! `countMetric.getValueAndReset()` is called every 60 seconds as we registered as period, and pair of (" -execute_count", value) will be pushed to MetricsConsumer. +Done! `countMetric.getValueAndReset()` is called every 60 seconds as we registered as period, and pair of ("execute_count", value) will be pushed to MetricsConsumer. ### Build your own metrics (worker level) -You can register your own worker level metrics by adding them to `Config.WORKER_METRICS` for all workers in cluster, or -`Config.TOPOLOGY_WORKER_METRICS` for all workers in specific topology. +You can register your own worker level metrics by adding them to `Config.WORKER_METRICS` for all workers in cluster, or `Config.TOPOLOGY_WORKER_METRICS` for all workers in specific topology. For example, we can add `worker.metrics` to storm.yaml in cluster, ```yaml -worker.metrics: +worker.metrics: metricA: "aaa.bbb.ccc.ddd.MetricA" metricB: "aaa.bbb.ccc.ddd.MetricB" ... @@ -160,10 +114,9 @@ worker.metrics: or put `Map` (metric name, metric class name) with key `Config.TOPOLOGY_WORKER_METRICS` to config map. -There are some restrictions for worker level metric instances: +There are some restrictions for worker level metric instances: -A) Metrics for worker level should be kind of gauge since it is initialized and registered from SystemBolt and not -exposed to user tasks. +A) Metrics for worker level should be kind of gauge since it is initialized and registered from SystemBolt and not exposed to user tasks. B) Metrics will be initialized with default constructor, and no injection for configuration or object will be performed. @@ -171,35 +124,22 @@ C) Bucket size (seconds) for metrics is fixed to `Config.TOPOLOGY_BUILTIN_METRIC ### Builtin Metrics -The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) -instrument Storm itself. +The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) instrument Storm itself. -[BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) -sets up data structures for the built-in metrics, and facade methods that the other framework components can use to -update them. The metrics themselves are calculated in the calling code -- see for example [ -`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java). +[BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java). #### Reporting Rate -The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` -config. If you set this too low it can overload the consumers, +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` config. If you set this too low it can overload the consumers, so please use caution when modifying it. #### Tuple Counting Metrics -There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like -emitting, transferring, acking, and failing of tuples. +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples. -In general all of these tuple count metrics are randomly sub-sampled unless otherwise stated. This means that the counts -you see both on the UI and from the built in metrics are not necessarily exact. In fact by default we sample only 5% of -the events and estimate the total number of events from that. The sampling percentage is configurable per topology -through the `topology.stats.sample.rate` config. Setting it to 1.0 will make the counts exact, but be aware that the -more events we sample the slower your topology will run (as the metrics are counted in the same code path as tuples are -processed). This is why we have a 5% sample rate as the default. +In general all of these tuple count metrics are randomly sub-sampled unless otherwise stated. This means that the counts you see both on the UI and from the built in metrics are not necessarily exact. In fact by default we sample only 5% of the events and estimate the total number of events from that. The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config. Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted in the same code path as tuples are processed). This is why we have a 5% sample rate as the default. -The tuple counting metric names contain `"${stream_name}"` or `"${upstream_component}:${stream_name}"`. The former is -used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`). The latter is used for -bolt metrics that deal with incoming tuples. +The tuple counting metric names contain `"${stream_name}"` or `"${upstream_component}:${stream_name}"`. The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`). The latter is used for bolt metrics that deal with incoming tuples. So for a word count topology the count bolt might show something like the following for the `__ack-count` metric @@ -213,40 +153,30 @@ But the spout instead would show something like the following for the `__ack-cou "__ack-count-default": 12500 ``` + ##### `__ack-count` -For bolts it is the number of incoming tuples that had the `ack` method called on them. For spouts it is the number of -tuples trees that were fully acked. See Guaranteeing Message Processing[](Guaranteeing-message-processing.html) for more -information about what a tuple tree is. If acking is disabled this metric is still reported, but it is not really -meaningful. +For bolts it is the number of incoming tuples that had the `ack` method called on them. For spouts it is the number of tuples trees that were fully acked. See Guaranteeing Message Processing[](Guaranteeing-message-processing.html) for more information about what a tuple tree is. If acking is disabled this metric is still reported, but it is not really meaningful. ##### `__fail-count` -For bolts this is the number of incoming tuples that had the `fail` method called on them. For spouts this is the number -of tuple trees that failed. Tuple trees may fail from timing out or because a bolt called fail on it. The two are not -separated out by this metric. +For bolts this is the number of incoming tuples that had the `fail` method called on them. For spouts this is the number of tuple trees that failed. Tuple trees may fail from timing out or because a bolt called fail on it. The two are not separated out by this metric. ##### `__emit-count` -This is the total number of times the `emit` method was called to send a tuple. This is the same for both bolts and -spouts. +This is the total number of times the `emit` method was called to send a tuple. This is the same for both bolts and spouts. ##### `__transfer-count` -This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always -match `__emit_count`. If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were -emitted. Similarly if there are multiple down stream consumers it may be a multiple of the number emitted. The grouping -also can play a role if it sends the tuple to multiple instances of a single bolt down stream. +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`. If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted. Similarly if there are multiple down stream consumers it may be a multiple of the number emitted. The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream. ##### `__execute-count` -This count metric is bolt specific. It counts the number of times that a bolt's `execute` method was called. +This count metric is bolt specific. It counts the number of times that a bolt's `execute` method was called. #### Tuple Latency Metrics -Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts. These follow the -same structure as the bolt/spout maps and are sub-sampled in the same way as well. In all cases the latency is measured -in milliseconds. +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts. These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well. In all cases the latency is measured in milliseconds. ##### `__complete-latency` @@ -293,24 +223,20 @@ significantly, even if the average remains within acceptable limits. ##### `__skipped-max-spout-ms` -This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still -outstanding. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. + ##### `__skipped-backpressure-ms` -This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the -topology were too full. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. -This is similar to skipped-throttle-ms in Storm 1.x. +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. This is similar to skipped-throttle-ms in Storm 1.x. ##### `__backpressure-last-overflow-count` -This metric indicates the overflow count last time BP status was sent, with a minimum value of 1 if a task has -backpressure on. +This metric indicates the overflow count last time BP status was sent, with a minimum value of 1 if a task has backpressure on. ##### `skipped-inactive-ms` -This metric records how much time a spout was idle because the topology was deactivated. This is the total time in -milliseconds, not the average amount of time and is not sub-sampled. +This metric records how much time a spout was idle because the topology was deactivated. This is the total time in milliseconds, not the average amount of time and is not sub-sampled. #### Error Reporting Metrics @@ -318,16 +244,13 @@ Storm also collects error reporting metrics for bolts and spouts. ##### `__reported-error-count` -This metric records how many errors were reported by a spout/bolt. It is the total number of times the `reportError` -method was called. +This metric records how many errors were reported by a spout/bolt. It is the total number of times the `reportError` method was called. #### Queue Metrics -Each bolt or spout instance in a topology has a receive queue. Each worker also has a worker transfer queue for sending -messages to other workers. All of these have metrics that are reported. +Each bolt or spout instance in a topology has a receive queue. Each worker also has a worker transfer queue for sending messages to other workers. All of these have metrics that are reported. -The receive queue metrics are reported under the `receive_queue` name. The metrics for the queue that sends messages to -other workers is under the `worker-transfer-queue` metric name for the system bolt (`__system`). +The receive queue metrics are reported under the `receive_queue` name. The metrics for the queue that sends messages to other workers is under the `worker-transfer-queue` metric name for the system bolt (`__system`). These queues report the following metrics: @@ -344,33 +267,23 @@ These queues report the following metrics: } ``` -`arrival_rate_secs` is an estimation of the number of tuples that are inserted into the queue in one second, although it -is actually the dequeue rate. -The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each tuple sits in -the queue before it is processed. +`arrival_rate_secs` is an estimation of the number of tuples that are inserted into the queue in one second, although it is actually the dequeue rate. +The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each tuple sits in the queue before it is processed. -The queue has a set maximum number of entries. If the regular queue fills up an overflow queue takes over. The number of -tuples stored in this overflow section are represented by the `overflow` metric. Note that an overflow queue is only -used for executors to receive tuples from remote workers. It doesn't apply to intra-worker tuple transfer. +The queue has a set maximum number of entries. If the regular queue fills up an overflow queue takes over. The number of tuples stored in this overflow section are represented by the `overflow` metric. Note that an overflow queue is only used for executors to receive tuples from remote workers. It doesn't apply to intra-worker tuple transfer. -`capacity` is the maximum number of entries in the queue. `population` is the number of entries currently filled in the -queue. 'pct_full' tracks the percentage of capacity in use. +`capacity` is the maximum number of entries in the queue. `population` is the number of entries currently filled in the queue. 'pct_full' tracks the percentage of capacity in use. -'insert_failures' tracks the number of failures inserting into the queue. 'dropped_messages' tracks messages dropped due -to the overflow queue being full. +'insert_failures' tracks the number of failures inserting into the queue. 'dropped_messages' tracks messages dropped due to the overflow queue being full. #### System Bolt (Worker) Metrics -The System Bolt `__system` provides lots of metrics for different worker wide things. The one metric not described here -is the `__transfer` queue metric, because it fits with the other disruptor metrics described above. +The System Bolt `__system` provides lots of metrics for different worker wide things. The one metric not described here is the `__transfer` queue metric, because it fits with the other disruptor metrics described above. -Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for -it. +Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for it. ##### Receive (NettyServer) - -`__recv-iconnection` reports stats for the netty server on the worker. This is what gets messages from other workers. It -is of the form +`__recv-iconnection` reports stats for the netty server on the worker. This is what gets messages from other workers. It is of the form ``` { @@ -381,65 +294,51 @@ is of the form } ``` -`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the -bolts/spouts. That is no longer the case and the value can be ignored. -`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this -worker. +`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the bolts/spouts. That is no longer the case and the value can be ignored. +`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this worker. ##### Send (Netty Client) -The `__send-iconnection` metrics report information about all of the clients for this worker. They are named __ -send-iconnection-METRIC_TYPE-HOST:PORT for a given Client that is -connected to a worker with the given host/port. These metrics can be disabled by setting -topology.enable.send.iconnection.metrics to false. +The `__send-iconnection` metrics report information about all of the clients for this worker. They are named __send-iconnection-METRIC_TYPE-HOST:PORT for a given Client that is +connected to a worker with the given host/port. These metrics can be disabled by setting topology.enable.send.iconnection.metrics to false. The metric types reported for each client are: -* `reconnects` the number of reconnections that have happened. -* `pending` the number of messages that have not been sent. (This corresponds to messages, not tuples) -* `sent` the number of messages that have been sent. (This is messages not tuples) -* `lostOnSend`. This is the number of messages that were lost because of connection issues. (This is messages not - tuples). + * `reconnects` the number of reconnections that have happened. + * `pending` the number of messages that have not been sent. (This corresponds to messages, not tuples) + * `sent` the number of messages that have been sent. (This is messages not tuples) + * `lostOnSend`. This is the number of messages that were lost because of connection issues. (This is messages not tuples). ##### JVM Memory -JVM memory usage is reported through `memory.non-heap` for off heap memory, `memory.heap` for on heap memory and -`memory.total` for combined values. These values come from -the [MemoryUsage](https://docs.oracle.com/javase/8/docs/api/index.html?java/lang/management/MemoryUsage.html) mxbean. -Each of the metrics are reported as a map with the following keys, and values returned by the corresponding java code. +JVM memory usage is reported through `memory.non-heap` for off heap memory, `memory.heap` for on heap memory and `memory.total` for combined values. These values come from the [MemoryUsage](https://docs.oracle.com/javase/8/docs/api/index.html?java/lang/management/MemoryUsage.html) mxbean. Each of the metrics are reported as a map with the following keys, and values returned by the corresponding java code. -| Key | Corresponding Code | -|-------------|---------------------------------------------------| -| `max` | `memUsage.getMax()` | -| `committed` | `memUsage.getCommitted()` | -| `init` | `memUsage.getInit()` | -| `used` | `memUsage.getUsed()` | -| `usage` | `Ratio.of(memUsage.getUsed(), memUsage.getMax())` | +| Key | Corresponding Code | +|--------|--------------------| +| `max` | `memUsage.getMax()` | +| `committed` | `memUsage.getCommitted()` | +| `init` | `memUsage.getInit()` | +| `used` | `memUsage.getUsed()` | +| `usage` | `Ratio.of(memUsage.getUsed(), memUsage.getMax())` | ##### JVM Garbage Collection -The exact GC metric name depends on the garbage collector that your worker uses. The data is all collected from -`ManagementFactory.getGarbageCollectorMXBeans()` and the name of the metrics is `"GC"` followed by the name of the -returned bean with white space removed. The reported metrics are just +The exact GC metric name depends on the garbage collector that your worker uses. The data is all collected from `ManagementFactory.getGarbageCollectorMXBeans()` and the name of the metrics is `"GC"` followed by the name of the returned bean with white space removed. The reported metrics are just * `count` the number of gc events that happened and -* `time` the total number of milliseconds that were spent doing gc. +* `time` the total number of milliseconds that were spent doing gc. -Please refer to -the [JVM documentation](https://docs.oracle.com/javase/8/docs/api/java/lang/management/ManagementFactory.html#getGarbageCollectorMXBeans--) -for more details. +Please refer to the [JVM documentation](https://docs.oracle.com/javase/8/docs/api/java/lang/management/ManagementFactory.html#getGarbageCollectorMXBeans--) for more details. ##### JVM Misc -* There are metrics prefixed with `threads` providing the number of threads, daemon threads, blocked and deadlocked - threads. +* There are metrics prefixed with `threads` providing the number of threads, daemon threads, blocked and deadlocked threads. ##### Other worker metrics * `doHeartbeat-calls` is a meter that indicates the rate the worker is performing heartbeats. -* `newWorkerEvent` is 1 when a worker is first started and 0 all other times. This can be used to tell when a worker has - crashed and is restarted. +* `newWorkerEvent` is 1 when a worker is first started and 0 all other times. This can be used to tell when a worker has crashed and is restarted. * `startTimeSecs` is when the worker started in seconds since the epoch * `uptimeSecs` reports the number of seconds the worker has been up for -* `workerCpuUsage` reports the cpu usage of the worker as a percentage of cores. 1.0 indicates 1 cpu core. +* `workerCpuUsage` reports the cpu usage of the worker as a percentage of cores. 1.0 indicates 1 cpu core. From ac4e3b1e3806cf248b7a800f2aadfc55669f9ae9 Mon Sep 17 00:00:00 2001 From: Gianluca Graziadei Date: Tue, 5 May 2026 19:07:36 +0200 Subject: [PATCH 6/8] format --- .../src/jvm/org/apache/storm/Config.java | 101 +++++++----------- .../apache/storm/metrics2/TaskMetrics.java | 8 +- .../org/apache/storm/utils/ConfigUtils.java | 16 +-- 3 files changed, 53 insertions(+), 72 deletions(-) diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java index 09b0790475b..ee2a09e56b6 100644 --- a/storm-client/src/jvm/org/apache/storm/Config.java +++ b/storm-client/src/jvm/org/apache/storm/Config.java @@ -19,14 +19,12 @@ package org.apache.storm; import com.esotericsoftware.kryo.Serializer; - import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; - import org.apache.storm.metric.IEventLogger; import org.apache.storm.policy.IWaitStrategy; import org.apache.storm.serialization.IKryoDecorator; @@ -257,14 +255,14 @@ public class Config extends HashMap { */ @IsPositiveNumber(includeZero = true) public static final String TOPOLOGY_METRICS_CONSUMER_RESOURCES_ONHEAP_MEMORY_MB = - "topology.metrics.consumer.resources.onheap.memory.mb"; + "topology.metrics.consumer.resources.onheap.memory.mb"; /** * The maximum amount of memory an instance of a metrics consumer will take off heap. This enables the scheduler to allocate slots on * machines with enough available memory. A default value will be set for this config if user does not override */ @IsPositiveNumber(includeZero = true) public static final String TOPOLOGY_METRICS_CONSUMER_RESOURCES_OFFHEAP_MEMORY_MB = - "topology.metrics.consumer.resources.offheap.memory.mb"; + "topology.metrics.consumer.resources.offheap.memory.mb"; /** * The config indicates the percentage of cpu for a core an instance(executor) of a metrics consumer will use. Assuming the a core value * to be 100, a value of 10 indicates 10% of the core. The P in PCORE represents the term "physical". A default value will be set for @@ -350,16 +348,15 @@ public class Config extends HashMap { *

* *

comp-1 cannot exist on same worker as comp-2 or comp-3, and at most "2" comp-1 on same node

- * *

comp-2 and comp-4 cannot be on same worker (missing comp-1 is implied from comp-1 constraint)

* - *

- * { "comp-1": { "maxNodeCoLocationCnt": 2, "incompatibleComponents": ["comp-2", "comp-3" ] }, - * "comp-2": { "incompatibleComponents": [ "comp-4" ] } - * } - *

+ *

+ * { "comp-1": { "maxNodeCoLocationCnt": 2, "incompatibleComponents": ["comp-2", "comp-3" ] }, + * "comp-2": { "incompatibleComponents": [ "comp-4" ] } + * } + *

*/ - @IsExactlyOneOf(valueValidatorClasses = {ListOfListOfStringValidator.class, RasConstraintsTypeValidator.class}) + @IsExactlyOneOf(valueValidatorClasses = { ListOfListOfStringValidator.class, RasConstraintsTypeValidator.class }) public static final String TOPOLOGY_RAS_CONSTRAINTS = "topology.ras.constraints"; /** @@ -427,17 +424,17 @@ public class Config extends HashMap { * *

* 1. If not setting this variable or setting it as null, - * a. If RAS is not used: - * Nimbus will set it to {@link Config#TOPOLOGY_WORKERS}. - * b. If RAS is used: - * Nimbus will set it to (the estimate number of workers * {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER}). - * {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER} is default to be 1 if not set. + * a. If RAS is not used: + * Nimbus will set it to {@link Config#TOPOLOGY_WORKERS}. + * b. If RAS is used: + * Nimbus will set it to (the estimate number of workers * {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER}). + * {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER} is default to be 1 if not set. * 2. If this variable is set to 0, - * then Storm will immediately ack tuples as soon as they come off the spout, - * effectively disabling reliability. + * then Storm will immediately ack tuples as soon as they come off the spout, + * effectively disabling reliability. * 3. If this variable is set to a positive integer, - * Storm will not honor {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER} setting. - * Instead, nimbus will set it as (this variable / estimate num of workers). + * Storm will not honor {@link Config#TOPOLOGY_RAS_ACKER_EXECUTORS_PER_WORKER} setting. + * Instead, nimbus will set it as (this variable / estimate num of workers). *

*/ @IsInteger @@ -451,7 +448,7 @@ public class Config extends HashMap { * This setting is RAS specific. * If {@link Config#TOPOLOGY_ACKER_EXECUTORS} is not configured, * this setting will be used to calculate {@link Config#TOPOLOGY_ACKER_EXECUTORS}. - *

+ * * If {@link Config#TOPOLOGY_ACKER_EXECUTORS} is configured, * nimbus will ignore this and set it as ({@link Config#TOPOLOGY_ACKER_EXECUTORS} / estimate num of workers). *

@@ -468,7 +465,7 @@ public class Config extends HashMap { *

Note that EventLoggerBolt takes care of all the implementations of IEventLogger, hence registering many * implementations (especially they're implemented as 'blocking' manner) would slow down overall topology. */ - @IsListEntryCustom(entryValidatorClasses = {EventLoggerRegistryValidator.class}) + @IsListEntryCustom(entryValidatorClasses = { EventLoggerRegistryValidator.class }) public static final String TOPOLOGY_EVENT_LOGGER_REGISTER = "topology.event.logger.register"; /** * How many executors to spawn for event logger. @@ -546,7 +543,7 @@ public class Config extends HashMap { * it's parallelism is configurable. */ - @IsListEntryCustom(entryValidatorClasses = {MetricRegistryValidator.class}) + @IsListEntryCustom(entryValidatorClasses = { MetricRegistryValidator.class }) public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register"; /** * Enable tracking of network message byte counts per source-destination task. This is off by default as it creates tasks^2 metric @@ -850,14 +847,14 @@ public class Config extends HashMap { * Topology central logging sensitivity to determine who has access to logs in central logging system. The possible values are: S0 - * Public (open to all users on grid) S1 - Restricted S2 - Confidential S3 - Secret (default.) */ - @IsString(acceptedValues = {"S0", "S1", "S2", "S3"}) + @IsString(acceptedValues = { "S0", "S1", "S2", "S3" }) public static final String TOPOLOGY_LOGGING_SENSITIVITY = "topology.logging.sensitivity"; /** * Log file the user can use to configure Log4j2. * Can be a resource in the jar (specified with classpath:/path/to/resource) or a file. * This configuration is applied in addition to the regular worker log4j2 configuration. * The configs are merged according to the rules here: - * https://logging.apache.org/log4j/2.x/manual/configuration.html#CompositeConfiguration + * https://logging.apache.org/log4j/2.x/manual/configuration.html#CompositeConfiguration */ @IsString public static final String TOPOLOGY_LOGGING_CONFIG_FILE = "topology.logging.config"; @@ -901,8 +898,7 @@ public class Config extends HashMap { * Alternatively set {@code storm.scheduler} to {@code org.apache.storm.scheduler.resource.ResourceAwareScheduler} * using {@link Config#TOPOLOGY_SCHEDULER_STRATEGY} set to * {@code org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy} - * - */ + * */ @IsInteger @IsPositiveNumber public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines"; @@ -995,7 +991,7 @@ public class Config extends HashMap { @NotNull @IsPositiveNumber(includeZero = true) public static final String TOPOLOGY_BACKPRESSURE_WAIT_PROGRESSIVE_LEVEL3_SLEEP_MILLIS = - "topology.backpressure.wait.progressive.level3.sleep.millis"; + "topology.backpressure.wait.progressive.level3.sleep.millis"; /** * Configures steps used to determine progression to the next level of wait .. if using WaitStrategyProgressive for BackPressure. */ @@ -1452,34 +1448,22 @@ public class Config extends HashMap { @IsString public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME = "storm.zookeeper.topology.auth.scheme"; - /** - * Enable SSL/TLS for ZooKeeper client connection. - */ + /** Enable SSL/TLS for ZooKeeper client connection. */ @IsBoolean public static final String ZK_SSL_ENABLE = "storm.zookeeper.ssl.enable"; - /** - * Keystore location for ZooKeeper client connection over SSL. - */ + /** Keystore location for ZooKeeper client connection over SSL. */ @IsString public static final String STORM_ZOOKEEPER_SSL_KEYSTORE_PATH = "storm.zookeeper.ssl.keystore.path"; - /** - * Keystore password for ZooKeeper client connection over SSL. - */ + /** Keystore password for ZooKeeper client connection over SSL. */ @IsString public static final String STORM_ZOOKEEPER_SSL_KEYSTORE_PASSWORD = "storm.zookeeper.ssl.keystore.password"; - /** - * Truststore location for ZooKeeper client connection over SSL. - */ + /** Truststore location for ZooKeeper client connection over SSL. */ @IsString public static final String STORM_ZOOKEEPER_SSL_TRUSTSTORE_PATH = "storm.zookeeper.ssl.truststore.path"; - /** - * Truststore password for ZooKeeper client connection over SSL. - */ + /** Truststore password for ZooKeeper client connection over SSL. */ @IsString public static final String STORM_ZOOKEEPER_SSL_TRUSTSTORE_PASSWORD = "storm.zookeeper.ssl.truststore.password"; - /** - * Enable or disable hostname verification. - */ + /** Enable or disable hostname verification.*/ @IsBoolean public static final String STORM_ZOOKEEPER_SSL_HOSTNAME_VERIFICATION = "storm.zookeeper.ssl.hostnameVerification"; /** @@ -1492,13 +1476,13 @@ public class Config extends HashMap { /** * Configure the topology metrics reporters to be used on workers. */ - @IsListEntryCustom(entryValidatorClasses = {MetricReportersValidator.class}) + @IsListEntryCustom(entryValidatorClasses = { MetricReportersValidator.class }) public static final String TOPOLOGY_METRICS_REPORTERS = "topology.metrics.reporters"; /** * A list of system metrics reporters that will get added to each topology. */ - @IsListEntryCustom(entryValidatorClasses = {MetricReportersValidator.class}) + @IsListEntryCustom(entryValidatorClasses = { MetricReportersValidator.class }) public static final String STORM_TOPOLOGY_METRICS_SYSTEM_REPORTERS = "storm.topology.metrics.system.reporters"; /** @@ -1506,7 +1490,7 @@ public class Config extends HashMap { * Use {@link Config#TOPOLOGY_METRICS_REPORTERS} instead. */ @Deprecated(forRemoval = true, since = "2.0.0") - @IsListEntryCustom(entryValidatorClasses = {MetricReportersValidator.class}) + @IsListEntryCustom(entryValidatorClasses = { MetricReportersValidator.class }) public static final String STORM_METRICS_REPORTERS = "storm.metrics.reporters"; /** @@ -1541,7 +1525,6 @@ public class Config extends HashMap { public static final String BLOBSTORE_HDFS_PRINCIPAL = "blobstore.hdfs.principal"; /** * keytab for nimbus/supervisor to use to access secure hdfs for the blobstore. - * * @Deprecated Use {@link Config#STORM_HDFS_LOGIN_KEYTAB} instead. */ @Deprecated @@ -1861,7 +1844,7 @@ public class Config extends HashMap { public static final String STORM_MESSAGING_NETTY_TLS_SSL_PROTOCOLS = "storm.messaging.netty.tls.ssl.protocols"; /** - * /** + /** * Netty based messaging: The number of milliseconds that a Netty client will retry flushing messages that are already * buffered to be sent. */ @@ -1955,8 +1938,8 @@ public class Config extends HashMap { /** * Impersonation user ACL config entries. */ - @IsMapEntryCustom(keyValidatorClasses = {ConfigValidation.StringValidator.class}, - valueValidatorClasses = {ConfigValidation.ImpersonationAclUserEntryValidator.class}) + @IsMapEntryCustom(keyValidatorClasses = { ConfigValidation.StringValidator.class }, + valueValidatorClasses = { ConfigValidation.ImpersonationAclUserEntryValidator.class }) public static final String NIMBUS_IMPERSONATION_ACL = "nimbus.impersonation.acl"; /** * A whitelist of the RAS scheduler strategies allowed by nimbus. Should be a list of fully-qualified class names or null to allow all. @@ -2416,7 +2399,7 @@ public void setTopologyComponentWorkerConstraints(String component1, String comp if (component1 != null && component2 != null) { List constraintPair = Arrays.asList(component1, component2); List> constraints = (List>) computeIfAbsent(Config.TOPOLOGY_RAS_CONSTRAINTS, - (k) -> new ArrayList<>(1)); + (k) -> new ArrayList<>(1)); constraints.add(constraintPair); } } @@ -2461,7 +2444,6 @@ public static String getBlobstoreHDFSPrincipal(Map conf) throws UnknownHostExcep /** * Get the hostname substituted hdfs principal. - * * @param conf the storm Configuration * @return the principal * @throws UnknownHostException on UnknowHostException @@ -2477,12 +2459,12 @@ public static String getHdfsPrincipal(Map conf) throws UnknownHo ret = hdfsPrincipal; } else if (hdfsPrincipal == null) { LOG.warn("{} is used as the hdfs principal. Please use {} instead", - Config.BLOBSTORE_HDFS_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL); + Config.BLOBSTORE_HDFS_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL); ret = blobstorePrincipal; } else { //both not null; LOG.warn("Both {} and {} are set. Use {} only.", - Config.BLOBSTORE_HDFS_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL); + Config.BLOBSTORE_HDFS_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL, Config.STORM_HDFS_LOGIN_PRINCIPAL); ret = hdfsPrincipal; } return substituteHostnameInPrincipal(ret); @@ -2490,7 +2472,6 @@ public static String getHdfsPrincipal(Map conf) throws UnknownHo /** * Get the hdfs keytab. - * * @param conf the storm Configuration * @return the keytab */ @@ -2505,12 +2486,12 @@ public static String getHdfsKeytab(Map conf) { ret = hdfsKeyTab; } else if (hdfsKeyTab == null) { LOG.warn("{} is used as the hdfs keytab. Please use {} instead", - Config.BLOBSTORE_HDFS_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB); + Config.BLOBSTORE_HDFS_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB); ret = blobstoreKeyTab; } else { //both not null; LOG.warn("Both {} and {} are set. Use {} only.", - Config.BLOBSTORE_HDFS_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB); + Config.BLOBSTORE_HDFS_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB, Config.STORM_HDFS_LOGIN_KEYTAB); ret = hdfsKeyTab; } return ret; diff --git a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java index e9062172f70..cc6a23c0992 100644 --- a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java +++ b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java @@ -2,9 +2,9 @@ * 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. @@ -202,14 +202,14 @@ private > G getOrCreateGauge( return gaugeClass.cast(existing); } - /** + /* * Safe cast: G is bounded by Gauge in the signature of getOrCreateGauge, * so every instance of G is by definition a Gauge. * The cast to raw Gauge is required because metricRegistry.gauge() does not * accept Gauge the wildcard is not compatible with the type parameter T * expected by the external API. Type-safety is guaranteed by the bound * > declared at the call site. - **/ + */ @SuppressWarnings({"unchecked", "rawtypes"}) private void registerGauge(String metricName, String streamId, Gauge gauge) { metricRegistry.gauge(metricName, (Gauge) gauge, this.topologyId, diff --git a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java index 9f6d5285542..94599ec8935 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java +++ b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java @@ -2,9 +2,9 @@ * 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. @@ -35,7 +35,6 @@ import org.apache.storm.validation.ConfigValidation; import org.apache.storm.validation.ConfigValidationAnnotations; - public class ConfigUtils { public static final String FILE_SEPARATOR = File.separator; public static final String STORM_HOME = "storm.home"; @@ -45,7 +44,7 @@ public class ConfigUtils { private static final Set passwordConfigKeys = new HashSet<>(); static { - for (Class clazz : ConfigValidation.getConfigClasses()) { + for (Class clazz: ConfigValidation.getConfigClasses()) { for (Field field : clazz.getFields()) { for (Annotation annotation : field.getAnnotations()) { boolean isPassword = annotation.annotationType().getName().equals( @@ -338,7 +337,7 @@ public static Map readSupervisorStormConfGivenPath(Map overrideLoginConfigWithSystemProperty( - Map conf) { // note that we delete the return value + Map conf) { // note that we delete the return value String loginConfFile = System.getProperty("java.security.auth.login.config"); if (loginConfFile != null) { conf.put("java.security.auth.login.config", loginConfFile); @@ -437,6 +436,7 @@ public static File getWorkerDirFromRoot(String logRoot, String id, Integer port) * @param name - the config key * @param conf - the config map * @return - the config value converted to a List <String> if found, otherwise null. + * * @throws IllegalArgumentException if conf is null * @throws NullPointerException if name is null and the conf map doesn't support null keys */ @@ -450,9 +450,9 @@ public static List getValueAsList(String name, Map conf) listValue = null; } else if (value instanceof Collection) { listValue = ((Collection) value) - .stream() - .map(ObjectReader::getString) - .collect(Collectors.toList()); + .stream() + .map(ObjectReader::getString) + .collect(Collectors.toList()); } else { listValue = Arrays.asList(ObjectReader.getString(value).split("\\s+")); } From 23247e3a07002c5fd31522901141000ed41eea7d Mon Sep 17 00:00:00 2001 From: Gianluca Graziadei Date: Tue, 5 May 2026 21:20:37 +0200 Subject: [PATCH 7/8] fix checkstyle --- .../src/jvm/org/apache/storm/Config.java | 2 +- .../{EWMAGauge.java => EwmaGauge.java} | 16 ++++---- .../apache/storm/metrics2/TaskMetrics.java | 16 ++++---- .../storm/validation/ConfigValidation.java | 4 +- ...{EWMAGaugeTest.java => EwmaGaugeTest.java} | 38 +++++++++---------- .../storm/metrics2/TaskMetricsTest.java | 5 +-- 6 files changed, 39 insertions(+), 42 deletions(-) rename storm-client/src/jvm/org/apache/storm/metrics2/{EWMAGauge.java => EwmaGauge.java} (95%) rename storm-client/test/jvm/org/apache/storm/metrics2/{EWMAGaugeTest.java => EwmaGaugeTest.java} (92%) diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java index ee2a09e56b6..9e28aaec2dc 100644 --- a/storm-client/src/jvm/org/apache/storm/Config.java +++ b/storm-client/src/jvm/org/apache/storm/Config.java @@ -608,7 +608,7 @@ public class Config extends HashMap { * * @see RFC 1889 Appendix A.8 */ - @CustomValidator(validatorClass = ConfigValidation.EWMASmoothingFactorValidator.class) + @CustomValidator(validatorClass = ConfigValidation.EwmaSmoothingFactorValidator.class) public static final String TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR = "topology.stats.ewma.smoothing_factor"; /** * The time period that builtin metrics data in bucketed into. diff --git a/storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java b/storm-client/src/jvm/org/apache/storm/metrics2/EwmaGauge.java similarity index 95% rename from storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java rename to storm-client/src/jvm/org/apache/storm/metrics2/EwmaGauge.java index f55eff736f7..cd5ff9a7b90 100644 --- a/storm-client/src/jvm/org/apache/storm/metrics2/EWMAGauge.java +++ b/storm-client/src/jvm/org/apache/storm/metrics2/EwmaGauge.java @@ -9,23 +9,22 @@ * 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.storm.metrics2; +import static org.apache.storm.utils.ConfigUtils.RFC1889_ALPHA; + import com.codahale.metrics.Gauge; import java.util.concurrent.atomic.AtomicLong; -import static org.apache.storm.utils.ConfigUtils.RFC1889_ALPHA; - /** * Lock-free jitter estimator following RFC 1889 Section 6.3.1. - *

* The jitter accumulator is stored as raw IEEE 754 bits in an AtomicLong * so that CAS can be used without locks. - *

* Thread safety: addValue is lock-free; getValue is wait-free. */ -public class EWMAGauge implements Gauge { +public class EwmaGauge implements Gauge { private static final long UNSEEDED = Long.MIN_VALUE; private static final long ZERO_BITS = Double.doubleToLongBits(0.0); @@ -34,7 +33,7 @@ public class EWMAGauge implements Gauge { private final AtomicLong jitterBits = new AtomicLong(ZERO_BITS); private final double alpha; - EWMAGauge(double alpha) { + EwmaGauge(double alpha) { if (alpha <= 0.0 || alpha >= 1.0 || Double.isNaN(alpha)) { throw new IllegalArgumentException( "alpha must be in (0, 1), got: " + alpha); @@ -42,7 +41,7 @@ public class EWMAGauge implements Gauge { this.alpha = alpha; } - EWMAGauge() { + EwmaGauge() { this(RFC1889_ALPHA); // 1.0 / 16.0 } @@ -74,7 +73,8 @@ public void addValue(long transitMs) { return; } - long currentBits, updatedBits; + long currentBits; + long updatedBits; do { currentBits = jitterBits.get(); double currentJitter = Double.longBitsToDouble(currentBits); diff --git a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java index cc6a23c0992..6bea1d3b822 100644 --- a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java +++ b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java @@ -12,12 +12,12 @@ package org.apache.storm.metrics2; +import com.codahale.metrics.Gauge; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.function.Supplier; -import com.codahale.metrics.Gauge; import org.apache.storm.task.WorkerTopologyContext; import org.apache.storm.utils.ConfigUtils; import org.apache.storm.utils.Utils; @@ -39,7 +39,7 @@ public class TaskMetrics { private final ConcurrentMap rateCounters = new ConcurrentHashMap<>(); private final ConcurrentMap gauges = new ConcurrentHashMap<>(); // Gauge supplier singleton factories - private final Supplier ewmaGaugeFactory; + private final Supplier ewmaGaugeFactory; private final Supplier rollingAverageGaugeFactory; private final String topologyId; @@ -62,7 +62,7 @@ public TaskMetrics(WorkerTopologyContext context, String componentId, Integer ta double ewmaSmoothingFactor = ConfigUtils.ewmaSmoothingFactor(topoConf); this.ewmaEnable = ConfigUtils.ewmaEnable(topoConf); this.rollingAverageGaugeFactory = RollingAverageGauge::new; - this.ewmaGaugeFactory = () -> new EWMAGauge(ewmaSmoothingFactor); + this.ewmaGaugeFactory = () -> new EwmaGauge(ewmaSmoothingFactor); } public void setCapacity(double capacity) { @@ -83,7 +83,7 @@ public void spoutAckedTuple(String streamId, long latencyMs) { if (this.ewmaEnable) { metricName = METRIC_NAME_COMPLETE_RFC_1889a_JITTER + "-" + streamId; - EWMAGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, streamId); + EwmaGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, streamId); ewmaGauge.addValue(latencyMs); } } @@ -100,7 +100,7 @@ public void boltAckedTuple(String sourceComponentId, String sourceStreamId, long if (this.ewmaEnable) { metricName = METRIC_NAME_PROCESS_RFC_1889a_JITTER + "-" + key; - EWMAGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId); + EwmaGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId); ewmaGauge.addValue(latencyMs); } } @@ -145,7 +145,7 @@ public void boltExecuteTuple(String sourceComponentId, String sourceStreamId, lo if (this.ewmaEnable) { metricName = METRIC_NAME_EXECUTE_RFC_1889a_JITTER + "-" + key; - EWMAGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId); + EwmaGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId); ewmaGauge.addValue(latencyMs); } } @@ -169,8 +169,8 @@ private RollingAverageGauge getRollingAverageGauge(String metricName, String str return getOrCreateGauge(metricName, streamId, RollingAverageGauge.class, this.rollingAverageGaugeFactory); } - private EWMAGauge getExponentialWeightedMobileAverageGauge(String metricName, String streamId) { - return getOrCreateGauge(metricName, streamId, EWMAGauge.class, this.ewmaGaugeFactory); + private EwmaGauge getExponentialWeightedMobileAverageGauge(String metricName, String streamId) { + return getOrCreateGauge(metricName, streamId, EwmaGauge.class, this.ewmaGaugeFactory); } private > G getOrCreateGauge( diff --git a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java index 36ab2b1661a..228ca8bfb25 100644 --- a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java +++ b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java @@ -849,10 +849,10 @@ public void validateField(String name, Object o) { } } - public static class EWMASmoothingFactorValidator extends Validator { + public static class EwmaSmoothingFactorValidator extends Validator { @Override public void validateField(String name, Object o) { - if (o == null){ + if (o == null) { return; } if (o instanceof Number) { diff --git a/storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java b/storm-client/test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java similarity index 92% rename from storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java rename to storm-client/test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java index 6bf53ff0784..0ed418c65ca 100644 --- a/storm-client/test/jvm/org/apache/storm/metrics2/EWMAGaugeTest.java +++ b/storm-client/test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java @@ -28,7 +28,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -class EWMAGaugeTest { +class EwmaGaugeTest { private static final double DELTA = 1e-9; @@ -39,7 +39,7 @@ class ConstructionTest { @Test @DisplayName("Default constructor uses RFC 1889 alpha (1/16)") void defaultAlpha() { - EWMAGauge gauge = new EWMAGauge(); + EwmaGauge gauge = new EwmaGauge(); gauge.addValue(0L); gauge.addValue(16L); // D = 16 ; J = 0 + (16 - 0) * (1/16) = 1.0 assertEquals(1.0, gauge.getValue(), DELTA); @@ -54,7 +54,7 @@ void invalidAlphaThrows() { }; for (double alpha : invalidAlphas) { assertThrows(IllegalArgumentException.class, - () -> new EWMAGauge(alpha), + () -> new EwmaGauge(alpha), "Expected IllegalArgumentException for alpha=" + alpha); } } @@ -64,7 +64,7 @@ void invalidAlphaThrows() { void validAlphaAccepted() { double[] validAlphas = {0.001, 0.0625, 0.5, 0.999}; for (double alpha : validAlphas) { - assertNotNull(new EWMAGauge(alpha), + assertNotNull(new EwmaGauge(alpha), "Expected no exception for alpha=" + alpha); } } @@ -75,11 +75,11 @@ void validAlphaAccepted() { @DisplayName("Cold-start semantics") class ColdStartTest { - private EWMAGauge gauge; + private EwmaGauge gauge; @BeforeEach void setUp() { - gauge = new EWMAGauge(); + gauge = new EwmaGauge(); } @Test @@ -104,7 +104,7 @@ class FormulaTest { @Test @DisplayName("Single update: J = 0 + (D - 0) * alpha") void singleDeviation() { - EWMAGauge gauge = new EWMAGauge(0.5); + EwmaGauge gauge = new EwmaGauge(0.5); gauge.addValue(0L); gauge.addValue(10L); assertEquals(5.0, gauge.getValue(), DELTA); @@ -113,7 +113,7 @@ void singleDeviation() { @Test @DisplayName("Manual step-by-step verification against reference values") void manualSteps() { - EWMAGauge gauge = new EWMAGauge(0.5); + EwmaGauge gauge = new EwmaGauge(0.5); gauge.addValue(0L); // seed @@ -133,7 +133,7 @@ void manualSteps() { @Test @DisplayName("Zero deviation decays jitter toward zero") void zeroDeviationDecays() { - EWMAGauge gauge = new EWMAGauge(0.5); + EwmaGauge gauge = new EwmaGauge(0.5); gauge.addValue(0L); // 0 gauge.addValue(10L); // 0 + 5*alpha = 2.5 double afterFirst = gauge.getValue(); @@ -152,7 +152,7 @@ class NegativeValueTest { @Test @DisplayName("Negative transit values are silently ignored before seed") void negativeIgnoredBeforeSeed() { - EWMAGauge gauge = new EWMAGauge(); + EwmaGauge gauge = new EwmaGauge(); gauge.addValue(-1L); gauge.addValue(-100L); assertEquals(0.0, gauge.getValue(), DELTA); @@ -161,7 +161,7 @@ void negativeIgnoredBeforeSeed() { @Test @DisplayName("Negative value after seed does not corrupt lastTransit") void negativeAfterSeedIgnored() { - EWMAGauge gauge = new EWMAGauge(0.5); + EwmaGauge gauge = new EwmaGauge(0.5); gauge.addValue(10L); gauge.addValue(-5L); gauge.addValue(20L); @@ -177,7 +177,7 @@ class GetValueIdempotentTest { @Test @DisplayName("Repeated getValue() without new samples returns same estimate") void repeatedGetValueStable() { - EWMAGauge gauge = new EWMAGauge(0.5); + EwmaGauge gauge = new EwmaGauge(0.5); gauge.addValue(0L); gauge.addValue(10L); double first = gauge.getValue(); @@ -189,7 +189,7 @@ void repeatedGetValueStable() { @Test @DisplayName("EWMA accumulates correctly across multiple reporting windows") void acrossReportingWindows() { - EWMAGauge gauge = new EWMAGauge(0.5); + EwmaGauge gauge = new EwmaGauge(0.5); gauge.addValue(0L); gauge.addValue(10L); @@ -210,7 +210,7 @@ class ConcurrencyTest { @Test @DisplayName("Concurrent addValue() calls do not corrupt state") void concurrentAddValue() throws InterruptedException { - EWMAGauge gauge = new EWMAGauge(); + EwmaGauge gauge = new EwmaGauge(); int threads = 8; int samplesPerThread = 10_000; CountDownLatch ready = new CountDownLatch(threads); @@ -246,7 +246,7 @@ void concurrentAddValue() throws InterruptedException { @Test @DisplayName("Concurrent getValue() and addValue() do not deadlock") void concurrentGetAndAdd() throws Exception { - EWMAGauge gauge = new EWMAGauge(); + EwmaGauge gauge = new EwmaGauge(); ExecutorService pool = Executors.newFixedThreadPool(2); CountDownLatch done = new CountDownLatch(2); @@ -277,7 +277,7 @@ void concurrentGetAndAdd() throws Exception { @Test @DisplayName("Only one thread seeds lastTransit all same value gives zero jitter") void seedRace() throws InterruptedException { - EWMAGauge gauge = new EWMAGauge(); + EwmaGauge gauge = new EwmaGauge(); int threads = 16; CountDownLatch ready = new CountDownLatch(threads); CountDownLatch start = new CountDownLatch(1); @@ -312,7 +312,7 @@ class EdgeCaseTest { @Test @DisplayName("Long.MAX_VALUE transit does not overflow deviation") void maxLongTransit() { - EWMAGauge gauge = new EWMAGauge(0.5); + EwmaGauge gauge = new EwmaGauge(0.5); gauge.addValue(0L); gauge.addValue(Long.MAX_VALUE); double value = gauge.getValue(); @@ -323,7 +323,7 @@ void maxLongTransit() { @Test @DisplayName("Zero transit time is valid and produces zero deviation") void zeroTransit() { - EWMAGauge gauge = new EWMAGauge(0.5); + EwmaGauge gauge = new EwmaGauge(0.5); gauge.addValue(0L); gauge.addValue(0L); assertEquals(0.0, gauge.getValue(), DELTA); @@ -332,7 +332,7 @@ void zeroTransit() { @Test @DisplayName("Large number of samples does not overflow LongAdder") void manySamples() { - EWMAGauge gauge = new EWMAGauge(); + EwmaGauge gauge = new EwmaGauge(); gauge.addValue(0L); for (int i = 1; i <= 100_000; i++) { gauge.addValue(i % 2 == 0 ? 0L : 10L); diff --git a/storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java b/storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java index b2c504a3621..6b6c7af786b 100644 --- a/storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java +++ b/storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java @@ -15,7 +15,6 @@ import com.codahale.metrics.Gauge; import org.apache.storm.task.WorkerTopologyContext; import org.apache.storm.utils.ConfigUtils; -import org.apache.storm.utils.Utils; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -32,8 +31,6 @@ import java.util.concurrent.TimeUnit; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.*; import static org.mockito.Mockito.*; @@ -54,7 +51,7 @@ class TaskMetricsTest { @Mock private StormMetricRegistry metricRegistry; @Mock private RateCounter rateCounter; @Mock private RollingAverageGauge rollingAverageGauge; - @Mock private EWMAGauge ewmaGauge; + @Mock private EwmaGauge ewmaGauge; private Map topoConf; From ce8e2b143d9d331764c931c77000a6df6880071e Mon Sep 17 00:00:00 2001 From: Gianluca Graziadei Date: Wed, 6 May 2026 10:10:35 +0200 Subject: [PATCH 8/8] fix RFC 1889 jitter definition, guarantee jitter decay if stable latency --- .../src/jvm/org/apache/storm/metrics2/EwmaGauge.java | 4 ---- .../test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java | 5 +++-- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/storm-client/src/jvm/org/apache/storm/metrics2/EwmaGauge.java b/storm-client/src/jvm/org/apache/storm/metrics2/EwmaGauge.java index cd5ff9a7b90..baff872f374 100644 --- a/storm-client/src/jvm/org/apache/storm/metrics2/EwmaGauge.java +++ b/storm-client/src/jvm/org/apache/storm/metrics2/EwmaGauge.java @@ -69,10 +69,6 @@ public void addValue(long transitMs) { double d = Math.abs(transitMs - prev); - if (d <= 0) { - return; - } - long currentBits; long updatedBits; do { diff --git a/storm-client/test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java b/storm-client/test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java index 0ed418c65ca..48d11bc1262 100644 --- a/storm-client/test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java +++ b/storm-client/test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java @@ -137,9 +137,10 @@ void zeroDeviationDecays() { gauge.addValue(0L); // 0 gauge.addValue(10L); // 0 + 5*alpha = 2.5 double afterFirst = gauge.getValue(); - - gauge.addValue(10L); // 2.5 + 0*alpha = 2.5 assertEquals(afterFirst, gauge.getValue(), DELTA); + + gauge.addValue(10L); // 2.5 - 2.5*alpha = 2.5 - 1.25 = 1.25 + assertEquals(afterFirst * 0.5, gauge.getValue(), DELTA); } }