From aae2554b3fbb809ca7afbd90ee89dcf0ad4a8565 Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Sun, 3 May 2026 15:38:03 +0200
Subject: [PATCH 01/21] 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)
*/
- @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
+ *
* 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 03/21] 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
- *
+ *
* 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 04/21] 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 05/21] 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 06/21] 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)
*/
- @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
- *
+ *
* 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
- *
+ *
* 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 07/21] 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 08/21] 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);
}
}
From f040770f7b2169697556a9fafcc66b53d79976ed Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Fri, 8 May 2026 17:37:07 +0200
Subject: [PATCH 09/21] minor changes
---
conf/defaults.yaml | 47 ++++-----
docs/Metrics.md | 53 +++++-----
.../src/jvm/org/apache/storm/Config.java | 98 +++++++++++--------
.../org/apache/storm/metrics2/EwmaGauge.java | 21 ++--
.../apache/storm/metrics2/TaskMetrics.java | 22 ++---
.../org/apache/storm/utils/ObjectReader.java | 2 +
.../storm/validation/ConfigValidation.java | 11 ++-
.../org/apache/storm/TestConfigValidate.java | 4 +-
.../apache/storm/metrics2/EwmaGaugeTest.java | 8 +-
.../storm/metrics2/TaskMetricsTest.java | 16 ++-
10 files changed, 151 insertions(+), 131 deletions(-)
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index d92da7b4658..bc9a5979d4c 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
@@ -279,6 +279,7 @@ topology.max.spout.pending: null # ideally should be larger than topology.pro
topology.state.synchronization.timeout.secs: 60
topology.stats.sample.rate: 0.05
topology.stats.ewma.enable: false
+topology.stats.ewma.smoothing.factor: 0.0625
topology.builtin.metrics.bucket.size.secs: 60
topology.fall.back.on.java.serialization: false
topology.worker.childopts: null
@@ -288,16 +289,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 +365,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 +373,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 +401,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 +421,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 6c65c0419dc..fe13e9d80f5 100644
--- a/docs/Metrics.md
+++ b/docs/Metrics.md
@@ -184,27 +184,11 @@ The complete latency is just for spouts. It is the average amount of time it too
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.
-##### `__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
@@ -213,14 +197,6 @@ then `__process-latency` and `__execute-latency` should be very close to one ano
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.
-##### `__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.
-
##### `__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.
@@ -238,6 +214,35 @@ This metric indicates the overflow count last time BP status was sent, with a mi
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.
+#### Tuple Jitter Metrics
+To activate jitter-based metrics, `topology.stats.ewma.enable` must be set to `true`, which switches the system to a jitter estimation based on an Exponential Moving Average (EWMA).
+In this model, jitter is dynamically updated by weighting new latency samples against historical data using a smoothing factor (`topology.stats.ewma.smoothing.factor`).
+This parameter, which defaults to 0.0625 (equivalent to $1/16$ or a 4-bit right shift), determines the metric's reactivity: higher values make the jitter more sensitive to recent spikes, while lower values prioritize long-term stability.
+Operators should be aware that enabling this feature triples the gauge count for every component-stream pair per task; this significant increase in metric cardinality can impact TSDB storage and costs, so backend capacity should be verified before deployment.
+
+##### `__complete-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 §A.8 / RFC 3550 §A.8.
+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-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 §A.8 / RFC 3550 §A.8.
+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-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 §A.8 / RFC 3550 §A.8.
+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.
+
#### Error Reporting Metrics
Storm also collects error reporting metrics for bolts and spouts.
diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java
index 9e28aaec2dc..62770b62ec9 100644
--- a/storm-client/src/jvm/org/apache/storm/Config.java
+++ b/storm-client/src/jvm/org/apache/storm/Config.java
@@ -350,13 +350,13 @@ 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)
*/
- @IsExactlyOneOf(valueValidatorClasses = { ListOfListOfStringValidator.class, RasConstraintsTypeValidator.class })
+ @IsExactlyOneOf(valueValidatorClasses = {ListOfListOfStringValidator.class, RasConstraintsTypeValidator.class})
public static final String TOPOLOGY_RAS_CONSTRAINTS = "topology.ras.constraints";
/**
@@ -424,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
@@ -465,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.
@@ -543,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
@@ -597,19 +597,19 @@ public class Config extends HashMap {
@IsPositiveNumber
public static final String TOPOLOGY_STATS_SAMPLE_RATE = "topology.stats.sample.rate";
/**
- * Enabling jitter streaming calculation (RFC 1889a).
+ * Enabling jitter streaming calculation (RFC 1889 §A.8).
*
- * @see RFC 1889 Appendix A.8
+ * @see RFC 1889 §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).
+ * The smoothing factor (alpha) used for exponential jitter calculation (RFC 1889 §A.8). The default value is set to 1/16.
*
- * @see RFC 1889 Appendix A.8
+ * @see RFC 1889 §A.8
*/
@CustomValidator(validatorClass = ConfigValidation.EwmaSmoothingFactorValidator.class)
- public static final String TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR = "topology.stats.ewma.smoothing_factor";
+ public static final String TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR = "topology.stats.ewma.smoothing.factor";
/**
* The time period that builtin metrics data in bucketed into.
*/
@@ -847,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";
@@ -898,7 +898,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";
@@ -1448,22 +1449,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";
/**
@@ -1476,13 +1489,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";
/**
@@ -1490,7 +1503,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";
/**
@@ -1525,6 +1538,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
@@ -1767,7 +1781,7 @@ public class Config extends HashMap {
*/
@IsInteger
public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS =
- "storm.messaging.netty.client_worker_threads";
+ "storm.messaging.netty.client_worker_threads";
/**
* Netty based messaging: Enables TLS connections between workers.
@@ -1822,7 +1836,7 @@ public class Config extends HashMap {
*/
@IsString
public static final String STORM_MESSAGING_NETTY_TLS_CLIENT_TRUSTSTORE_PASSWORD =
- "storm.messaging.netty.tls.client.truststore.password";
+ "storm.messaging.netty.tls.client.truststore.password";
/**
* Netty based messaging: Specifies the client keystore when TLS is enabled.
@@ -1835,7 +1849,7 @@ public class Config extends HashMap {
*/
@IsString
public static final String STORM_MESSAGING_NETTY_TLS_CLIENT_KEYSTORE_PASSWORD =
- "storm.messaging.netty.tls.client.keystore.password";
+ "storm.messaging.netty.tls.client.keystore.password";
/**
* Netty based messaging: Specifies the protocols TLS is enabled.
@@ -1844,7 +1858,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.
*/
@@ -1929,7 +1943,7 @@ public class Config extends HashMap {
@IsPositiveNumber
@IsInteger
public static final String STORM_BLOBSTORE_DEPENDENCY_JAR_UPLOAD_CHUNK_SIZE_BYTES =
- "storm.blobstore.dependency.jar.upload.chunk.size.bytes";
+ "storm.blobstore.dependency.jar.upload.chunk.size.bytes";
/**
* FQCN of a class that implements {@code ISubmitterHook} @see ISubmitterHook for details.
*/
@@ -1938,8 +1952,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.
@@ -2444,6 +2458,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
@@ -2472,6 +2487,7 @@ public static String getHdfsPrincipal(Map conf) throws UnknownHo
/**
* Get the hdfs keytab.
+ *
* @param conf the storm Configuration
* @return the keytab
*/
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 baff872f374..857e34215f0 100644
--- a/storm-client/src/jvm/org/apache/storm/metrics2/EwmaGauge.java
+++ b/storm-client/src/jvm/org/apache/storm/metrics2/EwmaGauge.java
@@ -1,10 +1,10 @@
-/*
+/**
* 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
- *
+ *
* 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.
@@ -19,7 +19,7 @@
import java.util.concurrent.atomic.AtomicLong;
/**
- * Lock-free jitter estimator following RFC 1889 Section 6.3.1.
+ * Lock-free jitter estimator following RFC 1889 §A.8 / RFC 3550 §A.8.
* 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.
@@ -55,20 +55,15 @@ 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;
- }
-
+ // Safe from Math.abs(Long.MIN_VALUE) pathology: both transitMs and prev
+ // are >= 0 (enforced by the negative-guard), so their
+ // difference is in [-Long.MAX_VALUE, Long.MAX_VALUE].
double d = Math.abs(transitMs - prev);
-
long currentBits;
long updatedBits;
do {
@@ -86,4 +81,4 @@ public void addValue(long transitMs) {
public Double getValue() {
return Double.longBitsToDouble(jitterBits.get());
}
-}
\ 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 6bea1d3b822..78cd6d3b966 100644
--- a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
+++ b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
@@ -29,15 +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_PROCESS_JITTER = "__process-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_COMPLETE_JITTER = "__complete-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_EXECUTE_JITTER = "__execute-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<>();
// Gauge supplier singleton factories
private final Supplier ewmaGaugeFactory;
private final Supplier rollingAverageGaugeFactory;
@@ -82,8 +82,8 @@ public void spoutAckedTuple(String streamId, long latencyMs) {
gauge.addValue(latencyMs);
if (this.ewmaEnable) {
- metricName = METRIC_NAME_COMPLETE_RFC_1889a_JITTER + "-" + streamId;
- EwmaGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, streamId);
+ metricName = METRIC_NAME_COMPLETE_JITTER + "-" + streamId;
+ EwmaGauge ewmaGauge = this.getExponentialWeightedMovingAverageGauge(metricName, streamId);
ewmaGauge.addValue(latencyMs);
}
}
@@ -99,8 +99,8 @@ public void boltAckedTuple(String sourceComponentId, String sourceStreamId, long
gauge.addValue(latencyMs);
if (this.ewmaEnable) {
- metricName = METRIC_NAME_PROCESS_RFC_1889a_JITTER + "-" + key;
- EwmaGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId);
+ metricName = METRIC_NAME_PROCESS_JITTER + "-" + key;
+ EwmaGauge ewmaGauge = this.getExponentialWeightedMovingAverageGauge(metricName, sourceStreamId);
ewmaGauge.addValue(latencyMs);
}
}
@@ -144,8 +144,8 @@ public void boltExecuteTuple(String sourceComponentId, String sourceStreamId, lo
gauge.addValue(latencyMs);
if (this.ewmaEnable) {
- metricName = METRIC_NAME_EXECUTE_RFC_1889a_JITTER + "-" + key;
- EwmaGauge ewmaGauge = this.getExponentialWeightedMobileAverageGauge(metricName, sourceStreamId);
+ metricName = METRIC_NAME_EXECUTE_JITTER + "-" + key;
+ EwmaGauge ewmaGauge = this.getExponentialWeightedMovingAverageGauge(metricName, sourceStreamId);
ewmaGauge.addValue(latencyMs);
}
}
@@ -169,7 +169,7 @@ private RollingAverageGauge getRollingAverageGauge(String metricName, String str
return getOrCreateGauge(metricName, streamId, RollingAverageGauge.class, this.rollingAverageGaugeFactory);
}
- private EwmaGauge getExponentialWeightedMobileAverageGauge(String metricName, String streamId) {
+ private EwmaGauge getExponentialWeightedMovingAverageGauge(String metricName, String streamId) {
return getOrCreateGauge(metricName, streamId, EwmaGauge.class, this.ewmaGaugeFactory);
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ObjectReader.java b/storm-client/src/jvm/org/apache/storm/utils/ObjectReader.java
index 54445b5eb49..ac28be8cf30 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ObjectReader.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ObjectReader.java
@@ -128,6 +128,8 @@ public static Double getDouble(Object o, Double defaultValue) {
}
if (o instanceof Number) {
return ((Number) o).doubleValue();
+ } else if (o instanceof String) {
+ return Double.parseDouble((String) o);
} else {
throw new IllegalArgumentException("Don't know how to convert (" + o + ") to double");
}
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 228ca8bfb25..0206ec458c4 100644
--- a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java
+++ b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java
@@ -30,6 +30,7 @@
import java.util.stream.Collectors;
import org.apache.storm.Config;
+import org.apache.storm.utils.ObjectReader;
import org.apache.storm.utils.Utils;
import org.apache.storm.validation.ConfigValidationAnnotations.ValidatorParams;
import org.slf4j.Logger;
@@ -855,11 +856,11 @@ 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;
- }
+ // ObjectReader.getDouble(o) handles the type conversion and will throw an
+ // IllegalArgumentException if the value cannot be parsed as a number.
+ double alpha = ObjectReader.getDouble(o);
+ 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);
diff --git a/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java b/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java
index 58668db1db6..31b9c483446 100644
--- a/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java
+++ b/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java
@@ -224,9 +224,11 @@ public void testTopologyStatsEwmaSmoothingFactorCustomValidator() {
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.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}) {
+ for (Object notAllowedValue : new Object[]{0.0, -0.1, 1.9, "1.9"}) {
conf.put(Config.TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR, notAllowedValue);
assertThrows(IllegalArgumentException.class, () -> ConfigValidation.validateFields(conf));
}
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 48d11bc1262..ed1b5009ceb 100644
--- a/storm-client/test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java
+++ b/storm-client/test/jvm/org/apache/storm/metrics2/EwmaGaugeTest.java
@@ -1,10 +1,10 @@
-/*
+/**
* 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.
@@ -344,4 +344,4 @@ void manySamples() {
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
index 6b6c7af786b..c812994280c 100644
--- a/storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java
+++ b/storm-client/test/jvm/org/apache/storm/metrics2/TaskMetricsTest.java
@@ -1,10 +1,10 @@
-/*
+/**
* 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.
@@ -50,8 +50,6 @@ class TaskMetricsTest {
@Mock private WorkerTopologyContext context;
@Mock private StormMetricRegistry metricRegistry;
@Mock private RateCounter rateCounter;
- @Mock private RollingAverageGauge rollingAverageGauge;
- @Mock private EwmaGauge ewmaGauge;
private Map topoConf;
@@ -109,7 +107,7 @@ void spoutAckedTuple_withEwmaEnabled_registersJitterGauge() {
tm.spoutAckedTuple(STREAM_ID, 150L);
verify(metricRegistry, atLeastOnce()).gauge(
- contains("__complete-rfc1889a-jitter"), any(Gauge.class),
+ contains("__complete-jitter"), any(Gauge.class),
anyString(), anyString(), anyString(), anyInt(), anyInt());
}
@@ -159,7 +157,7 @@ void boltAckedTuple_withEwmaEnabled_registersJitterGauge() {
tm.boltAckedTuple(SOURCE_COMP, STREAM_ID, 75L);
verify(metricRegistry, atLeastOnce()).gauge(
- contains("__process-rfc1889a-jitter"), any(Gauge.class),
+ contains("__process-jitter"), any(Gauge.class),
anyString(), anyString(), anyString(), anyInt(), anyInt());
}
@@ -290,7 +288,7 @@ void boltExecuteTuple_withEwmaEnabled_registersJitterGauge() {
tm.boltExecuteTuple(SOURCE_COMP, STREAM_ID, 30L);
verify(metricRegistry, atLeastOnce()).gauge(
- contains("__execute-rfc1889a-jitter"), any(Gauge.class),
+ contains("__execute-jitter"), any(Gauge.class),
anyString(), anyString(), anyString(), anyInt(), anyInt());
}
@@ -484,4 +482,4 @@ void contextFields_propagatedCorrectlyToRegistry() {
assertEquals(TASK_ID, taskCaptor.getValue());
assertEquals(WORKER_PORT, portCaptor.getValue());
}
-}
\ No newline at end of file
+}
From cb69a04c1eff47d74aa8ce0f2143ab92456b3666 Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Sat, 9 May 2026 17:49:16 +0200
Subject: [PATCH 10/21] publish control signal based on ewma metrics
---
.../src/jvm/org/apache/storm/Config.java | 40 +++++++++++++++-
.../org/apache/storm/daemon/StormCommon.java | 20 ++++++++
.../src/jvm/org/apache/storm/daemon/Task.java | 21 ++++++++
.../org/apache/storm/executor/Executor.java | 48 +++++++++++++++++++
.../bolt/BoltOutputCollectorImpl.java | 27 +++++++++++
.../apache/storm/metrics2/TaskMetrics.java | 8 ++++
.../org/apache/storm/utils/ConfigUtils.java | 32 +++++++++++++
.../storm/validation/ConfigValidation.java | 2 +-
8 files changed, 196 insertions(+), 2 deletions(-)
diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java
index 62770b62ec9..b368a41fc1e 100644
--- a/storm-client/src/jvm/org/apache/storm/Config.java
+++ b/storm-client/src/jvm/org/apache/storm/Config.java
@@ -608,8 +608,46 @@ public class Config extends HashMap {
*
* @see RFC 1889 §A.8
*/
- @CustomValidator(validatorClass = ConfigValidation.EwmaSmoothingFactorValidator.class)
+ @CustomValidator(validatorClass = ConfigValidation.ZerOneOpenIntervalValidator.class)
public static final String TOPOLOGY_STATS_EWMA_SMOOTHING_FACTOR = "topology.stats.ewma.smoothing.factor";
+ /**
+ * Flag to enable or disable the feedback channel for upstream communication.
+ * When true, components can send unanchored tuples back to their source tasks.
+ */
+ @IsBoolean
+ public static final String TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE = "topology.upstream.feedback.enable";
+ /**
+ * The specific stream ID used for upstream feedback communication.
+ * Defaults to "__feedback" if not explicitly configured.
+ */
+ @IsString
+ public static final String TOPOLOGY_UPSTREAM_FEEDBACK_STREAM_ID = "topology.upstream.feedback.stream";
+ /**
+ * Configuration for the sampling rate of upstream feedback messages within the topology.
+ *
+ *
This ratio defines the probability with which a task will emit a feedback tuple
+ * (containing metrics such as EWMA jitter stats) back to its parent tasks.
+ * This mechanism allows parent tasks to receive performance signals from downstream
+ * components to facilitate adaptive flow control or load balancing.
+ *
+ *
Validation: Must be a double value within the open interval (0.0, 1.0).
+ * Values of 0.0 (disabled) or 1.0 (every tuple) are rejected by the
+ * {@link ConfigValidation.ZerOneOpenIntervalValidator} to prevent improper
+ * configuration of the feedback loop.
+ *
+ *
Impact:
+ *
+ *
Higher values provide more precise, real-time performance data but increase
+ * network overhead and CPU usage on the control plane.
+ *
Lower values minimize the "observer effect" on the topology's throughput
+ * while still providing statistical snapshots of health.
+ *
+ *
+ *
+ * Defaults to 0.1 if not explicitly configured.
+ */
+ @CustomValidator(validatorClass = ConfigValidation.ZerOneOpenIntervalValidator.class)
+ public static final String TOPOLOGY_UPSTREAM_FEEDBACK_RATIO = "topology.upstream.feedback.ratio";
/**
* The time period that builtin metrics data in bucketed into.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
index b3cfd90d4d6..f9445bf2ef7 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
@@ -360,6 +360,19 @@ public static void addEventLogger(Map conf, StormTopology topolo
topology.put_to_bolts(EVENTLOGGER_COMPONENT_ID, eventLoggerBolt);
}
+ public static void addUpstreamFeedback(Map conf, StormTopology topology) {
+ Integer numExecutors = ObjectReader.getInt(conf.get(Config.TOPOLOGY_EVENTLOGGER_EXECUTORS),
+ ObjectReader.getInt(conf.get(Config.TOPOLOGY_WORKERS)));
+ if (numExecutors == null || numExecutors == 0) {
+ return;
+ }
+ String feedbackStreamId = ConfigUtils.upstreamFeedbackStreamId(conf);
+ for (Object component : allComponents(topology).values()) {
+ ComponentCommon common = getComponentCommon(component);
+ common.put_to_streams(feedbackStreamId, Thrift.outputFields(eventLoggerBoltFields()));
+ }
+ }
+
@SuppressWarnings("unchecked")
public static Map metricsConsumerBoltSpecs(Map conf, StormTopology topology) {
Map metricsConsumerBolts = new HashMap<>();
@@ -464,6 +477,10 @@ public static boolean hasEventLoggers(Map topoConf) {
return eventLoggerNum == null || ObjectReader.getInt(eventLoggerNum) > 0;
}
+ public static boolean hasUpstreamFeedback(Map topoConf) {
+ return ConfigUtils.upstreamFeedbackEnable(topoConf);
+ }
+
public static int numStartExecutors(Object component) throws InvalidTopologyException {
ComponentCommon common = getComponentCommon(component);
return Thrift.getParallelismHint(common);
@@ -538,6 +555,9 @@ protected StormTopology systemTopologyImpl(Map topoConf, StormTo
if (hasEventLoggers(topoConf)) {
addEventLogger(topoConf, ret);
}
+ if (hasUpstreamFeedback(topoConf)) {
+ addUpstreamFeedback(topoConf, ret);
+ }
addMetricComponents(topoConf, ret);
addSystemComponents(topoConf, ret);
addMetricStreams(ret);
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/Task.java b/storm-client/src/jvm/org/apache/storm/daemon/Task.java
index 45a1f2d7e3a..8020445229b 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/Task.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/Task.java
@@ -216,6 +216,27 @@ public void sendUnanchored(String stream, List
*
- * @param tuple The {@link TupleImpl} object emitted by the upstream feedback builder[cite: 1].
- * @throws ClassCastException if the Values object does not contain the expected types.
- * @throws NullPointerException if the feedbackTuple or its elements are null.
+ * @param task The {@link Task} associated with this update.
+ * @param tuple The {@link TupleImpl} emitted by the upstream feedback builder.
*/
- public void updateChildEwmaStats(TupleImpl tuple) {
- if (!this.upstreamFeedbackEnabled) {
+ public void updateChildEwmaStats(Task task, TupleImpl tuple) {
+ if (!this.upstreamFeedbackEnabled || tuple == null) {
+ return;
+ }
+
+ List values = tuple.getValues();
+ if (values == null || values.size() < 2) {
+ LOG.warn("Feedback tuple for task {} has insufficient elements (size={})",
+ task.getTaskId(), values == null ? 0 : values.size());
return;
}
- if (tuple == null) {
+ // Safe type check replaces unchecked cast and suppression
+ if (!(values.get(0) instanceof IMetricsConsumer.TaskInfo taskInfo)) {
+ LOG.warn("Unexpected type at index 0 in feedbackTuple for task {}: {}",
+ task.getTaskId(), values.get(0) == null ? "null" : values.get(0).getClass().getName());
return;
}
- Values feedbackTuple = new Values(tuple.getValues().toArray());
- if (feedbackTuple.size() < 2) {
+ if (!(values.get(1) instanceof Collection> rawDataPoints)) {
+ LOG.warn("Unexpected type at index 1 in feedbackTuple for task {}: {}",
+ task.getTaskId(), values.get(1) == null ? "null" : values.get(1).getClass().getName());
return;
}
-
- // Extract TaskInfo to retrieve the source Task ID
- IMetricsConsumer.TaskInfo taskInfo = (IMetricsConsumer.TaskInfo) feedbackTuple.get(0);
- int taskId = taskInfo.srcTaskId;
- // Extract the collection of DataPoints (e.g., Jitter EWMA)
- @SuppressWarnings("unchecked")
- Collection dataPoints = (Collection) feedbackTuple.get(1);
+ int taskId = task.getTaskId();
+ int childTaskId = taskInfo.srcTaskId;
- if (dataPoints != null) {
- // Atomically retrieve or create the map for this specific Task id
- Map taskMap = childEwmaStats.computeIfAbsent(
- taskId,
- k -> new ConcurrentHashMap()
- );
+ // Filter to only valid DataPoint instances, safely skipping any unexpected elements
+ List dataPoints = rawDataPoints.stream()
+ .filter(IMetricsConsumer.DataPoint.class::isInstance)
+ .map(IMetricsConsumer.DataPoint.class::cast)
+ .toList();
+
+ if (!dataPoints.isEmpty()) {
+ Map metricsMap =
+ getOrCreateMetricsMap(taskId, childTaskId);
- // Update the task map with the latest metrics snapshot
for (IMetricsConsumer.DataPoint dp : dataPoints) {
- taskMap.put(dp.name, dp);
+ metricsMap.put(dp.name, dp);
}
+
+ // Invalidate cached averages for this taskId since underlying data changed
+ avgCache.remove(taskId);
}
}
/**
- * Reduces the statistics collected in childStats by calculating the average
- * for each metric dimension across all tasks using Java Stream API.
- *
- *
The stream flattens the nested maps, filters for numeric values,
- * and groups them by metric name to compute the mean.
+ * Retrieves the collected metric statistics for all child tasks of a given parent task.
*
- * @return A Map containing the average value for each metric name.
+ * @param taskId The ID of the parent task.
+ * @return A map of childTaskId to metric name to {@link IMetricsConsumer.DataPoint}.
+ * Returns an empty map if feedback is disabled or no data exists.
*/
- public Map getChildEwmaStats() {
+ public Map> getChildEwmaStats(int taskId) {
if (!this.upstreamFeedbackEnabled) {
- return null;
+ return Collections.emptyMap();
+ }
+ return this.childEwmaStats.getOrDefault(taskId, Collections.emptyMap());
+ }
+
+ /**
+ * Calculates the average value for each metric across all child tasks
+ * associated with the given parent task.
+ *
+ *
Results are cached per {@code taskId} and recomputed only when the
+ * underlying data has changed (i.e. after a call to {@link #updateChildEwmaStats}).
+ *
+ * @param taskId The ID of the parent task.
+ * @return A map of metric name to computed average value.
+ * Returns an empty map if no stats are found or feedback is disabled.
+ */
+ public Map getChildEwmaAvgStats(int taskId) {
+ Map> taskStats = this.getChildEwmaStats(taskId);
+ if (taskStats.isEmpty()) {
+ return Collections.emptyMap();
+ }
+ return avgCache.computeIfAbsent(taskId, this::computeAvgStats);
+ }
+
+ private Map computeAvgStats(int taskId) {
+ Map> taskStats = this.getChildEwmaStats(taskId);
+
+ Map accumulators = new HashMap<>();
+
+ for (Map childMetrics : taskStats.values()) {
+ for (Map.Entry entry : childMetrics.entrySet()) {
+ if (entry.getValue().value instanceof Number n) {
+ accumulators.merge(
+ entry.getKey(),
+ new StatsAccumulator(n.doubleValue(), 1),
+ StatsAccumulator::combine
+ );
+ }
+ }
}
- return childEwmaStats.values().stream()
- .flatMap(taskMap -> taskMap.values().stream())
- .filter(dp -> dp.value instanceof Number)
- .collect(Collectors.groupingBy(
- dp -> dp.name,
- Collectors.averagingDouble(dp -> ((Number) dp.value).doubleValue())
+ return accumulators.entrySet().stream()
+ .collect(Collectors.toMap(
+ Map.Entry::getKey,
+ e -> e.getValue().average()
));
}
+ private Map getOrCreateMetricsMap(int taskId, int childTaskId) {
+ return childEwmaStats
+ .computeIfAbsent(taskId, k -> new ConcurrentHashMap<>())
+ .computeIfAbsent(childTaskId, k -> new ConcurrentHashMap<>());
+ }
+
+ // Functional pattern
+ private record StatsAccumulator(double sum, int count) {
+ StatsAccumulator combine(StatsAccumulator other) {
+ return new StatsAccumulator(this.sum + other.sum, this.count + other.count);
+ }
+
+ double average() {
+ return count > 0 ? sum / count : 0.0;
+ }
+ }
+
private List buildEwmaDataPoints(int taskId, Set metrics) {
if (metrics == null || metrics.isEmpty()) {
return Collections.emptyList();
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
index 65ecc2a8f73..9f099ded740 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
@@ -203,7 +203,7 @@ public void tupleActionFn(int taskId, TupleImpl tuple) throws Exception {
LOG.debug("Upstream feedback skipped.");
} else {
// update internal metrics
- this.updateChildEwmaStats(tuple);
+ this.updateChildEwmaStats(idToTask.get(taskId - idToTaskBase), tuple);
}
} else {
IBolt boltObject = (IBolt) idToTask.get(taskId - idToTaskBase).getTaskObject();
diff --git a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
index 90393149502..880c22e7d41 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
@@ -326,7 +326,7 @@ public void tupleActionFn(int taskId, TupleImpl tuple) throws Exception {
LOG.debug("Upstream feedback skipped.");
} else {
// update internal metrics
- this.updateChildEwmaStats(tuple);
+ this.updateChildEwmaStats(idToTask.get(taskId - idToTaskBase), tuple);
}
} else {
Long id = (Long) tuple.getValue(0);
From 215e3e0ba2749ce828cdc4cb13beef60c4bc8b14 Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Thu, 21 May 2026 11:04:22 +0200
Subject: [PATCH 14/21] feedback message serialization and path decoupling
---
.../org/apache/storm/executor/Executor.java | 51 +++++++++----------
.../bolt/BoltOutputCollectorImpl.java | 17 +++----
.../apache/storm/metrics2/TaskMetrics.java | 12 ++---
3 files changed, 36 insertions(+), 44 deletions(-)
diff --git a/storm-client/src/jvm/org/apache/storm/executor/Executor.java b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
index 847430b7438..05c25aaa83d 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/Executor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
@@ -31,7 +31,6 @@
import java.util.Objects;
import java.util.Queue;
import java.util.Random;
-import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
@@ -70,6 +69,7 @@
import org.apache.storm.metric.api.IMetricsConsumer;
import org.apache.storm.metrics2.PerReporterGauge;
import org.apache.storm.metrics2.RateCounter;
+import org.apache.storm.metrics2.TaskMetrics;
import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
import org.apache.storm.shade.com.google.common.collect.Lists;
import org.apache.storm.shade.net.minidev.json.JSONValue;
@@ -137,7 +137,7 @@ public abstract class Executor implements Callable, JCQueue.Consumer {
protected final String upstreamFeedbackStreamId;
protected final boolean upstreamFeedbackEnabled;
protected final Map>> childEwmaStats;
- protected final Map> avgCache = new ConcurrentHashMap<>();
+ protected final Map> avgCache;
protected Executor(WorkerState workerData, List executorId, Map credentials, String type) {
this.workerData = workerData;
@@ -198,7 +198,13 @@ protected Executor(WorkerState workerData, List executorId, Map();
+ if (this.upstreamFeedbackEnabled) {
+ this.childEwmaStats = new ConcurrentHashMap<>();
+ this.avgCache = new ConcurrentHashMap<>();
+ } else {
+ this.childEwmaStats = Collections.emptyMap();
+ this.avgCache = Collections.emptyMap();
+ }
}
public static Executor mkExecutor(WorkerState workerState, List executorId, Map credentials) {
@@ -383,15 +389,14 @@ public void metricsTick(Task task, TupleImpl tuple) {
* followed by a list of filtered DataPoints.
*
* @param taskId The ID of the task for which metrics are being collected.
- * @param metrics A set of metric names (e.g., EWMA stats) to include in the feedback.
* @return A {@link Values} object containing [TaskInfo, List],
* compatible with the metrics stream schema.
*/
- public Values buildUpstreamFeedbackTuple(int taskId, Set metrics) {
+ public Values buildUpstreamFeedbackTuple(int taskId) {
IMetricsConsumer.TaskInfo taskInfo = new IMetricsConsumer.TaskInfo(
hostname, workerTopologyContext.getThisWorkerPort(),
componentId, taskId, Time.currentTimeSecs(), -1);
- return new Values(taskInfo, buildEwmaDataPoints(taskId, metrics));
+ return new Values(taskInfo, EwmaFeedbackRecord.fromWorkerState(this.workerData, taskId));
}
/**
@@ -533,31 +538,25 @@ StatsAccumulator combine(StatsAccumulator other) {
}
}
- private List buildEwmaDataPoints(int taskId, Set metrics) {
- if (metrics == null || metrics.isEmpty()) {
- return Collections.emptyList();
- }
+ private record EwmaFeedbackRecord (double processJitter, double completeJitter, double executeJitter) {
+ private static final double VOID = -1;
- List dataPoints = new ArrayList<>(metrics.size());
- Map allGauges = workerData.getMetricRegistry().getTaskGauges(taskId);
-
- if (allGauges == null || allGauges.isEmpty()) {
- return dataPoints;
- }
-
- for (String metricName : metrics) {
- Gauge gauge = allGauges.get(metricName);
-
- if (gauge != null) {
- Object v = (gauge instanceof PerReporterGauge)
- ? ((PerReporterGauge) gauge).getValueForReporter(this)
- : gauge.getValue();
+ private static double fromGauge (Gauge> gauge) {
+ if (gauge != null && !(gauge instanceof PerReporterGauge)) {
+ Object v = gauge.getValue();
if (v instanceof Number) {
- dataPoints.add(new IMetricsConsumer.DataPoint(metricName, v));
+ return ((Number) v).doubleValue();
}
}
+ return VOID;
+ }
+
+ public static EwmaFeedbackRecord fromWorkerState(WorkerState workerData, int taskId) {
+ Map allGauges = workerData.getMetricRegistry().getTaskGauges(taskId);
+ return new EwmaFeedbackRecord(fromGauge(allGauges.get(TaskMetrics.METRIC_NAME_PROCESS_JITTER)),
+ fromGauge(allGauges.get(TaskMetrics.METRIC_NAME_COMPLETE_JITTER)),
+ fromGauge(allGauges.get(TaskMetrics.METRIC_NAME_EXECUTE_JITTER)));
}
- return dataPoints;
}
// updates v1 metric dataPoints with v2 metric API data
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
index 5fa89f6e3d5..423ea9bfb8c 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
@@ -12,10 +12,7 @@
package org.apache.storm.executor.bolt;
-import static org.apache.storm.metrics2.TaskMetrics.EWMA_METRICS_SET;
-
import java.util.Collection;
-import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -115,7 +112,6 @@ private List boltEmit(String streamId, Collection anchors, List<
MessageId msgId;
if (ackingEnabled && anchors != null) {
final Map anchorsToIds = new HashMap<>();
- final boolean sendUpstreamFeedback = isUpstreamFeedback && upstreamFeedbackRate.get();
for (Tuple a : anchors) { // perf critical path. would be nice to avoid iterator allocation here and below
Set rootIds = a.getMessageId().getAnchorsToIds().keySet();
if (rootIds.size() > 0) {
@@ -125,11 +121,6 @@ private List boltEmit(String streamId, Collection anchors, List<
putXor(anchorsToIds, rootId, edgeId);
}
}
- if (sendUpstreamFeedback) {
- int parentTask = a.getSourceTask();
- task.sendUnanchoredFeedback(upstreamFeedbackStreamId, executor.buildUpstreamFeedbackTuple(taskId, EWMA_METRICS_SET),
- parentTask, xsfer, executor.getPendingEmits());
- }
}
msgId = MessageId.makeId(anchorsToIds);
} else {
@@ -142,6 +133,14 @@ private List boltEmit(String streamId, Collection anchors, List<
if (isEventLoggers) {
task.sendToEventLogger(executor, values, executor.getComponentId(), null, random, executor.getPendingEmits());
}
+ // send upstream feedback if enabled
+ if (anchors != null && isUpstreamFeedback && upstreamFeedbackRate.get()) {
+ for (Tuple a : anchors) {
+ int parentTask = a.getSourceTask();
+ task.sendUnanchoredFeedback(upstreamFeedbackStreamId, executor.buildUpstreamFeedbackTuple(taskId),
+ parentTask, xsfer, executor.getPendingEmits());
+ }
+ }
return outTasks;
}
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 df8db5162d5..0fd21134ca5 100644
--- a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
+++ b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
@@ -31,19 +31,13 @@ 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_JITTER = "__process-jitter";
+ public static final String METRIC_NAME_PROCESS_JITTER = "__process-jitter";
private static final String METRIC_NAME_COMPLETE_LATENCY = "__complete-latency";
- private static final String METRIC_NAME_COMPLETE_JITTER = "__complete-jitter";
+ public static final String METRIC_NAME_COMPLETE_JITTER = "__complete-jitter";
private static final String METRIC_NAME_EXECUTE_LATENCY = "__execute-latency";
- private static final String METRIC_NAME_EXECUTE_JITTER = "__execute-jitter";
+ public static final String METRIC_NAME_EXECUTE_JITTER = "__execute-jitter";
private static final String METRIC_NAME_CAPACITY = "__capacity";
- public static final Set EWMA_METRICS_SET = Set.of(
- METRIC_NAME_PROCESS_JITTER,
- METRIC_NAME_COMPLETE_JITTER,
- METRIC_NAME_EXECUTE_JITTER
- );
-
private final ConcurrentMap rateCounters = new ConcurrentHashMap<>();
private final ConcurrentMap> gauges = new ConcurrentHashMap<>();
// Gauge supplier singleton factories
From 99f53e4da5948e86a30eaba6d27186019a33f731 Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Sun, 24 May 2026 17:16:38 +0200
Subject: [PATCH 15/21] feedback refactoring + JitterAwareStreamGrouping
---
conf/defaults.yaml | 2 +
.../perf/UpstreamFeedbackCompareTopo.java | 212 ++++++++++++++++++
.../starter/JitterAwareGroupingTopology.java | 125 +++++++++++
.../src/jvm/org/apache/storm/Config.java | 2 +-
.../apache/storm/daemon/GrouperFactory.java | 11 +
.../org/apache/storm/daemon/StormCommon.java | 14 +-
.../apache/storm/executor/ChildEwmaStats.java | 84 +++++++
.../storm/executor/EwmaFeedbackRecord.java | 64 ++++++
.../org/apache/storm/executor/Executor.java | 143 +-----------
.../grouping/JitterAwareStreamGrouping.java | 101 +++++++++
.../LoadAwareCustomStreamGrouping.java | 4 +
.../grouping/LoadAwareShuffleGrouping.java | 6 +
.../org/apache/storm/utils/ConfigUtils.java | 8 -
13 files changed, 629 insertions(+), 147 deletions(-)
create mode 100644 examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java
create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java
create mode 100644 storm-client/src/jvm/org/apache/storm/executor/ChildEwmaStats.java
create mode 100644 storm-client/src/jvm/org/apache/storm/executor/EwmaFeedbackRecord.java
create mode 100644 storm-client/src/jvm/org/apache/storm/grouping/JitterAwareStreamGrouping.java
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 2c3bb9e063a..7ae90ff1eb2 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -283,6 +283,8 @@ topology.state.synchronization.timeout.secs: 60
topology.stats.sample.rate: 0.05
topology.stats.ewma.enable: false
topology.stats.ewma.smoothing.factor: 0.0625
+topology.upstream.feedback.ratio: 0.01
+topology.upstream.feedback.stream: "__feedback"
topology.builtin.metrics.bucket.size.secs: 60
topology.fall.back.on.java.serialization: false
topology.worker.childopts: null
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java
new file mode 100644
index 00000000000..3125b8ee813
--- /dev/null
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java
@@ -0,0 +1,212 @@
+/*
+ * 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.perf;
+
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.JitterAwareStreamGrouping;
+import org.apache.storm.perf.utils.Helper;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+
+/**
+ * Benchmark to compare the effect of {@link Config#TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE}.
+ *
+ *
Pipeline (acked, so jitter feedback can flow): {@code gen -> source -> worker -> sink}. The
+ * {@code source -> worker} edge uses {@link JitterAwareStreamGrouping}, and each {@code worker} task adds
+ * an uneven, task-dependent processing delay so the per-task jitter diverges. With feedback enabled the
+ * grouping steers tuples toward the lower-jitter workers; with it disabled the grouping has no stats and
+ * falls back to round-robin.
+ *
+ *
The {@code worker} bolt must emit (anchored) for its jitter to be reported back to {@code source};
+ * that is why a terminal {@code sink} bolt sits downstream. Feedback also requires the EWMA jitter gauges,
+ * which this topology enables via {@link Config#TOPOLOGY_STATS_EWMA_ENABLE}.
+ *
+ *
Run the same topology twice and compare the throughput/latency printed by the perf framework:
+ *
+ * storm jar storm-perf.jar org.apache.storm.perf.UpstreamFeedbackCompareTopo 120 -c topology.upstream.feedback.enable=false
+ * storm jar storm-perf.jar org.apache.storm.perf.UpstreamFeedbackCompareTopo 120 -c topology.upstream.feedback.enable=true
+ *
+ */
+public class UpstreamFeedbackCompareTopo {
+
+ static final String SPOUT_ID = "gen";
+ static final String SOURCE_ID = "source";
+ static final String WORKER_ID = "worker";
+ static final String SINK_ID = "sink";
+
+ // Parallelism / tuning keys (override with -c =).
+ static final String SPOUT_NUM = "spout.count";
+ static final String SOURCE_NUM = "source.count";
+ static final String WORKER_NUM = "worker.count";
+ static final String SINK_NUM = "sink.count";
+ static final String WORKER_BASE_DELAY_US = "worker.base.delay.us";
+
+ private static final String FIELD = "t";
+
+ static StormTopology getTopology(Map conf) {
+ int spouts = Helper.getInt(conf, SPOUT_NUM, 1);
+ int sources = Helper.getInt(conf, SOURCE_NUM, 2);
+ int workers = Helper.getInt(conf, WORKER_NUM, 4);
+ int sinks = Helper.getInt(conf, SINK_NUM, 1);
+ long baseDelayUs = Helper.getInt(conf, WORKER_BASE_DELAY_US, 100);
+
+ TopologyBuilder builder = new TopologyBuilder();
+ builder.setSpout(SPOUT_ID, new GenSpout(), spouts);
+ builder.setBolt(SOURCE_ID, new ForwardBolt(), sources).localOrShuffleGrouping(SPOUT_ID);
+ builder.setBolt(WORKER_ID, new JitteryBolt(baseDelayUs), workers)
+ .customGrouping(SOURCE_ID, new JitterAwareStreamGrouping());
+ builder.setBolt(SINK_ID, new SinkBolt(), sinks).localOrShuffleGrouping(WORKER_ID);
+ return builder.createTopology();
+ }
+
+ public static void main(String[] args) throws Exception {
+ int runTime = -1;
+ Config topoConf = new Config();
+ if (args.length > 0) {
+ runTime = Integer.parseInt(args[0]);
+ }
+ if (args.length > 1) {
+ topoConf.putAll(Utils.findAndReadConfigFile(args[1]));
+ }
+ if (args.length > 2) {
+ System.err.println("args: [runDurationSec] [optionalConfFile]");
+ return;
+ }
+
+ // EWMA jitter gauges feed the feedback signal; enable them regardless of the toggle under test.
+ topoConf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, true);
+ topoConf.putIfAbsent(Config.TOPOLOGY_UPSTREAM_FEEDBACK_RATIO, 0.01);
+ topoConf.putIfAbsent(Config.TOPOLOGY_MAX_SPOUT_PENDING, 2000);
+ // TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE is left to the command line so the two runs differ only by it.
+ topoConf.putAll(Utils.readCommandLineOpts());
+
+ Helper.runOnClusterAndPrintMetrics(runTime, "UpstreamFeedbackCompareTopo", topoConf, getTopology(topoConf));
+ }
+
+ /** Emits a timestamp per tuple, anchored (with a msgId) so the chain is acked end to end. */
+ private static class GenSpout extends BaseRichSpout {
+ private SpoutOutputCollector collector;
+ private long msgId;
+
+ @Override
+ public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void nextTuple() {
+ collector.emit(new Values(System.currentTimeMillis()), ++msgId);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields(FIELD));
+ }
+ }
+
+ /** Forwards each tuple, anchored, onto the jitter-aware stream. */
+ private static class ForwardBolt extends BaseRichBolt {
+ private OutputCollector collector;
+
+ @Override
+ public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void execute(Tuple tuple) {
+ collector.emit(tuple, new Values(tuple.getValue(0)));
+ collector.ack(tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields(FIELD));
+ }
+ }
+
+ /** Adds an uneven, per-task processing delay, then emits anchored so its jitter is reported upstream. */
+ private static class JitteryBolt extends BaseRichBolt {
+ private final long baseDelayUs;
+ private OutputCollector collector;
+ private long taskBaselineUs;
+
+ JitteryBolt(long baseDelayUs) {
+ this.baseDelayUs = baseDelayUs;
+ }
+
+ @Override
+ public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ // Each task gets its own baseline latency so the workers' jitter profiles differ.
+ this.taskBaselineUs = baseDelayUs * context.getThisTaskIndex();
+ }
+
+ @Override
+ public void execute(Tuple tuple) {
+ busyWaitMicros(taskBaselineUs + ThreadLocalRandom.current().nextLong(0, baseDelayUs + 1));
+ collector.emit(tuple, new Values(tuple.getValue(0)));
+ collector.ack(tuple);
+ }
+
+ private static void busyWaitMicros(long micros) {
+ long deadline = System.nanoTime() + micros * 1_000L;
+ while (System.nanoTime() < deadline) {
+ Thread.onSpinWait();
+ }
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields(FIELD));
+ }
+ }
+
+ /** Terminal bolt: acks to complete the tuple tree (drives spout complete-latency). */
+ private static class SinkBolt extends BaseRichBolt {
+ private OutputCollector collector;
+
+ @Override
+ public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void execute(Tuple tuple) {
+ collector.ack(tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ // terminal
+ }
+ }
+}
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java
new file mode 100644
index 00000000000..899671b8d5c
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java
@@ -0,0 +1,125 @@
+/*
+ * 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.starter;
+
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.storm.Config;
+import org.apache.storm.grouping.JitterAwareStreamGrouping;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.testing.TestWordSpout;
+import org.apache.storm.topology.ConfigurableTopology;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+
+/**
+ * Example wiring of {@link JitterAwareStreamGrouping}.
+ *
+ *
The {@code source} bolt emits to the {@code worker} bolt through a {@link JitterAwareStreamGrouping}:
+ * each tuple is routed to the {@code worker} task that currently reports the lowest jitter. The jitter
+ * figures travel back to {@code source} as upstream feedback, which must be enabled on the topology via
+ * {@link Config#TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE}. The {@code worker} bolt below injects an artificial,
+ * uneven processing delay so the per-task jitter actually differs and the grouping has something to
+ * optimize; in a real topology the variance comes from the bolt's own work.
+ *
+ *
Until feedback data has accumulated (and on any task that has not reported yet) the grouping falls
+ * back to round-robin, so the topology behaves like an even spread at start-up.
+ */
+public class JitterAwareGroupingTopology extends ConfigurableTopology {
+
+ public static void main(String[] args) throws Exception {
+ ConfigurableTopology.start(new JitterAwareGroupingTopology(), args);
+ }
+
+ @Override
+ protected int run(String[] args) {
+ TopologyBuilder builder = new TopologyBuilder();
+
+ builder.setSpout("words", new TestWordSpout(), 2);
+ builder.setBolt("source", new ForwardBolt(), 2).shuffleGrouping("words");
+ // Route source -> worker by lowest reported jitter instead of shuffle/fields.
+ builder.setBolt("worker", new JitteryBolt(), 4)
+ .customGrouping("source", new JitterAwareStreamGrouping());
+
+ // Required: jitter feedback must flow from the children (worker) back to the parents (source).
+ conf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, true);
+ conf.put(Config.TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE, true);
+ conf.put(Config.TOPOLOGY_UPSTREAM_FEEDBACK_RATIO, 0.01);
+
+ conf.setNumWorkers(2);
+
+ String topologyName = "jitter-aware-grouping";
+ if (args != null && args.length > 0) {
+ topologyName = args[0];
+ }
+ return submit(topologyName, conf, builder);
+ }
+
+ /** Passes each word straight through to the jitter-aware stream. */
+ public static class ForwardBolt extends BaseRichBolt {
+ private OutputCollector collector;
+
+ @Override
+ public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void execute(Tuple tuple) {
+ collector.emit(tuple, new Values(tuple.getString(0)));
+ collector.ack(tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields("word"));
+ }
+ }
+
+ /** Simulates uneven per-task processing time so tasks accrue different jitter. */
+ public static class JitteryBolt extends BaseRichBolt {
+ private OutputCollector collector;
+ private long baseDelayMicros;
+
+ @Override
+ public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ // Give each task its own baseline latency so their jitter profiles diverge.
+ this.baseDelayMicros = 50L * context.getThisTaskIndex();
+ }
+
+ @Override
+ public void execute(Tuple tuple) {
+ long jitterMicros = ThreadLocalRandom.current().nextLong(0, 200);
+ busyWaitMicros(baseDelayMicros + jitterMicros);
+ collector.ack(tuple);
+ }
+
+ private static void busyWaitMicros(long micros) {
+ long deadline = System.nanoTime() + micros * 1_000L;
+ while (System.nanoTime() < deadline) {
+ Thread.onSpinWait();
+ }
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ // terminal bolt: no output
+ }
+ }
+}
diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java
index f598c333e1c..0dd0ee2384f 100644
--- a/storm-client/src/jvm/org/apache/storm/Config.java
+++ b/storm-client/src/jvm/org/apache/storm/Config.java
@@ -644,7 +644,7 @@ public class Config extends HashMap {
*
*
*
- * Defaults to 0.1 if not explicitly configured.
+ * Defaults to 0.01 if not explicitly configured.
*/
@CustomValidator(validatorClass = ConfigValidation.ZeroOneOpenIntervalValidator.class)
public static final String TOPOLOGY_UPSTREAM_FEEDBACK_RATIO = "topology.upstream.feedback.ratio";
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java b/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java
index 0f8359783c2..ac558c0c9b3 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java
@@ -20,6 +20,7 @@
import java.util.Set;
import org.apache.storm.Config;
import org.apache.storm.Thrift;
+import org.apache.storm.executor.ChildEwmaStats;
import org.apache.storm.generated.GlobalStreamId;
import org.apache.storm.generated.Grouping;
import org.apache.storm.grouping.CustomStreamGrouping;
@@ -43,6 +44,11 @@ public void refreshLoad(LoadMapping loadMapping) {
}
+ @Override
+ public void registerEwmaStats(ChildEwmaStats childEwmaStats) {
+
+ }
+
@Override
public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) {
@@ -136,6 +142,11 @@ public void refreshLoad(LoadMapping loadMapping) {
}
+ @Override
+ public void registerEwmaStats(ChildEwmaStats childEwmaStats) {
+
+ }
+
@Override
public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) {
customStreamGrouping.prepare(context, stream, targetTasks);
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
index f9445bf2ef7..886ec3727ee 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
@@ -361,18 +361,20 @@ public static void addEventLogger(Map conf, StormTopology topolo
}
public static void addUpstreamFeedback(Map conf, StormTopology topology) {
- Integer numExecutors = ObjectReader.getInt(conf.get(Config.TOPOLOGY_EVENTLOGGER_EXECUTORS),
- ObjectReader.getInt(conf.get(Config.TOPOLOGY_WORKERS)));
- if (numExecutors == null || numExecutors == 0) {
- return;
- }
+ // Only invoked when hasUpstreamFeedback(conf) is true, so declare the feedback stream on every
+ // component unconditionally. The schema must match the tuple emitted by
+ // Executor.buildUpstreamFeedbackTuple: [TaskInfo, EwmaFeedbackRecord].
String feedbackStreamId = ConfigUtils.upstreamFeedbackStreamId(conf);
for (Object component : allComponents(topology).values()) {
ComponentCommon common = getComponentCommon(component);
- common.put_to_streams(feedbackStreamId, Thrift.outputFields(eventLoggerBoltFields()));
+ common.put_to_streams(feedbackStreamId, Thrift.outputFields(upstreamFeedbackFields()));
}
}
+ public static List upstreamFeedbackFields() {
+ return Arrays.asList("task-info", "feedback");
+ }
+
@SuppressWarnings("unchecked")
public static Map metricsConsumerBoltSpecs(Map conf, StormTopology topology) {
Map metricsConsumerBolts = new HashMap<>();
diff --git a/storm-client/src/jvm/org/apache/storm/executor/ChildEwmaStats.java b/storm-client/src/jvm/org/apache/storm/executor/ChildEwmaStats.java
new file mode 100644
index 00000000000..0eb5fac88d5
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/ChildEwmaStats.java
@@ -0,0 +1,84 @@
+/*
+ * 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.executor;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.storm.metrics2.TaskMetrics;
+
+/**
+ * Thread-safe store of EWMA jitter statistics reported by downstream (child) tasks back to a parent task.
+ * The data is indexed by parent {@code taskId} so a lookup touches only that task's children:
+ * {@link #getStats} is an O(1) map lookup and {@link #update} is O(metrics), neither scanning the whole
+ * store. This keeps cost bound to a single task's child fan-out, independent of how many tasks the
+ * executor hosts.
+ */
+public class ChildEwmaStats {
+
+ private final boolean enabled;
+ private final Map>> byTask;
+
+ private static final String[] JITTER_PRIORITY = {
+ TaskMetrics.METRIC_NAME_EXECUTE_JITTER,
+ TaskMetrics.METRIC_NAME_PROCESS_JITTER,
+ TaskMetrics.METRIC_NAME_COMPLETE_JITTER,
+ };
+
+ public ChildEwmaStats(boolean enabled) {
+ this.enabled = enabled;
+ this.byTask = enabled ? new ConcurrentHashMap<>() : Collections.emptyMap();
+ }
+
+ /**
+ * Records the jitter metrics reported by {@code childTaskId} for the given parent {@code taskId}.
+ * Runs in O(metrics) by writing straight into the task's bucket; no rescanning of existing data.
+ */
+ public void update(int taskId, int childTaskId, EwmaFeedbackRecord feedback) {
+ if (!enabled) {
+ return;
+ }
+ ConcurrentHashMap> children =
+ byTask.computeIfAbsent(taskId, k -> new ConcurrentHashMap<>());
+ Map metrics = children.computeIfAbsent(childTaskId, k -> new ConcurrentHashMap<>());
+ feedback.forEachMetric(metrics::put);
+ }
+
+ /**
+ * Returns the latest reported value of each metric, per child task, for the given source
+ * {@code taskId} as {@code childTaskId -> (metricName -> value)}.
+ */
+ public Map> getStats(int taskId) {
+ if (!enabled) {
+ return Collections.emptyMap();
+ }
+ Map> children = byTask.get(taskId);
+ return children == null ? Collections.emptyMap() : children;
+ }
+
+ /**
+ * Compares two stats maps following {@link #JITTER_PRIORITY}, ascending. A missing metric
+ * is treated as {@link Double#MAX_VALUE} so it loses to any measured value.
+ */
+ public static int compareByJitter(Map a, Map b) {
+ for (String metric : JITTER_PRIORITY) {
+ int cmp = Double.compare(
+ a.getOrDefault(metric, Double.MAX_VALUE),
+ b.getOrDefault(metric, Double.MAX_VALUE));
+ if (cmp != 0) {
+ return cmp;
+ }
+ }
+ return 0;
+ }
+}
diff --git a/storm-client/src/jvm/org/apache/storm/executor/EwmaFeedbackRecord.java b/storm-client/src/jvm/org/apache/storm/executor/EwmaFeedbackRecord.java
new file mode 100644
index 00000000000..a0296c145ea
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/executor/EwmaFeedbackRecord.java
@@ -0,0 +1,64 @@
+/*
+ * 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.executor;
+
+import com.codahale.metrics.Gauge;
+import java.util.Map;
+import java.util.function.ObjDoubleConsumer;
+import org.apache.storm.daemon.worker.WorkerState;
+import org.apache.storm.metrics2.PerReporterGauge;
+import org.apache.storm.metrics2.TaskMetrics;
+
+/**
+ * Immutable snapshot of a task's jitter metrics, used as the payload of an upstream feedback tuple.
+ *
+ * @param processJitter The {@code __process-jitter} gauge value, or {@link #VOID} if absent.
+ * @param completeJitter The {@code __complete-jitter} gauge value, or {@link #VOID} if absent.
+ * @param executeJitter The {@code __execute-jitter} gauge value, or {@link #VOID} if absent.
+ */
+public record EwmaFeedbackRecord(double processJitter, double completeJitter, double executeJitter) {
+
+ private static final double VOID = -1;
+
+ private static double fromGauge(Gauge> gauge) {
+ if (gauge != null && !(gauge instanceof PerReporterGauge)) {
+ Object v = gauge.getValue();
+ if (v instanceof Number) {
+ return ((Number) v).doubleValue();
+ }
+ }
+ return VOID;
+ }
+
+ public static EwmaFeedbackRecord fromWorkerState(WorkerState workerData, int taskId) {
+ Map allGauges = workerData.getMetricRegistry().getTaskGauges(taskId);
+ return new EwmaFeedbackRecord(fromGauge(allGauges.get(TaskMetrics.METRIC_NAME_PROCESS_JITTER)),
+ fromGauge(allGauges.get(TaskMetrics.METRIC_NAME_COMPLETE_JITTER)),
+ fromGauge(allGauges.get(TaskMetrics.METRIC_NAME_EXECUTE_JITTER)));
+ }
+
+ /**
+ * Invokes {@code consumer} once for each present jitter metric.
+ */
+ public void forEachMetric(ObjDoubleConsumer consumer) {
+ if (processJitter != VOID) {
+ consumer.accept(TaskMetrics.METRIC_NAME_PROCESS_JITTER, processJitter);
+ }
+ if (completeJitter != VOID) {
+ consumer.accept(TaskMetrics.METRIC_NAME_COMPLETE_JITTER, completeJitter);
+ }
+ if (executeJitter != VOID) {
+ consumer.accept(TaskMetrics.METRIC_NAME_EXECUTE_JITTER, executeJitter);
+ }
+ }
+}
diff --git a/storm-client/src/jvm/org/apache/storm/executor/Executor.java b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
index 05c25aaa83d..6dd9203229f 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/Executor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -23,7 +23,6 @@
import java.lang.reflect.Field;
import java.net.UnknownHostException;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@@ -32,7 +31,6 @@
import java.util.Queue;
import java.util.Random;
import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -69,7 +67,6 @@
import org.apache.storm.metric.api.IMetricsConsumer;
import org.apache.storm.metrics2.PerReporterGauge;
import org.apache.storm.metrics2.RateCounter;
-import org.apache.storm.metrics2.TaskMetrics;
import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
import org.apache.storm.shade.com.google.common.collect.Lists;
import org.apache.storm.shade.net.minidev.json.JSONValue;
@@ -136,8 +133,7 @@ public abstract class Executor implements Callable, JCQueue.Consumer {
private final Integer v2MetricsTickInterval;
protected final String upstreamFeedbackStreamId;
protected final boolean upstreamFeedbackEnabled;
- protected final Map>> childEwmaStats;
- protected final Map> avgCache;
+ protected final ChildEwmaStats childEwmaStats;
protected Executor(WorkerState workerData, List executorId, Map credentials, String type) {
this.workerData = workerData;
@@ -198,12 +194,10 @@ protected Executor(WorkerState workerData, List executorId, Map();
- this.avgCache = new ConcurrentHashMap<>();
- } else {
- this.childEwmaStats = Collections.emptyMap();
- this.avgCache = Collections.emptyMap();
+ // register ewma stats for loadaware streaming grouping
+ groupers.forEach(g -> g.registerEwmaStats(childEwmaStats));
}
}
@@ -402,15 +396,14 @@ public Values buildUpstreamFeedbackTuple(int taskId) {
/**
* Updates child task statistics by unwrapping the Storm Values object.
*
- *
Extracts the {@link IMetricsConsumer.TaskInfo} and the collection of
- * {@link IMetricsConsumer.DataPoint} objects from the provided tuple. Updates
- * to the {@code childEwmaStats} map are thread-safe via {@link ConcurrentHashMap}
- * atomic operations.
+ *
Extracts the {@link IMetricsConsumer.TaskInfo} and the {@link EwmaFeedbackRecord}
+ * produced by {@link #buildUpstreamFeedbackTuple(int)} and forwards them to the thread-safe
+ * {@link ChildEwmaStats} store.
*
*
Data Mapping:
*
*
Index 0: {@link IMetricsConsumer.TaskInfo}
- *
Index 1: {@code Collection} of {@link IMetricsConsumer.DataPoint}
+ *
Index 1: {@link EwmaFeedbackRecord}
*
*
*
@@ -436,127 +429,13 @@ public void updateChildEwmaStats(Task task, TupleImpl tuple) {
return;
}
- if (!(values.get(1) instanceof Collection> rawDataPoints)) {
+ if (!(values.get(1) instanceof EwmaFeedbackRecord feedback)) {
LOG.warn("Unexpected type at index 1 in feedbackTuple for task {}: {}",
task.getTaskId(), values.get(1) == null ? "null" : values.get(1).getClass().getName());
return;
}
- int taskId = task.getTaskId();
- int childTaskId = taskInfo.srcTaskId;
-
- // Filter to only valid DataPoint instances, safely skipping any unexpected elements
- List dataPoints = rawDataPoints.stream()
- .filter(IMetricsConsumer.DataPoint.class::isInstance)
- .map(IMetricsConsumer.DataPoint.class::cast)
- .toList();
-
- if (!dataPoints.isEmpty()) {
- Map metricsMap =
- getOrCreateMetricsMap(taskId, childTaskId);
-
- for (IMetricsConsumer.DataPoint dp : dataPoints) {
- metricsMap.put(dp.name, dp);
- }
-
- // Invalidate cached averages for this taskId since underlying data changed
- avgCache.remove(taskId);
- }
- }
-
- /**
- * Retrieves the collected metric statistics for all child tasks of a given parent task.
- *
- * @param taskId The ID of the parent task.
- * @return A map of childTaskId to metric name to {@link IMetricsConsumer.DataPoint}.
- * Returns an empty map if feedback is disabled or no data exists.
- */
- public Map> getChildEwmaStats(int taskId) {
- if (!this.upstreamFeedbackEnabled) {
- return Collections.emptyMap();
- }
- return this.childEwmaStats.getOrDefault(taskId, Collections.emptyMap());
- }
-
- /**
- * Calculates the average value for each metric across all child tasks
- * associated with the given parent task.
- *
- *
Results are cached per {@code taskId} and recomputed only when the
- * underlying data has changed (i.e. after a call to {@link #updateChildEwmaStats}).
- *
- * @param taskId The ID of the parent task.
- * @return A map of metric name to computed average value.
- * Returns an empty map if no stats are found or feedback is disabled.
- */
- public Map getChildEwmaAvgStats(int taskId) {
- Map> taskStats = this.getChildEwmaStats(taskId);
- if (taskStats.isEmpty()) {
- return Collections.emptyMap();
- }
- return avgCache.computeIfAbsent(taskId, this::computeAvgStats);
- }
-
- private Map computeAvgStats(int taskId) {
- Map> taskStats = this.getChildEwmaStats(taskId);
-
- Map accumulators = new HashMap<>();
-
- for (Map childMetrics : taskStats.values()) {
- for (Map.Entry entry : childMetrics.entrySet()) {
- if (entry.getValue().value instanceof Number n) {
- accumulators.merge(
- entry.getKey(),
- new StatsAccumulator(n.doubleValue(), 1),
- StatsAccumulator::combine
- );
- }
- }
- }
-
- return accumulators.entrySet().stream()
- .collect(Collectors.toMap(
- Map.Entry::getKey,
- e -> e.getValue().average()
- ));
- }
-
- private Map getOrCreateMetricsMap(int taskId, int childTaskId) {
- return childEwmaStats
- .computeIfAbsent(taskId, k -> new ConcurrentHashMap<>())
- .computeIfAbsent(childTaskId, k -> new ConcurrentHashMap<>());
- }
-
- // Functional pattern
- private record StatsAccumulator(double sum, int count) {
- StatsAccumulator combine(StatsAccumulator other) {
- return new StatsAccumulator(this.sum + other.sum, this.count + other.count);
- }
-
- double average() {
- return count > 0 ? sum / count : 0.0;
- }
- }
-
- private record EwmaFeedbackRecord (double processJitter, double completeJitter, double executeJitter) {
- private static final double VOID = -1;
-
- private static double fromGauge (Gauge> gauge) {
- if (gauge != null && !(gauge instanceof PerReporterGauge)) {
- Object v = gauge.getValue();
- if (v instanceof Number) {
- return ((Number) v).doubleValue();
- }
- }
- return VOID;
- }
-
- public static EwmaFeedbackRecord fromWorkerState(WorkerState workerData, int taskId) {
- Map allGauges = workerData.getMetricRegistry().getTaskGauges(taskId);
- return new EwmaFeedbackRecord(fromGauge(allGauges.get(TaskMetrics.METRIC_NAME_PROCESS_JITTER)),
- fromGauge(allGauges.get(TaskMetrics.METRIC_NAME_COMPLETE_JITTER)),
- fromGauge(allGauges.get(TaskMetrics.METRIC_NAME_EXECUTE_JITTER)));
- }
+ childEwmaStats.update(task.getTaskId(), taskInfo.srcTaskId, feedback);
}
// updates v1 metric dataPoints with v2 metric API data
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/JitterAwareStreamGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/JitterAwareStreamGrouping.java
new file mode 100644
index 00000000000..a5927f4f204
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/grouping/JitterAwareStreamGrouping.java
@@ -0,0 +1,101 @@
+/*
+ * 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.grouping;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.storm.executor.ChildEwmaStats;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.task.WorkerTopologyContext;
+
+/**
+ * A {@link CustomStreamGrouping} that routes each tuple to the downstream (child) task with the lowest
+ * jitter, as reported back to the emitting task through upstream feedback and aggregated by
+ * {@link ChildEwmaStats}. Candidates are ordered with {@link ChildEwmaStats#compareByJitter}, so a lower
+ * {@code __execute-jitter} wins first, then {@code __process-jitter}, then {@code __complete-jitter}.
+ * Until a source task has any feedback data — and for targets that have not reported yet — the grouping
+ * falls back to round-robin over the target tasks, so it degrades to an even spread rather than pinning a
+ * single task.
+ */
+public class JitterAwareStreamGrouping implements LoadAwareCustomStreamGrouping {
+
+ private final AtomicInteger roundRobin = new AtomicInteger();
+ private List targetTasks;
+ private ChildEwmaStats stats;
+
+ @Override
+ public void refreshLoad(LoadMapping loadMapping) {
+ // load mapping agnostic
+ }
+
+ @Override
+ public void registerEwmaStats(ChildEwmaStats childEwmaStats) {
+ this.stats = childEwmaStats;
+ }
+
+ @Override
+ public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) {
+ this.targetTasks = targetTasks;
+ }
+
+ @Override
+ public List chooseTasks(int taskId, List values) {
+ if (targetTasks == null || targetTasks.isEmpty()) {
+ return Collections.emptyList();
+ }
+ if (targetTasks.size() == 1) {
+ return targetTasks;
+ }
+
+ if (stats == null) {
+ return roundRobin();
+ }
+
+ // childTaskId -> (metricName -> averaged value), as reported back to this source task.
+ Map> childStats = stats.getStats(taskId);
+ if (childStats.isEmpty()) {
+ return roundRobin();
+ }
+
+ Integer best = null;
+ Map bestMetrics = null;
+ boolean anyData = false;
+ for (Integer target : targetTasks) {
+ Map metrics = childStats.get(target);
+ if (metrics != null && !metrics.isEmpty()) {
+ anyData = true;
+ } else {
+ // A target with no feedback yet is treated as worst by compareByJitter (empty map).
+ metrics = Collections.emptyMap();
+ }
+ if (best == null || ChildEwmaStats.compareByJitter(metrics, bestMetrics) < 0) {
+ best = target;
+ bestMetrics = metrics;
+ }
+ }
+
+ if (!anyData) {
+ // No target has reported for this source task yet: spread evenly instead of pinning the first.
+ return roundRobin();
+ }
+ return Collections.singletonList(best);
+ }
+
+ private List roundRobin() {
+ int index = Math.floorMod(roundRobin.getAndIncrement(), targetTasks.size());
+ return Collections.singletonList(targetTasks.get(index));
+ }
+
+}
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareCustomStreamGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareCustomStreamGrouping.java
index 5a4d4a671e2..9fbf55cd84c 100644
--- a/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareCustomStreamGrouping.java
+++ b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareCustomStreamGrouping.java
@@ -12,6 +12,10 @@
package org.apache.storm.grouping;
+import org.apache.storm.executor.ChildEwmaStats;
+
public interface LoadAwareCustomStreamGrouping extends CustomStreamGrouping {
void refreshLoad(LoadMapping loadMapping);
+
+ void registerEwmaStats(ChildEwmaStats childEwmaStats);
}
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java
index 15b690dd399..c2484e0650d 100644
--- a/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java
+++ b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java
@@ -24,6 +24,7 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.storm.Config;
+import org.apache.storm.executor.ChildEwmaStats;
import org.apache.storm.generated.GlobalStreamId;
import org.apache.storm.generated.NodeInfo;
import org.apache.storm.networktopography.DNSToSwitchMapping;
@@ -111,6 +112,11 @@ public void refreshLoad(LoadMapping loadMapping) {
updateRing(loadMapping);
}
+ @Override
+ public void registerEwmaStats(ChildEwmaStats childEwmaStats) {
+ // jitter agnostic. see JitterAwareStreamGrouping
+ }
+
private void refreshLocalityGroup() {
// taskToNodePort and nodeToHost might be out of sync when they are refreshed by WorkerState
// but this is okay since it will only cause a temporary misjudgement on LocalityScope
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 b2485203d4f..57eb2f98a64 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
@@ -40,8 +40,6 @@ public class ConfigUtils {
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;
- public static final String UPSTREAM_FEEDBACK_STREAM_ID = "__feedback";
- public static final double UPSTREAM_FEEDBACK_RATIO = 0.1;
private static final Set passwordConfigKeys = new HashSet<>();
@@ -210,17 +208,11 @@ public static boolean upstreamFeedbackEnable(Map conf) {
public static String upstreamFeedbackStreamId(Map conf) {
Object value = conf.get(Config.TOPOLOGY_UPSTREAM_FEEDBACK_STREAM_ID);
- if (value == null) {
- return UPSTREAM_FEEDBACK_STREAM_ID;
- }
return ObjectReader.getString(value);
}
public static double upstreamFeedbackRatio(Map conf) {
Object value = conf.get(Config.TOPOLOGY_UPSTREAM_FEEDBACK_RATIO);
- if (value == null) {
- return UPSTREAM_FEEDBACK_RATIO;
- }
double ratio = ObjectReader.getDouble(value);
if (ratio > 0.0 && ratio < 1.0) {
return ratio;
From 7d34c90cbc5a4043bf842cc83b1430ec5439c66a Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Sat, 30 May 2026 15:48:00 +0200
Subject: [PATCH 16/21] control loop lazy update
---
.../jvm/org/apache/storm/executor/Executor.java | 14 ++++++++++++--
.../executor/bolt/BoltOutputCollectorImpl.java | 11 +++++++----
2 files changed, 19 insertions(+), 6 deletions(-)
diff --git a/storm-client/src/jvm/org/apache/storm/executor/Executor.java b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
index 6dd9203229f..1b6de123c31 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/Executor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
@@ -31,6 +31,7 @@
import java.util.Queue;
import java.util.Random;
import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -134,7 +135,7 @@ public abstract class Executor implements Callable, JCQueue.Consumer {
protected final String upstreamFeedbackStreamId;
protected final boolean upstreamFeedbackEnabled;
protected final ChildEwmaStats childEwmaStats;
-
+ private final Map ewmaRecordCache; // taskId, EwmaRecord implemented for lazy update
protected Executor(WorkerState workerData, List executorId, Map credentials, String type) {
this.workerData = workerData;
this.executorId = executorId;
@@ -196,8 +197,11 @@ protected Executor(WorkerState workerData, List executorId, Map();
// register ewma stats for loadaware streaming grouping
groupers.forEach(g -> g.registerEwmaStats(childEwmaStats));
+ } else {
+ this.ewmaRecordCache = Collections.emptyMap();
}
}
@@ -387,10 +391,16 @@ public void metricsTick(Task task, TupleImpl tuple) {
* compatible with the metrics stream schema.
*/
public Values buildUpstreamFeedbackTuple(int taskId) {
+ EwmaFeedbackRecord statsRecord = EwmaFeedbackRecord.fromWorkerState(this.workerData, taskId);
+ EwmaFeedbackRecord prevStatsRecords = this.ewmaRecordCache.put(taskId, statsRecord);
+ if (prevStatsRecords != null && prevStatsRecords.equals(statsRecord)) {
+ // lazy update, the metrics are stable, and it is not required to propagate the same value.
+ return null;
+ }
IMetricsConsumer.TaskInfo taskInfo = new IMetricsConsumer.TaskInfo(
hostname, workerTopologyContext.getThisWorkerPort(),
componentId, taskId, Time.currentTimeSecs(), -1);
- return new Values(taskInfo, EwmaFeedbackRecord.fromWorkerState(this.workerData, taskId));
+ return new Values(taskInfo, statsRecord);
}
/**
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
index 423ea9bfb8c..7ec5f3d4257 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
@@ -135,10 +135,13 @@ private List boltEmit(String streamId, Collection anchors, List<
}
// send upstream feedback if enabled
if (anchors != null && isUpstreamFeedback && upstreamFeedbackRate.get()) {
- for (Tuple a : anchors) {
- int parentTask = a.getSourceTask();
- task.sendUnanchoredFeedback(upstreamFeedbackStreamId, executor.buildUpstreamFeedbackTuple(taskId),
- parentTask, xsfer, executor.getPendingEmits());
+ Values upstreamFeedbackTuple = executor.buildUpstreamFeedbackTuple(taskId);
+ if (upstreamFeedbackTuple != null) {
+ for (Tuple a : anchors) {
+ int parentTask = a.getSourceTask();
+ task.sendUnanchoredFeedback(upstreamFeedbackStreamId, upstreamFeedbackTuple, parentTask, xsfer,
+ executor.getPendingEmits());
+ }
}
}
return outTasks;
From a76ee8d9b6acb241e14c0e49ed896c655cfae068 Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Sun, 31 May 2026 15:02:16 +0200
Subject: [PATCH 17/21] send feedback according to a periodic tick tuple
---
conf/defaults.yaml | 2 +-
.../perf/UpstreamFeedbackCompareTopo.java | 2 +-
.../starter/JitterAwareGroupingTopology.java | 2 +-
.../src/jvm/org/apache/storm/Config.java | 27 ++++---
.../src/jvm/org/apache/storm/Constants.java | 1 +
.../org/apache/storm/daemon/StormCommon.java | 1 +
.../org/apache/storm/executor/Executor.java | 70 +++++++++++++++++++
.../storm/executor/bolt/BoltExecutor.java | 8 +++
.../bolt/BoltOutputCollectorImpl.java | 30 --------
.../org/apache/storm/utils/ConfigUtils.java | 13 ++--
10 files changed, 102 insertions(+), 54 deletions(-)
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 7ae90ff1eb2..e78df30355e 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -283,7 +283,7 @@ topology.state.synchronization.timeout.secs: 60
topology.stats.sample.rate: 0.05
topology.stats.ewma.enable: false
topology.stats.ewma.smoothing.factor: 0.0625
-topology.upstream.feedback.ratio: 0.01
+topology.upstream.feedback.freq.secs: 10
topology.upstream.feedback.stream: "__feedback"
topology.builtin.metrics.bucket.size.secs: 60
topology.fall.back.on.java.serialization: false
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java
index 3125b8ee813..6f307baf5ba 100644
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java
@@ -103,7 +103,7 @@ public static void main(String[] args) throws Exception {
// EWMA jitter gauges feed the feedback signal; enable them regardless of the toggle under test.
topoConf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, true);
- topoConf.putIfAbsent(Config.TOPOLOGY_UPSTREAM_FEEDBACK_RATIO, 0.01);
+ topoConf.putIfAbsent(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS, 10);
topoConf.putIfAbsent(Config.TOPOLOGY_MAX_SPOUT_PENDING, 2000);
// TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE is left to the command line so the two runs differ only by it.
topoConf.putAll(Utils.readCommandLineOpts());
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java
index 899671b8d5c..7aa5c6e50ed 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java
@@ -59,7 +59,7 @@ protected int run(String[] args) {
// Required: jitter feedback must flow from the children (worker) back to the parents (source).
conf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, true);
conf.put(Config.TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE, true);
- conf.put(Config.TOPOLOGY_UPSTREAM_FEEDBACK_RATIO, 0.01);
+ conf.put(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS, 2);
conf.setNumWorkers(2);
diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java
index 0dd0ee2384f..d81177be8ff 100644
--- a/storm-client/src/jvm/org/apache/storm/Config.java
+++ b/storm-client/src/jvm/org/apache/storm/Config.java
@@ -623,31 +623,30 @@ public class Config extends HashMap {
@IsString
public static final String TOPOLOGY_UPSTREAM_FEEDBACK_STREAM_ID = "topology.upstream.feedback.stream";
/**
- * Configuration for the sampling rate of upstream feedback messages within the topology.
+ * The period, in seconds, between upstream feedback messages within the topology.
*
- *
This ratio defines the probability with which a task will emit a feedback tuple
- * (containing metrics such as EWMA jitter stats) back to its parent tasks.
+ *
A dedicated internal feedback tick fires on this interval; on each tick a task emits
+ * a feedback tuple (containing metrics such as EWMA jitter stats) back to its parent tasks.
* This mechanism allows parent tasks to receive performance signals from downstream
- * components to facilitate adaptive flow control or load balancing.
+ * components to facilitate adaptive flow control or load balancing. Unlike a probabilistic
+ * trigger, the period yields a deterministic, data-volume-independent feedback cadence.
*
- *
Validation: Must be a double value within the open interval (0.0, 1.0).
- * Values of 0.0 (disabled) or 1.0 (every tuple) are rejected by the
- * {@link ConfigValidation.ZeroOneOpenIntervalValidator} to prevent improper
- * configuration of the feedback loop.
+ *
Validation: Must be a positive integer (seconds).
*
*
Impact:
*
- *
Higher values provide more precise, real-time performance data but increase
+ *
Lower values provide more precise, real-time performance data but increase
* network overhead and CPU usage on the control plane.
- *
Lower values minimize the "observer effect" on the topology's throughput
- * while still providing statistical snapshots of health.
+ *
Higher values minimize the "observer effect" on the topology's throughput
+ * while still providing periodic statistical snapshots of health.
*
*
*
- * Defaults to 0.01 if not explicitly configured.
+ * Defaults to 10 if not explicitly configured.
*/
- @CustomValidator(validatorClass = ConfigValidation.ZeroOneOpenIntervalValidator.class)
- public static final String TOPOLOGY_UPSTREAM_FEEDBACK_RATIO = "topology.upstream.feedback.ratio";
+ @IsInteger
+ @IsPositiveNumber
+ public static final String TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS = "topology.upstream.feedback.freq.secs";
/**
* The time period that builtin metrics data in bucketed into.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/Constants.java b/storm-client/src/jvm/org/apache/storm/Constants.java
index c3cd0808709..c7790e56fa3 100644
--- a/storm-client/src/jvm/org/apache/storm/Constants.java
+++ b/storm-client/src/jvm/org/apache/storm/Constants.java
@@ -28,6 +28,7 @@ public class Constants {
public static final String METRICS_COMPONENT_ID_PREFIX = "__metrics_";
public static final String METRICS_STREAM_ID = "__metrics";
public static final String METRICS_TICK_STREAM_ID = "__metrics_tick";
+ public static final String FEEDBACK_TICK_STREAM_ID = "__feedback_tick";
public static final Object TOPOLOGY = "topology";
public static final String SYSTEM_TOPOLOGY = "system-topology";
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
index 886ec3727ee..5f58cdda642 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
@@ -444,6 +444,7 @@ public static void addSystemComponents(Map conf, StormTopology t
outputStreams.put(Constants.SYSTEM_TICK_STREAM_ID, Thrift.outputFields(Arrays.asList("rate_secs")));
outputStreams.put(Constants.SYSTEM_FLUSH_STREAM_ID, Thrift.outputFields(Arrays.asList()));
outputStreams.put(Constants.METRICS_TICK_STREAM_ID, Thrift.outputFields(Arrays.asList("interval")));
+ outputStreams.put(Constants.FEEDBACK_TICK_STREAM_ID, Thrift.outputFields(Arrays.asList("interval")));
Map boltConf = new HashMap<>();
boltConf.put(Config.TOPOLOGY_TASKS, 0);
diff --git a/storm-client/src/jvm/org/apache/storm/executor/Executor.java b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
index 1b6de123c31..43a4ee11d59 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/Executor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
@@ -25,11 +25,13 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Queue;
import java.util.Random;
+import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
@@ -59,6 +61,7 @@
import org.apache.storm.generated.Bolt;
import org.apache.storm.generated.Credentials;
import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.GlobalStreamId;
import org.apache.storm.generated.Grouping;
import org.apache.storm.generated.SpoutSpec;
import org.apache.storm.generated.StormTopology;
@@ -134,6 +137,9 @@ public abstract class Executor implements Callable, JCQueue.Consumer {
private final Integer v2MetricsTickInterval;
protected final String upstreamFeedbackStreamId;
protected final boolean upstreamFeedbackEnabled;
+ protected final int upstreamFeedbackFreqSecs;
+ // task ids of all upstream (source component) tasks, recipients of the periodic feedback tick
+ protected final List upstreamTaskIds;
protected final ChildEwmaStats childEwmaStats;
private final Map ewmaRecordCache; // taskId, EwmaRecord implemented for lazy update
protected Executor(WorkerState workerData, List executorId, Map credentials, String type) {
@@ -198,10 +204,14 @@ protected Executor(WorkerState workerData, List executorId, Map();
+ this.upstreamFeedbackFreqSecs = ConfigUtils.upstreamFeedbackFreqSecs(topoConf);
+ this.upstreamTaskIds = computeUpstreamTaskIds();
// register ewma stats for loadaware streaming grouping
groupers.forEach(g -> g.registerEwmaStats(childEwmaStats));
} else {
this.ewmaRecordCache = Collections.emptyMap();
+ this.upstreamFeedbackFreqSecs = 0;
+ this.upstreamTaskIds = Collections.emptyList();
}
}
@@ -587,6 +597,66 @@ private void scheduleMetricsTick(int interval) {
);
}
+ /**
+ * Collects the task ids of every upstream (source component) task. These are the recipients of
+ * the periodic upstream feedback tick. System components (e.g. ackers, metrics) are excluded.
+ */
+ private List computeUpstreamTaskIds() {
+ Set taskIds = new HashSet<>();
+ for (GlobalStreamId source : workerTopologyContext.getSources(componentId).keySet()) {
+ String sourceComponentId = source.get_componentId();
+ if (Utils.isSystemId(sourceComponentId)) {
+ continue;
+ }
+ taskIds.addAll(workerTopologyContext.getComponentTasks(sourceComponentId));
+ }
+ return new ArrayList<>(taskIds);
+ }
+
+ /**
+ * Schedules a recurring internal tick on {@link Constants#FEEDBACK_TICK_STREAM_ID}. Handling the
+ * tick (see BoltExecutor.tupleActionFn) triggers {@link #sendUpstreamFeedback(Task)}, replacing
+ * the former probabilistic per-emit trigger with a deterministic periodic one.
+ */
+ protected void scheduleUpstreamFeedbackTick(int interval) {
+ StormTimer timerTask = workerData.getUserTimer();
+ timerTask.scheduleRecurring(interval, interval,
+ () -> {
+ TupleImpl tuple =
+ new TupleImpl(workerTopologyContext, new Values(interval), Constants.SYSTEM_COMPONENT_ID,
+ (int) Constants.SYSTEM_TASK_ID, Constants.FEEDBACK_TICK_STREAM_ID);
+ AddressedTuple feedbackTickTuple = new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple);
+ try {
+ receiveQueue.publish(feedbackTickTuple);
+ receiveQueue.flush(); // avoid buffering
+ } catch (InterruptedException e) {
+ LOG.warn("Thread interrupted when publishing upstream feedback tick. Setting interrupt flag.");
+ Thread.currentThread().interrupt();
+ return;
+ }
+ }
+ );
+ }
+
+ /**
+ * Sends an upstream feedback tuple for the given task to all of its upstream tasks. Invoked on
+ * each feedback tick. The snapshot is built by {@link #buildUpstreamFeedbackTuple(int)}, which
+ * returns {@code null} (and thus skips the send) when the metrics are stable (lazy update).
+ */
+ public void sendUpstreamFeedback(Task task) {
+ if (!upstreamFeedbackEnabled) {
+ return;
+ }
+ Values feedbackTuple = buildUpstreamFeedbackTuple(task.getTaskId());
+ if (feedbackTuple == null) {
+ return;
+ }
+ for (int parentTask : upstreamTaskIds) {
+ task.sendUnanchoredFeedback(upstreamFeedbackStreamId, feedbackTuple, parentTask,
+ executorTransfer, pendingEmits);
+ }
+ }
+
protected void setupTicks(boolean isSpout) {
final Integer tickTimeSecs = ObjectReader.getInt(topoConf.get(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS), null);
if (tickTimeSecs != null) {
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
index 9f099ded740..ebcf7587ed9 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
@@ -131,6 +131,9 @@ public void init(ArrayList idToTask, int idToTaskBase) throws InterruptedE
LOG.info("Prepared bolt {}:{}", componentId, taskIds);
setupTicks(false);
setupMetrics();
+ if (upstreamFeedbackEnabled) {
+ scheduleUpstreamFeedbackTick(upstreamFeedbackFreqSecs);
+ }
}
@Override
@@ -198,6 +201,11 @@ public void tupleActionFn(int taskId, TupleImpl tuple) throws Exception {
outputCollector.flush();
} else if (Constants.METRICS_TICK_STREAM_ID.equals(streamId)) {
metricsTick(idToTask.get(taskId - idToTaskBase), tuple);
+ } else if (Constants.FEEDBACK_TICK_STREAM_ID.equals(streamId)) {
+ if (this.upstreamFeedbackEnabled) {
+ // periodic trigger: emit this task's feedback snapshot to its upstream tasks
+ sendUpstreamFeedback(idToTask.get(taskId - idToTaskBase));
+ }
} else if (this.upstreamFeedbackStreamId.equals(streamId)) {
if (!this.upstreamFeedbackEnabled) {
LOG.debug("Upstream feedback skipped.");
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
index 7ec5f3d4257..886a00c7f89 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
@@ -18,9 +18,7 @@
import java.util.Map;
import java.util.Random;
import java.util.Set;
-import java.util.function.Supplier;
import org.apache.storm.daemon.Acker;
-import org.apache.storm.daemon.StormCommon;
import org.apache.storm.daemon.Task;
import org.apache.storm.executor.ExecutorTransfer;
import org.apache.storm.hooks.info.BoltAckInfo;
@@ -31,7 +29,6 @@
import org.apache.storm.tuple.Tuple;
import org.apache.storm.tuple.TupleImpl;
import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.ConfigUtils;
import org.apache.storm.utils.Time;
import org.apache.storm.utils.Utils;
import org.slf4j.Logger;
@@ -46,9 +43,6 @@ public class BoltOutputCollectorImpl implements IOutputCollector {
private final int taskId;
private final Random random;
private final boolean isEventLoggers;
- private final boolean isUpstreamFeedback;
- private final String upstreamFeedbackStreamId;
- private Supplier upstreamFeedbackRate;
private final ExecutorTransfer xsfer;
private final boolean isDebug;
private boolean ackingEnabled;
@@ -63,19 +57,6 @@ public BoltOutputCollectorImpl(BoltExecutor executor, Task taskData, Random rand
this.ackingEnabled = ackingEnabled;
this.isDebug = isDebug;
this.xsfer = executor.getExecutorTransfer();
-
- // configure the upstream feedback if enabled.
- Map conf = executor.getTopoConf();
- if (StormCommon.hasUpstreamFeedback(conf)) {
- this.isUpstreamFeedback = true;
- this.upstreamFeedbackStreamId = ConfigUtils.upstreamFeedbackStreamId(conf);
- double ratio = ConfigUtils.upstreamFeedbackRatio(conf);
- this.upstreamFeedbackRate = () -> random.nextDouble() < ratio;
- } else {
- // explicitly declare
- this.isUpstreamFeedback = false;
- this.upstreamFeedbackStreamId = "__NOT_SET__";
- }
}
@Override
@@ -133,17 +114,6 @@ private List boltEmit(String streamId, Collection anchors, List<
if (isEventLoggers) {
task.sendToEventLogger(executor, values, executor.getComponentId(), null, random, executor.getPendingEmits());
}
- // send upstream feedback if enabled
- if (anchors != null && isUpstreamFeedback && upstreamFeedbackRate.get()) {
- Values upstreamFeedbackTuple = executor.buildUpstreamFeedbackTuple(taskId);
- if (upstreamFeedbackTuple != null) {
- for (Tuple a : anchors) {
- int parentTask = a.getSourceTask();
- task.sendUnanchoredFeedback(upstreamFeedbackStreamId, upstreamFeedbackTuple, parentTask, xsfer,
- executor.getPendingEmits());
- }
- }
- }
return outTasks;
}
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 57eb2f98a64..ce12649e289 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
@@ -211,15 +211,14 @@ public static String upstreamFeedbackStreamId(Map conf) {
return ObjectReader.getString(value);
}
- public static double upstreamFeedbackRatio(Map conf) {
- Object value = conf.get(Config.TOPOLOGY_UPSTREAM_FEEDBACK_RATIO);
- double ratio = ObjectReader.getDouble(value);
- if (ratio > 0.0 && ratio < 1.0) {
- return ratio;
+ public static int upstreamFeedbackFreqSecs(Map conf) {
+ int freqSecs = ObjectReader.getInt(conf.get(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS), 10);
+ if (freqSecs > 0) {
+ return freqSecs;
}
throw new IllegalArgumentException(
- "Illegal " + Config.TOPOLOGY_UPSTREAM_FEEDBACK_RATIO
- + " in conf: " + ratio + " must be in (0, 1)");
+ "Illegal " + Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS
+ + " in conf: " + freqSecs + " must be > 0");
}
public static BooleanSupplier mkStatsSampler(Map conf) {
From c9381960c4cf69e437d0016a90aeeaa2862d5346 Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Sat, 6 Jun 2026 15:55:06 +0200
Subject: [PATCH 18/21] minor changes
---
conf/defaults.yaml | 1 +
.../jvm/org/apache/storm/serialization/SerializationFactory.java | 1 +
2 files changed, 2 insertions(+)
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 98ae6872ae1..78ec52cf7b1 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -288,6 +288,7 @@ topology.stats.ewma.enable: false
topology.stats.ewma.smoothing.factor: 0.0625
topology.upstream.feedback.freq.secs: 10
topology.upstream.feedback.stream: "__feedback"
+topology.upstream.feedback.enable: false
topology.builtin.metrics.bucket.size.secs: 60
topology.fall.back.on.java.serialization: false
topology.worker.childopts: null
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java b/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java
index eb734aee340..42d0390db8f 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java
@@ -74,6 +74,7 @@ public static Kryo getKryo(Map conf) {
k.register(Values.class);
k.register(org.apache.storm.metric.api.IMetricsConsumer.DataPoint.class);
k.register(org.apache.storm.metric.api.IMetricsConsumer.TaskInfo.class);
+ k.register(org.apache.storm.executor.EwmaFeedbackRecord.class);
k.register(ConsList.class);
k.register(BackPressureStatus.class);
k.register(NodeInfo.class);
From d8e7cda4b816e4a317329a28ff08444c47e7c1bb Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Sun, 7 Jun 2026 16:14:06 +0200
Subject: [PATCH 19/21] add jitter aware grouping topology bench
---
.../perf/JitterAwareGroupingTopology.java | 296 ++++++++++++++++++
.../starter/JitterAwareGroupingTopology.java | 125 --------
2 files changed, 296 insertions(+), 125 deletions(-)
create mode 100644 examples/storm-perf/src/main/java/org/apache/storm/perf/JitterAwareGroupingTopology.java
delete mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/JitterAwareGroupingTopology.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/JitterAwareGroupingTopology.java
new file mode 100644
index 00000000000..935a82351ea
--- /dev/null
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/JitterAwareGroupingTopology.java
@@ -0,0 +1,296 @@
+/*
+ * 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.perf;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.locks.LockSupport;
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.JitterAwareStreamGrouping;
+import org.apache.storm.perf.spout.FileReadSpout;
+import org.apache.storm.perf.utils.Helper;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+
+/**
+ * Benchmark for {@link JitterAwareStreamGrouping} in a word-count pipeline where worker tasks have
+ * artificially skewed processing latencies.
+ *
+ *
{@code JitteryWorkerBolt} tasks have task-index-dependent processing delays. Task 0 is fast;
+ * each subsequent task is progressively slower. This mimics real-world conditions (GC pressure,
+ * I/O, resource contention) where downstream tasks diverge in responsiveness. With upstream
+ * feedback enabled, {@link JitterAwareStreamGrouping} routes more tuples to the fastest tasks,
+ * improving throughput and reducing complete latency by ≥10% compared to plain round-robin.
+ *
+ *
Run the baseline and the jitter-aware run back-to-back to compare:
+ *
+ * # Baseline: JitterAwareStreamGrouping falls back to round-robin (no feedback stats).
+ * storm jar storm-perf.jar org.apache.storm.perf.JitterAwareGroupingTopology 120 \
+ * -c topology.upstream.feedback.enable=false
+ *
+ * # Jitter-aware: grouping steers tuples to lowest-jitter workers.
+ * storm jar storm-perf.jar org.apache.storm.perf.JitterAwareGroupingTopology 120 \
+ * -c topology.upstream.feedback.enable=true
+ *
+ *
+ *
Tuning knobs (pass with {@code -c key=value}):
+ *
+ *
{@code spout.count} — number of spout tasks (default 1)
+ *
{@code splitter.count} — number of splitter tasks (default 2)
+ *
{@code worker.count} — number of jittery worker tasks (default 4)
+ *
{@code sink.count} — number of sink tasks (default 1)
+ *
{@code input.file} — path to a text file whose lines are treated as sentences
+ * (e.g. {@code src/main/sampledata/randomwords.txt})
+ *
{@code worker.base.delay.us} — per-index processing delay step in µs (default 2000).
+ * Task {@code i} parks for {@code i * base} µs plus up to {@code base} µs of random noise,
+ * so a 4-worker setup has delays of ~0-2, ~2-4, ~4-6, ~6-8 ms. Must be ≥1000 µs so
+ * the EWMA latency gauge (millisecond resolution) sees distinct values per task.
+ *
+ */
+public class JitterAwareGroupingTopology {
+
+ public static final String TOPOLOGY_NAME = "JitterAwareGroupingTopology";
+
+ static final String SPOUT_ID = "gen";
+ static final String SPLITTER_ID = "splitter";
+ static final String WORKER_ID = "worker";
+ static final String SINK_ID = "sink";
+
+ static final String SPOUT_NUM = "spout.count";
+ static final String SPLITTER_NUM = "splitter.count";
+ static final String WORKER_NUM = "worker.count";
+ static final String SINK_NUM = "sink.count";
+ static final String INPUT_FILE = "input.file";
+ static final String WORKER_BASE_DELAY_US = "worker.base.delay.us";
+
+ private static final String FIELD_SENTENCE = "sentence";
+ private static final String FIELD_WORD = "word";
+ private static final String FIELD_COUNT = "count";
+
+ static StormTopology getTopology(Map conf) {
+ int spouts = Helper.getInt(conf, SPOUT_NUM, 1);
+ int splitters = Helper.getInt(conf, SPLITTER_NUM, 2);
+ int workers = Helper.getInt(conf, WORKER_NUM, 4);
+ int sinks = Helper.getInt(conf, SINK_NUM, 1);
+ long baseDelayUs = Helper.getInt(conf, WORKER_BASE_DELAY_US, 2000);
+ String inputFile = Helper.getStr(conf, INPUT_FILE);
+
+ TopologyBuilder builder = new TopologyBuilder();
+ builder.setSpout(SPOUT_ID, new GenSpout(inputFile), spouts);
+ builder.setBolt(SPLITTER_ID, new SplitterBolt(), splitters)
+ .localOrShuffleGrouping(SPOUT_ID);
+ builder.setBolt(WORKER_ID, new JitteryWorkerBolt(baseDelayUs), workers)
+ .customGrouping(SPLITTER_ID, new JitterAwareStreamGrouping());
+ builder.setBolt(SINK_ID, new SinkBolt(), sinks)
+ .localOrShuffleGrouping(WORKER_ID);
+
+ return builder.createTopology();
+ }
+
+ public static void main(String[] args) throws Exception {
+ int runTime = -1;
+ Config topoConf = new Config();
+ if (args.length > 0) {
+ runTime = Integer.parseInt(args[0]);
+ }
+ if (args.length > 1) {
+ topoConf.putAll(Utils.findAndReadConfigFile(args[1]));
+ }
+ if (args.length > 2) {
+ System.err.println("args: [runDurationSec] [optionalConfFile]");
+ return;
+ }
+
+ topoConf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, true);
+ topoConf.putIfAbsent(Config.TOPOLOGY_MAX_SPOUT_PENDING, 4000);
+ topoConf.putIfAbsent(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS, 10);
+ topoConf.putAll(Utils.readCommandLineOpts());
+
+ Helper.runOnClusterAndPrintMetrics(runTime, TOPOLOGY_NAME, topoConf, getTopology(topoConf));
+ }
+
+ /**
+ * Emits anchored sentences loaded from {@code input.file} at maximum rate. The file is read
+ * once into memory during {@link #open} and replayed in a round-robin loop. Anchoring (with a
+ * msgId) ensures Storm tracks each tuple tree to completion, so spout complete-latency is a
+ * reliable end-to-end signal.
+ */
+ private static class GenSpout extends BaseRichSpout {
+ private final String filePath;
+ private SpoutOutputCollector collector;
+ private List lines;
+ private int lineIdx;
+ private long msgId;
+
+ GenSpout(String filePath) {
+ this.filePath = filePath;
+ }
+
+ @Override
+ public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+ this.collector = collector;
+ try {
+ this.lines = FileReadSpout.readLines(new FileInputStream(filePath));
+ } catch (IOException e) {
+ throw new RuntimeException("Cannot open input file: " + filePath, e);
+ }
+ if (lines.isEmpty()) {
+ throw new RuntimeException("Input file is empty: " + filePath);
+ }
+ }
+
+ @Override
+ public void nextTuple() {
+ String sentence = lines.get(lineIdx++ % lines.size());
+ collector.emit(new Values(sentence), ++msgId);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields(FIELD_SENTENCE));
+ }
+ }
+
+ /**
+ * Splits each incoming sentence into words and emits one tuple per word, anchored so the ack
+ * tree extends to the downstream worker.
+ */
+ private static class SplitterBolt extends BaseRichBolt {
+ private OutputCollector collector;
+
+ @Override
+ public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void execute(Tuple tuple) {
+ String sentence = tuple.getString(0);
+ for (String word : sentence.split("\\s+")) {
+ collector.emit(tuple, new Values(word));
+ }
+ collector.ack(tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields(FIELD_WORD));
+ }
+ }
+
+ /**
+ * Counts words and adds a task-index-proportional processing delay to produce deliberately
+ * skewed jitter profiles across tasks.
+ *
+ *
Task {@code i} parks for {@code i * baseDelayUs} µs plus up to {@code baseDelayUs} µs of
+ * random noise per tuple. For a 4-task setup with the default {@code baseDelayUs = 2000}:
+ *
+ *
Task 0: ~0–2 ms (fast)
+ *
Task 1: ~2–4 ms
+ *
Task 2: ~4–6 ms
+ *
Task 3: ~6–8 ms (slow)
+ *
+ * Delays are millisecond-scale so the EWMA execute-latency gauge (which stores values in ms)
+ * records distinct values per task. {@link JitterAwareStreamGrouping} then steers tuples toward
+ * the task with the lowest EWMA execute-latency when feedback is enabled.
+ *
+ *
{@link LockSupport#parkNanos} is used instead of a spin loop so slow tasks yield the CPU
+ * and do not starve the fast task's executor thread.
+ */
+ private static class JitteryWorkerBolt extends BaseRichBolt {
+ private final long baseDelayUs;
+ private OutputCollector collector;
+ private long taskBaselineNs;
+ private long baseDelayNs;
+ private final Map counts = new HashMap<>();
+
+ JitteryWorkerBolt(long baseDelayUs) {
+ this.baseDelayUs = baseDelayUs;
+ }
+
+ @Override
+ public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ this.baseDelayNs = baseDelayUs * 1_000L;
+ // Higher task indices get proportionally larger baseline delays.
+ this.taskBaselineNs = baseDelayNs * context.getThisTaskIndex();
+ }
+
+ @Override
+ public void execute(Tuple tuple) {
+ String word = tuple.getString(0);
+ counts.merge(word, 1, Integer::sum);
+ int count = counts.get(word);
+
+ long noiseNs = (long) (ThreadLocalRandom.current().nextDouble() * baseDelayNs);
+ long sleepNs = taskBaselineNs + noiseNs;
+ if (sleepNs > 0) {
+ LockSupport.parkNanos(sleepNs);
+ }
+
+ // Emit anchored so the ack chain continues to SinkBolt, and so execute/process jitter
+ // is measured and reported back to SplitterBolt via upstream feedback.
+ collector.emit(tuple, new Values(word, count));
+ collector.ack(tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields(FIELD_WORD, FIELD_COUNT));
+ }
+ }
+
+ /**
+ * Terminal bolt: acks each tuple to complete the tuple tree and drive spout complete-latency.
+ */
+ private static class SinkBolt extends BaseRichBolt {
+ private OutputCollector collector;
+
+ @Override
+ public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void execute(Tuple tuple) {
+ collector.ack(tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ // terminal — no output
+ }
+ }
+}
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java
deleted file mode 100644
index 7aa5c6e50ed..00000000000
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/JitterAwareGroupingTopology.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.starter;
-
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-import org.apache.storm.Config;
-import org.apache.storm.grouping.JitterAwareStreamGrouping;
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.testing.TestWordSpout;
-import org.apache.storm.topology.ConfigurableTopology;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.topology.base.BaseRichBolt;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.tuple.Values;
-
-/**
- * Example wiring of {@link JitterAwareStreamGrouping}.
- *
- *
The {@code source} bolt emits to the {@code worker} bolt through a {@link JitterAwareStreamGrouping}:
- * each tuple is routed to the {@code worker} task that currently reports the lowest jitter. The jitter
- * figures travel back to {@code source} as upstream feedback, which must be enabled on the topology via
- * {@link Config#TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE}. The {@code worker} bolt below injects an artificial,
- * uneven processing delay so the per-task jitter actually differs and the grouping has something to
- * optimize; in a real topology the variance comes from the bolt's own work.
- *
- *
Until feedback data has accumulated (and on any task that has not reported yet) the grouping falls
- * back to round-robin, so the topology behaves like an even spread at start-up.
- */
-public class JitterAwareGroupingTopology extends ConfigurableTopology {
-
- public static void main(String[] args) throws Exception {
- ConfigurableTopology.start(new JitterAwareGroupingTopology(), args);
- }
-
- @Override
- protected int run(String[] args) {
- TopologyBuilder builder = new TopologyBuilder();
-
- builder.setSpout("words", new TestWordSpout(), 2);
- builder.setBolt("source", new ForwardBolt(), 2).shuffleGrouping("words");
- // Route source -> worker by lowest reported jitter instead of shuffle/fields.
- builder.setBolt("worker", new JitteryBolt(), 4)
- .customGrouping("source", new JitterAwareStreamGrouping());
-
- // Required: jitter feedback must flow from the children (worker) back to the parents (source).
- conf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, true);
- conf.put(Config.TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE, true);
- conf.put(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS, 2);
-
- conf.setNumWorkers(2);
-
- String topologyName = "jitter-aware-grouping";
- if (args != null && args.length > 0) {
- topologyName = args[0];
- }
- return submit(topologyName, conf, builder);
- }
-
- /** Passes each word straight through to the jitter-aware stream. */
- public static class ForwardBolt extends BaseRichBolt {
- private OutputCollector collector;
-
- @Override
- public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
- this.collector = collector;
- }
-
- @Override
- public void execute(Tuple tuple) {
- collector.emit(tuple, new Values(tuple.getString(0)));
- collector.ack(tuple);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(new Fields("word"));
- }
- }
-
- /** Simulates uneven per-task processing time so tasks accrue different jitter. */
- public static class JitteryBolt extends BaseRichBolt {
- private OutputCollector collector;
- private long baseDelayMicros;
-
- @Override
- public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
- this.collector = collector;
- // Give each task its own baseline latency so their jitter profiles diverge.
- this.baseDelayMicros = 50L * context.getThisTaskIndex();
- }
-
- @Override
- public void execute(Tuple tuple) {
- long jitterMicros = ThreadLocalRandom.current().nextLong(0, 200);
- busyWaitMicros(baseDelayMicros + jitterMicros);
- collector.ack(tuple);
- }
-
- private static void busyWaitMicros(long micros) {
- long deadline = System.nanoTime() + micros * 1_000L;
- while (System.nanoTime() < deadline) {
- Thread.onSpinWait();
- }
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- // terminal bolt: no output
- }
- }
-}
From 2b4d8ad43a0d9522bf6b24862538a6e5e6f0306b Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Sun, 7 Jun 2026 18:40:19 +0200
Subject: [PATCH 20/21] remove not necessary UpstreamFeedbackCompareTopo.java
---
.../perf/UpstreamFeedbackCompareTopo.java | 212 ------------------
1 file changed, 212 deletions(-)
delete mode 100644 examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java
deleted file mode 100644
index 6f307baf5ba..00000000000
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/UpstreamFeedbackCompareTopo.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * 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.perf;
-
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-import org.apache.storm.Config;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.grouping.JitterAwareStreamGrouping;
-import org.apache.storm.perf.utils.Helper;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.topology.base.BaseRichBolt;
-import org.apache.storm.topology.base.BaseRichSpout;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-
-/**
- * Benchmark to compare the effect of {@link Config#TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE}.
- *
- *
Pipeline (acked, so jitter feedback can flow): {@code gen -> source -> worker -> sink}. The
- * {@code source -> worker} edge uses {@link JitterAwareStreamGrouping}, and each {@code worker} task adds
- * an uneven, task-dependent processing delay so the per-task jitter diverges. With feedback enabled the
- * grouping steers tuples toward the lower-jitter workers; with it disabled the grouping has no stats and
- * falls back to round-robin.
- *
- *
The {@code worker} bolt must emit (anchored) for its jitter to be reported back to {@code source};
- * that is why a terminal {@code sink} bolt sits downstream. Feedback also requires the EWMA jitter gauges,
- * which this topology enables via {@link Config#TOPOLOGY_STATS_EWMA_ENABLE}.
- *
- *
Run the same topology twice and compare the throughput/latency printed by the perf framework:
- *
- * storm jar storm-perf.jar org.apache.storm.perf.UpstreamFeedbackCompareTopo 120 -c topology.upstream.feedback.enable=false
- * storm jar storm-perf.jar org.apache.storm.perf.UpstreamFeedbackCompareTopo 120 -c topology.upstream.feedback.enable=true
- *
- */
-public class UpstreamFeedbackCompareTopo {
-
- static final String SPOUT_ID = "gen";
- static final String SOURCE_ID = "source";
- static final String WORKER_ID = "worker";
- static final String SINK_ID = "sink";
-
- // Parallelism / tuning keys (override with -c =).
- static final String SPOUT_NUM = "spout.count";
- static final String SOURCE_NUM = "source.count";
- static final String WORKER_NUM = "worker.count";
- static final String SINK_NUM = "sink.count";
- static final String WORKER_BASE_DELAY_US = "worker.base.delay.us";
-
- private static final String FIELD = "t";
-
- static StormTopology getTopology(Map conf) {
- int spouts = Helper.getInt(conf, SPOUT_NUM, 1);
- int sources = Helper.getInt(conf, SOURCE_NUM, 2);
- int workers = Helper.getInt(conf, WORKER_NUM, 4);
- int sinks = Helper.getInt(conf, SINK_NUM, 1);
- long baseDelayUs = Helper.getInt(conf, WORKER_BASE_DELAY_US, 100);
-
- TopologyBuilder builder = new TopologyBuilder();
- builder.setSpout(SPOUT_ID, new GenSpout(), spouts);
- builder.setBolt(SOURCE_ID, new ForwardBolt(), sources).localOrShuffleGrouping(SPOUT_ID);
- builder.setBolt(WORKER_ID, new JitteryBolt(baseDelayUs), workers)
- .customGrouping(SOURCE_ID, new JitterAwareStreamGrouping());
- builder.setBolt(SINK_ID, new SinkBolt(), sinks).localOrShuffleGrouping(WORKER_ID);
- return builder.createTopology();
- }
-
- public static void main(String[] args) throws Exception {
- int runTime = -1;
- Config topoConf = new Config();
- if (args.length > 0) {
- runTime = Integer.parseInt(args[0]);
- }
- if (args.length > 1) {
- topoConf.putAll(Utils.findAndReadConfigFile(args[1]));
- }
- if (args.length > 2) {
- System.err.println("args: [runDurationSec] [optionalConfFile]");
- return;
- }
-
- // EWMA jitter gauges feed the feedback signal; enable them regardless of the toggle under test.
- topoConf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, true);
- topoConf.putIfAbsent(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS, 10);
- topoConf.putIfAbsent(Config.TOPOLOGY_MAX_SPOUT_PENDING, 2000);
- // TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE is left to the command line so the two runs differ only by it.
- topoConf.putAll(Utils.readCommandLineOpts());
-
- Helper.runOnClusterAndPrintMetrics(runTime, "UpstreamFeedbackCompareTopo", topoConf, getTopology(topoConf));
- }
-
- /** Emits a timestamp per tuple, anchored (with a msgId) so the chain is acked end to end. */
- private static class GenSpout extends BaseRichSpout {
- private SpoutOutputCollector collector;
- private long msgId;
-
- @Override
- public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
- this.collector = collector;
- }
-
- @Override
- public void nextTuple() {
- collector.emit(new Values(System.currentTimeMillis()), ++msgId);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(new Fields(FIELD));
- }
- }
-
- /** Forwards each tuple, anchored, onto the jitter-aware stream. */
- private static class ForwardBolt extends BaseRichBolt {
- private OutputCollector collector;
-
- @Override
- public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
- this.collector = collector;
- }
-
- @Override
- public void execute(Tuple tuple) {
- collector.emit(tuple, new Values(tuple.getValue(0)));
- collector.ack(tuple);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(new Fields(FIELD));
- }
- }
-
- /** Adds an uneven, per-task processing delay, then emits anchored so its jitter is reported upstream. */
- private static class JitteryBolt extends BaseRichBolt {
- private final long baseDelayUs;
- private OutputCollector collector;
- private long taskBaselineUs;
-
- JitteryBolt(long baseDelayUs) {
- this.baseDelayUs = baseDelayUs;
- }
-
- @Override
- public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
- this.collector = collector;
- // Each task gets its own baseline latency so the workers' jitter profiles differ.
- this.taskBaselineUs = baseDelayUs * context.getThisTaskIndex();
- }
-
- @Override
- public void execute(Tuple tuple) {
- busyWaitMicros(taskBaselineUs + ThreadLocalRandom.current().nextLong(0, baseDelayUs + 1));
- collector.emit(tuple, new Values(tuple.getValue(0)));
- collector.ack(tuple);
- }
-
- private static void busyWaitMicros(long micros) {
- long deadline = System.nanoTime() + micros * 1_000L;
- while (System.nanoTime() < deadline) {
- Thread.onSpinWait();
- }
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(new Fields(FIELD));
- }
- }
-
- /** Terminal bolt: acks to complete the tuple tree (drives spout complete-latency). */
- private static class SinkBolt extends BaseRichBolt {
- private OutputCollector collector;
-
- @Override
- public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
- this.collector = collector;
- }
-
- @Override
- public void execute(Tuple tuple) {
- collector.ack(tuple);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- // terminal
- }
- }
-}
From babd3d9e398719269f24d51d39f11b713027a8ca Mon Sep 17 00:00:00 2001
From: Gianluca Graziadei
Date: Sun, 7 Jun 2026 18:50:18 +0200
Subject: [PATCH 21/21] remove unnecessary import, FT for registering
FEEDBACK_TICK_STREAM
---
storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java | 4 +++-
.../src/jvm/org/apache/storm/metrics2/TaskMetrics.java | 2 --
2 files changed, 3 insertions(+), 3 deletions(-)
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
index 5f58cdda642..b144e39abaf 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
@@ -444,7 +444,9 @@ public static void addSystemComponents(Map conf, StormTopology t
outputStreams.put(Constants.SYSTEM_TICK_STREAM_ID, Thrift.outputFields(Arrays.asList("rate_secs")));
outputStreams.put(Constants.SYSTEM_FLUSH_STREAM_ID, Thrift.outputFields(Arrays.asList()));
outputStreams.put(Constants.METRICS_TICK_STREAM_ID, Thrift.outputFields(Arrays.asList("interval")));
- outputStreams.put(Constants.FEEDBACK_TICK_STREAM_ID, Thrift.outputFields(Arrays.asList("interval")));
+ if (ConfigUtils.upstreamFeedbackEnable(conf)) {
+ outputStreams.put(Constants.FEEDBACK_TICK_STREAM_ID, Thrift.outputFields(Arrays.asList("interval")));
+ }
Map boltConf = new HashMap<>();
boltConf.put(Config.TOPOLOGY_TASKS, 0);
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 0fd21134ca5..059109b0bd8 100644
--- a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
+++ b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
@@ -13,9 +13,7 @@
package org.apache.storm.metrics2;
import com.codahale.metrics.Gauge;
-import java.util.HashSet;
import java.util.Map;
-import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.function.Supplier;