diff --git a/conf/defaults.yaml b/conf/defaults.yaml index d5d6bb45139..4fd6c561a22 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -286,6 +286,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.freq.secs: 10 +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/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..7bc53476f3c --- /dev/null +++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/JitterAwareGroupingTopology.java @@ -0,0 +1,349 @@ +/* + * 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.CustomStreamGrouping; +import org.apache.storm.grouping.JitterAwareStreamGrouping; +import org.apache.storm.grouping.LoadAwareShuffleGrouping; +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 latency dispersion (jitter). + * + *

Pipeline: {@code GenSpout -> SplitterBolt -> JitteryWorkerBolt -> SinkBolt} + * + *

{@code JitteryWorkerBolt} tasks have task-index-dependent latency jitter: task 0 is + * perfectly steady and each higher index is progressively jitterier, because the per-tuple noise + * width grows with the task index over a constant floor. This deliberately exercises the signal + * the grouping ranks on — RFC-1889 jitter (the EWMA of {@code |Δlatency|}), which measures dispersion, + * not level. A per-task mean offset would not work: it cancels in the estimator's + * consecutive-difference, leaving every task with identical jitter. With upstream feedback enabled, + * {@link JitterAwareStreamGrouping} steers more tuples toward the steadiest (lowest-jitter) tasks; run it + * against the {@code loadaware} baseline mode to measure the effect. + * + *

Run the baseline and the jitter-aware run back-to-back to compare. Select the grouping with the + * {@code grouping.mode} flag: + *

+ *   # Baseline: plain LoadAwareShuffleGrouping (no upstream feedback needed).
+ *   storm jar storm-perf.jar org.apache.storm.perf.JitterAwareGroupingTopology 120 \
+ *       -c grouping.mode=loadaware
+ *
+ *   # Jitter-aware: feedback-driven grouping steers tuples to lowest-jitter workers (default mode).
+ *   storm jar storm-perf.jar org.apache.storm.perf.JitterAwareGroupingTopology 120 \
+ *       -c grouping.mode=jitter
+ * 
+ * + *

Tuning knobs (pass with {@code -c key=value}): + *

+ */ +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"; + + static final String GROUPING_MODE = "grouping.mode"; + static final String MODE_JITTER = "jitter"; + static final String MODE_LOADAWARE = "loadaware"; + + 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); + if (inputFile == null || inputFile.trim().isEmpty()) { + throw new IllegalArgumentException( + "Required config '" + INPUT_FILE + "' is not set. Pass a sentence file, e.g. " + + "-c " + INPUT_FILE + "=src/main/sampledata/randomwords.txt"); + } + + 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, selectGrouping(conf)); + builder.setBolt(SINK_ID, new SinkBolt(), sinks) + .localOrShuffleGrouping(WORKER_ID); + + return builder.createTopology(); + } + + /** + * Picks the {@code splitter -> worker} grouping from {@link #GROUPING_MODE}. Defaults to the + * feedback-driven {@link JitterAwareStreamGrouping}; {@code grouping.mode=loadaware} selects the plain + * {@link LoadAwareShuffleGrouping} baseline so the two can be benchmarked back-to-back. Both implement + * {@link org.apache.storm.grouping.LoadAwareCustomStreamGrouping} and require the locality-aware + * configuration set in {@link #main}. + */ + static CustomStreamGrouping selectGrouping(Map conf) { + String mode = Helper.getStr(conf, GROUPING_MODE); + if (MODE_LOADAWARE.equalsIgnoreCase(mode)) { + return new LoadAwareShuffleGrouping(); + } + return new JitterAwareStreamGrouping(); + } + + 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); + // max.spout.pending counts SPOUT tuples (sentences), but each fans out to ~7 word-tuples at the + // SplitterBolt, so the in-flight backlog at the slow workers is ~7x this number. Keep it low enough + // that steady-state complete latency stays well under topology.message.timeout.secs with ~0 fails; + // 4000 oversubscribed the synthetic workers and pinned latency at the timeout. Tune per cluster: + // complete_latency ~= (pending * fanout) / aggregate_worker_ack_rate (Little's law). + topoConf.putIfAbsent(Config.TOPOLOGY_MAX_SPOUT_PENDING, 500); + topoConf.putIfAbsent(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS, 10); + + // Both grouping modes resolve to a LoadAwareShuffleGrouping (directly, or as the jitter grouping's + // fallback), whose prepare() requires these locality-aware keys. Normally supplied by defaults.yaml; + // set defensively so the topology is self-contained. Placed before the CLI merge so -c can override. + topoConf.putIfAbsent(Config.STORM_NETWORK_TOPOGRAPHY_PLUGIN, + "org.apache.storm.networktopography.DefaultRackDNSToSwitchMapping"); + topoConf.putIfAbsent(Config.TOPOLOGY_LOCALITYAWARE_HIGHER_BOUND, 0.8); + topoConf.putIfAbsent(Config.TOPOLOGY_LOCALITYAWARE_LOWER_BOUND, 0.2); + + 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 parks for a constant floor plus task-index-proportional random noise, so each + * task's RFC-1889 execute-jitter (the EWMA of {@code |Δ execute-latency|}) differs by construction. + * + *

Task {@code i} parks for {@code baseDelayUs} µs plus uniform noise in {@code [0, i * baseDelayUs]} + * µs per tuple. Crucially, the constant floor cancels in the consecutive-difference the jitter + * estimator takes, so only the noise width drives jitter — not a per-task mean offset (which + * would cancel and leave every task with identical jitter). Steady-state jitter is {@code width / 3}. + * For a 4-task setup with the default {@code baseDelayUs = 2000}: + *

    + *
  • Task 0: constant ~2 ms — jitter ≈ 0 ms (steadiest)
  • + *
  • Task 1: ~2–4 ms — jitter ≈ 0.67 ms
  • + *
  • Task 2: ~2–6 ms — jitter ≈ 1.33 ms
  • + *
  • Task 3: ~2–8 ms — jitter ≈ 2 ms (jitteriest)
  • + *
+ * Widths are millisecond-scale so the EWMA jitter gauge (millisecond resolution) records distinct + * values per task. {@link JitterAwareStreamGrouping} then steers tuples toward the lowest-jitter + * task (task 0) when feedback is enabled. + * + *

{@link LockSupport#parkNanos} is used instead of a spin loop so jittery tasks yield the CPU + * and do not starve the steady task's executor thread. + */ + private static class JitteryWorkerBolt extends BaseRichBolt { + private final long baseDelayUs; + private OutputCollector collector; + private long baseFloorNs; + private long jitterWidthNs; + 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; + long baseDelayNs = baseDelayUs * 1_000L; + // Constant floor parked by every task: keeps each task doing real work, but cancels in the + // consecutive-difference the RFC-1889 jitter estimator takes, so it adds no jitter. + this.baseFloorNs = baseDelayNs; + // Noise WIDTH grows with task index, so execute-jitter (EWMA of |Δlatency|) genuinely differs + // per task: task 0 is perfectly steady (zero jitter), higher indices are progressively jitterier. + this.jitterWidthNs = 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 = jitterWidthNs == 0L ? 0L : (long) (ThreadLocalRandom.current().nextDouble() * jitterWidthNs); + long sleepNs = baseFloorNs + 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/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java index 8d3fc51ccdd..af8a03c09e8 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.ZeroOneOpenIntervalValidator.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. + * + *

Security: feedback tuples carry a routing control signal (e.g. per-task EWMA jitter + * used by grouping decisions), so a peer that can inject messages on the worker transport could + * forge feedback and deterministically steer a topology's traffic to a chosen task. This stays + * within Storm's existing worker-transport trust model, but because the default + * {@code storm.messaging.netty.authentication} is {@code false}, enable Netty authentication + * (and TLS where available) when running this feature in an untrusted network.

+ */ + @IsBoolean + public static final String TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE = "topology.upstream.feedback.enable"; + /** + * The period, in seconds, between upstream feedback messages within the topology. + * + *

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. Unlike a probabilistic + * trigger, the period yields a deterministic, data-volume-independent feedback cadence.

+ * + *

Validation: Must be a positive integer (seconds).

+ * + *

Impact: + *

    + *
  • Lower values provide more precise, real-time performance data but increase + * network overhead and CPU usage on the control plane.
  • + *
  • Higher values minimize the "observer effect" on the topology's throughput + * while still providing periodic statistical snapshots of health.
  • + *
+ *

+ * + * Defaults to 10 if not explicitly configured. + */ + @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. */ @@ -1901,7 +1939,6 @@ 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. */ diff --git a/storm-client/src/jvm/org/apache/storm/Constants.java b/storm-client/src/jvm/org/apache/storm/Constants.java index c3cd0808709..683b4240293 100644 --- a/storm-client/src/jvm/org/apache/storm/Constants.java +++ b/storm-client/src/jvm/org/apache/storm/Constants.java @@ -28,6 +28,8 @@ 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_STREAM_ID = "__feedback"; + 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 b3cfd90d4d6..0776e1eea47 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,20 @@ public static void addEventLogger(Map conf, StormTopology topolo topology.put_to_bolts(EVENTLOGGER_COMPONENT_ID, eventLoggerBolt); } + public static void addUpstreamFeedback(Map conf, StormTopology topology) { + // 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]. + for (Object component : allComponents(topology).values()) { + ComponentCommon common = getComponentCommon(component); + common.put_to_streams(Constants.FEEDBACK_STREAM_ID, 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<>(); @@ -429,6 +443,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"))); + 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); @@ -464,6 +481,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 +559,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 values, ExecutorTransfer } } + /** + * Sends an unanchored feedback tuple directly to a specific task ID (typically upstream). + *

+ * This method bypasses standard stream grouping logic and routes the tuple + * exclusively to the provided {@code targetTaskId}. It is a non-blocking call: + * if the destination buffer is full, the tuple is added to the {@code pendingEmits} + * queue for later retry, preventing executor stalls. + *

+ * + * @param stream The ID of the stream to emit on (must be declared in the topology). + * @param values The data payload to be sent. + * @param targetTaskId The unique ID of the destination task (e.g., the sourceTaskId of an incoming tuple). + * @param transfer The {@link ExecutorTransfer} instance handling the physical data transfer. + * @param pendingEmits A queue used to store tuples that cannot be transferred immediately due to backpressure. + */ + public void sendUnanchoredFeedback(String stream, List values, int targetTaskId, ExecutorTransfer transfer, Queue pendingEmits) { + Tuple tuple = getTuple(stream, values); + AddressedTuple addressedTuple = new AddressedTuple(targetTaskId, tuple); + transfer.tryTransfer(addressedTuple, pendingEmits); + } + /** * Send sampled data to the eventlogger if the global or component level debug flag is set (via nimbus api). */ 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..b99ce78c236 --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/executor/EwmaFeedbackRecord.java @@ -0,0 +1,82 @@ +/* + * 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) { + + // Sentinel for an absent metric. Jitter values are always >= 0, so a negative value can never + // collide with a real measurement and unambiguously marks "gauge missing / not a Number". + 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; + } + + private static double aggregate(Map gauges, String metricName) { + String suffixedPrefix = metricName + "-"; + double agg = VOID; + for (Map.Entry entry : gauges.entrySet()) { + String name = entry.getKey(); + if (!name.equals(metricName) && !name.startsWith(suffixedPrefix)) { + continue; + } + double value = fromGauge(entry.getValue()); + if (value != VOID && (agg == VOID || value > agg)) { + agg = value; + } + } + return agg; + } + + public static EwmaFeedbackRecord fromWorkerState(WorkerState workerData, int taskId) { + Map allGauges = workerData.getMetricRegistry().getTaskGauges(taskId); + return new EwmaFeedbackRecord(aggregate(allGauges, TaskMetrics.METRIC_NAME_PROCESS_JITTER), + aggregate(allGauges, TaskMetrics.METRIC_NAME_COMPLETE_JITTER), + aggregate(allGauges, 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 edd77743fcd..59f0894fbe2 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 @@ -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.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -58,6 +60,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; @@ -131,7 +134,11 @@ public abstract class Executor implements Callable, JCQueue.Consumer { private final RateCounter reportedErrorCount; private final boolean enableV2MetricsDataPoints; private final Integer v2MetricsTickInterval; - + 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; protected Executor(WorkerState workerData, List executorId, Map credentials, String type) { this.workerData = workerData; this.executorId = executorId; @@ -177,6 +184,7 @@ protected Executor(WorkerState workerData, List executorId, Map executorId, Map g.registerEwmaStats(childEwmaStats)); + } else { + this.upstreamFeedbackFreqSecs = 0; + this.upstreamTaskIds = Collections.emptyList(); + } } public static Executor mkExecutor(WorkerState workerState, List executorId, Map credentials) { @@ -364,6 +382,74 @@ public void metricsTick(Task task, TupleImpl tuple) { } } + /** + * Constructs a Storm {@link Values} object carrying a snapshot of this task's jitter metrics + * to be sent as upstream feedback. + * + *

This method generates a {@link IMetricsConsumer.TaskInfo} header with a timestamp + * and a default interval of -1 (indicating an on-demand, non-periodic-metrics tick), + * followed by the {@link EwmaFeedbackRecord} snapshot. The snapshot is emitted on every tick: + * the tick frequency ({@code topology.upstream.feedback.freq.secs}) is the rate limit, and an + * unconditional resend keeps a restarted/reassigned upstream task from being stranded with stale + * or empty stats when the metric value happens to be stable.

+ * + * @param taskId The ID of the task for which metrics are being collected. + * @return A {@link Values} object containing {@code [TaskInfo, EwmaFeedbackRecord]} (matching the + * feedback stream schema declared by {@link StormCommon#upstreamFeedbackFields()}). + */ + public Values buildUpstreamFeedbackTuple(int taskId) { + EwmaFeedbackRecord statsRecord = EwmaFeedbackRecord.fromWorkerState(this.workerData, taskId); + IMetricsConsumer.TaskInfo taskInfo = new IMetricsConsumer.TaskInfo( + hostname, workerTopologyContext.getThisWorkerPort(), + componentId, taskId, Time.currentTimeSecs(), -1); + return new Values(taskInfo, statsRecord); + } + + /** + * Updates child task statistics by unwrapping the Storm Values object. + * + *

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: {@link EwmaFeedbackRecord}
  • + *
+ *

+ * + * @param task The {@link Task} associated with this update. + * @param tuple The {@link TupleImpl} emitted by the upstream feedback builder. + */ + 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; + } + + // 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; + } + + 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; + } + + childEwmaStats.update(task.getTaskId(), taskInfo.srcTaskId, feedback); + } + // updates v1 metric dataPoints with v2 metric API data private void addV2Metrics(int taskId, List dataPoints, int interval) { if (!enableV2MetricsDataPoints) { @@ -503,6 +589,62 @@ 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)}. + */ + public void sendUpstreamFeedback(Task task) { + if (!upstreamFeedbackEnabled) { + return; + } + Values feedbackTuple = buildUpstreamFeedbackTuple(task.getTaskId()); + for (int parentTask : upstreamTaskIds) { + task.sendUnanchoredFeedback(Constants.FEEDBACK_STREAM_ID, 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 273bab5e69f..47a4940e72f 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 && !Utils.isSystemId(componentId)) { + scheduleUpstreamFeedbackTick(upstreamFeedbackFreqSecs); + } } @Override @@ -198,6 +201,18 @@ 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 (Constants.FEEDBACK_STREAM_ID.equals(streamId)) { + if (!this.upstreamFeedbackEnabled) { + LOG.debug("Upstream feedback skipped."); + } else { + // update internal metrics + this.updateChildEwmaStats(idToTask.get(taskId - idToTaskBase), tuple); + } } else { IBolt boltObject = (IBolt) idToTask.get(taskId - idToTaskBase).getTaskObject(); boolean isSampled = sampler.getAsBoolean(); 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 734fca2a2d6..2eab9b08816 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 @@ -95,7 +95,7 @@ public SpoutExecutorStats getStats() { } public void init(final ArrayList idToTask, int idToTaskBase) throws InterruptedException { - this.threadId = Thread.currentThread().getId(); + this.threadId = Thread.currentThread().threadId(); executorTransfer.initLocalRecvQueues(); workerReady.await(); while (!stormActive.get()) { @@ -321,6 +321,13 @@ public void tupleActionFn(int taskId, TupleImpl tuple) throws Exception { if (pendingForId != null) { pending.put(id, pendingForId); } + } else if (Constants.FEEDBACK_STREAM_ID.equals(streamId)) { + if (!this.upstreamFeedbackEnabled) { + LOG.debug("Upstream feedback skipped."); + } else { + // update internal metrics + this.updateChildEwmaStats(idToTask.get(taskId - idToTaskBase), tuple); + } } else { Long id = (Long) tuple.getValue(0); Long timeDeltaMs = (Long) tuple.getValue(1); 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..7f616898904 --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/grouping/JitterAwareStreamGrouping.java @@ -0,0 +1,117 @@ +/* + * 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.Random; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.storm.executor.ChildEwmaStats; +import org.apache.storm.generated.GlobalStreamId; +import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting; +import org.apache.storm.task.WorkerTopologyContext; + +/** + * A {@link LoadAwareCustomStreamGrouping} that steers each tuple toward the downstream (child) task with + * lower jitter, as reported back to the emitting task through upstream feedback and aggregated by + * {@link ChildEwmaStats}. Jitter is compared with {@link ChildEwmaStats#compareByJitter}, so a lower + * {@code __execute-jitter} wins first, then {@code __process-jitter}, then {@code __complete-jitter}. + * + *

Steering uses power-of-two-choices: for each tuple two target tasks are sampled at random and + * the lower-jitter one wins. Random sampling keeps the best task from receiving every tuple (the + * "thundering herd" a plain arg-min selection would cause) while still biasing traffic toward the good + * tasks. + * + *

Whenever jitter cannot pick a winner — the sampled pair ties (equal jitter, or neither has reported), + * no feedback exists yet for the source task, or no {@link ChildEwmaStats} was registered — the decision is + * delegated to an embedded {@link LoadAwareShuffleGrouping}. When all targets carry equal jitter, + * every sampled pair ties, so the grouping behaves as a pure load-aware shuffle. {@link #refreshLoad} is + * forwarded to that delegate, so the fallback path honours real system load and locality. + */ +public class JitterAwareStreamGrouping implements LoadAwareCustomStreamGrouping { + + private final LoadAwareShuffleGrouping fallback = new LoadAwareShuffleGrouping(); + private List targetTasks; + private ChildEwmaStats stats; + + // deterministic test + @VisibleForTesting + Random random; + + @Override + public void refreshLoad(LoadMapping loadMapping) { + fallback.refreshLoad(loadMapping); + } + + @Override + public void registerEwmaStats(ChildEwmaStats childEwmaStats) { + this.stats = childEwmaStats; + } + + @Override + public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) { + this.targetTasks = targetTasks; + // The fallback rejects an empty target list; chooseTasks short-circuits that case before delegating. + if (targetTasks != null && !targetTasks.isEmpty()) { + fallback.prepare(context, stream, 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 fallback.chooseTasks(taskId, values); + } + + Map> childStats = stats.getStats(taskId); + if (childStats.isEmpty()) { + return fallback.chooseTasks(taskId, values); + } + + // Power-of-two-choices: sample two distinct targets and keep the lower-jitter one. + int n = targetTasks.size(); + int i = nextInt(n); + int j = nextInt(n - 1); + if (j >= i) { + j++; + } + Integer a = targetTasks.get(i); + Integer b = targetTasks.get(j); + + // An unreported target is an empty map, which compareByJitter treats as worst. + Map metricsA = childStats.getOrDefault(a, Collections.emptyMap()); + Map metricsB = childStats.getOrDefault(b, Collections.emptyMap()); + int cmp = ChildEwmaStats.compareByJitter(metricsA, metricsB); + if (cmp < 0) { + return Collections.singletonList(a); + } + if (cmp > 0) { + return Collections.singletonList(b); + } + // Tie (equal jitter, or both unreported): no jitter winner -> defer to the load-aware fallback. + return fallback.chooseTasks(taskId, values); + } + + private int nextInt(int bound) { + return random != null ? random.nextInt(bound) : ThreadLocalRandom.current().nextInt(bound); + } + +} 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..7edeec83f33 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,12 @@ package org.apache.storm.grouping; +import org.apache.storm.executor.ChildEwmaStats; + public interface LoadAwareCustomStreamGrouping extends CustomStreamGrouping { void refreshLoad(LoadMapping loadMapping); + + default void registerEwmaStats(ChildEwmaStats childEwmaStats) { + // no-op: backward compatibility + } } 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 78cd6d3b966..059109b0bd8 100644 --- a/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java +++ b/storm-client/src/jvm/org/apache/storm/metrics2/TaskMetrics.java @@ -29,11 +29,11 @@ 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"; private final ConcurrentMap rateCounters = new ConcurrentHashMap<>(); 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..40430a572bc 100644 --- a/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java +++ b/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java @@ -77,6 +77,7 @@ public static Kryo getKryo(Map conf) { k.register(ConsList.class); k.register(BackPressureStatus.class); k.register(NodeInfo.class); + k.register(org.apache.storm.executor.EwmaFeedbackRecord.class); synchronized (loader) { for (SerializationRegister sr : loader) { 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 d0fc9691dbd..663a7915437 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java +++ b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java @@ -198,6 +198,24 @@ public static boolean ewmaEnable(Map conf) { return ObjectReader.getBoolean(value, false); } + public static boolean upstreamFeedbackEnable(Map conf) { + Object value = conf.get(Config.TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE); + if (value == null) { + return false; + } + return ObjectReader.getBoolean(value, false); + } + + 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_FREQ_SECS + + " in conf: " + freqSecs + " must be > 0"); + } + public static BooleanSupplier mkStatsSampler(Map conf) { return evenSampler(samplingRate(conf)); } diff --git a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java index 0d59fed77de..94979789e8a 100644 --- a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java +++ b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java @@ -870,7 +870,7 @@ public void validateField(String name, Object o) { } } - public static class EwmaSmoothingFactorValidator extends Validator { + public static class ZeroOneOpenIntervalValidator extends Validator { @Override public void validateField(String name, Object o) { if (o == null) { diff --git a/storm-client/test/jvm/org/apache/storm/executor/ChildEwmaStatsTest.java b/storm-client/test/jvm/org/apache/storm/executor/ChildEwmaStatsTest.java new file mode 100644 index 00000000000..2b4be5de9e7 --- /dev/null +++ b/storm-client/test/jvm/org/apache/storm/executor/ChildEwmaStatsTest.java @@ -0,0 +1,123 @@ +/** + * 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.HashMap; +import java.util.Map; +import org.apache.storm.metrics2.TaskMetrics; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Unit tests for {@link ChildEwmaStats}: the per-source-task aggregation of downstream jitter + * reports and the {@link ChildEwmaStats#compareByJitter} ordering used by + * {@link org.apache.storm.grouping.JitterAwareStreamGrouping} to pick the lowest-jitter child. + */ +public class ChildEwmaStatsTest { + + private static final int PARENT = 10; + private static final int CHILD_A = 20; + private static final int CHILD_B = 21; + + private Map metrics(double execute, double process, double complete) { + Map m = new HashMap<>(); + m.put(TaskMetrics.METRIC_NAME_EXECUTE_JITTER, execute); + m.put(TaskMetrics.METRIC_NAME_PROCESS_JITTER, process); + m.put(TaskMetrics.METRIC_NAME_COMPLETE_JITTER, complete); + return m; + } + + @Test + public void disabled_updateIsNoOpAndStatsEmpty() { + ChildEwmaStats stats = new ChildEwmaStats(false); + stats.update(PARENT, CHILD_A, new EwmaFeedbackRecord(1, 2, 3)); + assertTrue(stats.getStats(PARENT).isEmpty()); + } + + @Test + public void update_storesPerChildMetricsForParent() { + ChildEwmaStats stats = new ChildEwmaStats(true); + stats.update(PARENT, CHILD_A, new EwmaFeedbackRecord(1.0, 2.0, 3.0)); + + Map> byChild = stats.getStats(PARENT); + assertEquals(1, byChild.size()); + Map child = byChild.get(CHILD_A); + assertEquals(3.0, child.get(TaskMetrics.METRIC_NAME_EXECUTE_JITTER)); + assertEquals(1.0, child.get(TaskMetrics.METRIC_NAME_PROCESS_JITTER)); + assertEquals(2.0, child.get(TaskMetrics.METRIC_NAME_COMPLETE_JITTER)); + } + + @Test + public void update_isIsolatedPerParentTask() { + ChildEwmaStats stats = new ChildEwmaStats(true); + stats.update(PARENT, CHILD_A, new EwmaFeedbackRecord(1, 2, 3)); + assertTrue(stats.getStats(999).isEmpty(), "a parent with no reports sees no stats"); + } + + @Test + public void update_latestReportOverwritesPrevious() { + ChildEwmaStats stats = new ChildEwmaStats(true); + stats.update(PARENT, CHILD_A, new EwmaFeedbackRecord(1, 1, 1)); + stats.update(PARENT, CHILD_A, new EwmaFeedbackRecord(9, 9, 9)); + assertEquals(9.0, stats.getStats(PARENT).get(CHILD_A).get(TaskMetrics.METRIC_NAME_EXECUTE_JITTER)); + } + + @Test + public void update_absentMetricNotStored() { + // VOID-valued components are skipped by EwmaFeedbackRecord#forEachMetric, so they never reach the map. + ChildEwmaStats stats = new ChildEwmaStats(true); + stats.update(PARENT, CHILD_A, new EwmaFeedbackRecord(-1, -1, 5.0)); + Map child = stats.getStats(PARENT).get(CHILD_A); + assertEquals(1, child.size()); + assertEquals(5.0, child.get(TaskMetrics.METRIC_NAME_EXECUTE_JITTER)); + } + + @Test + public void compareByJitter_executeJitterWinsFirst() { + // Lower execute-jitter is preferred regardless of the other two metrics. + Map low = metrics(1.0, 9.0, 9.0); + Map high = metrics(2.0, 0.0, 0.0); + assertTrue(ChildEwmaStats.compareByJitter(low, high) < 0); + assertTrue(ChildEwmaStats.compareByJitter(high, low) > 0); + } + + @Test + public void compareByJitter_fallsThroughToProcessThenComplete() { + Map a = metrics(1.0, 1.0, 5.0); + Map b = metrics(1.0, 2.0, 0.0); + // tie on execute -> process decides (a wins) + assertTrue(ChildEwmaStats.compareByJitter(a, b) < 0); + + Map c = metrics(1.0, 1.0, 3.0); + Map d = metrics(1.0, 1.0, 4.0); + // tie on execute and process -> complete decides (c wins) + assertTrue(ChildEwmaStats.compareByJitter(c, d) < 0); + } + + @Test + public void compareByJitter_equalMetricsAreEqual() { + assertEquals(0, ChildEwmaStats.compareByJitter(metrics(1, 2, 3), metrics(1, 2, 3))); + } + + @Test + public void compareByJitter_missingMetricTreatedAsWorst() { + // An empty map (a child that has not reported) must lose to any measured value. + Map measured = metrics(5.0, 5.0, 5.0); + Map empty = new HashMap<>(); + assertTrue(ChildEwmaStats.compareByJitter(measured, empty) < 0); + assertTrue(ChildEwmaStats.compareByJitter(empty, measured) > 0); + assertEquals(0, ChildEwmaStats.compareByJitter(empty, empty)); + } +} diff --git a/storm-client/test/jvm/org/apache/storm/executor/EwmaFeedbackRecordTest.java b/storm-client/test/jvm/org/apache/storm/executor/EwmaFeedbackRecordTest.java new file mode 100644 index 00000000000..38913d9007b --- /dev/null +++ b/storm-client/test/jvm/org/apache/storm/executor/EwmaFeedbackRecordTest.java @@ -0,0 +1,228 @@ +/** + * 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.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.storm.Config; +import org.apache.storm.daemon.worker.WorkerState; +import org.apache.storm.metric.api.IMetricsConsumer; +import org.apache.storm.metrics2.StormMetricRegistry; +import org.apache.storm.metrics2.TaskMetrics; +import org.apache.storm.serialization.KryoValuesDeserializer; +import org.apache.storm.serialization.KryoValuesSerializer; +import org.apache.storm.task.WorkerTopologyContext; +import org.apache.storm.tuple.Values; +import org.apache.storm.utils.Utils; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests for {@link EwmaFeedbackRecord}, covering three concerns: + *
    + *
  • the {@link EwmaFeedbackRecord#forEachMetric} contract, including the VOID sentinel that + * suppresses absent metrics;
  • + *
  • Kryo wire round-trip — feedback tuples are emitted to upstream tasks that may live in a + * different worker JVM, so the record is serialized across the network exactly as registered in + * {@link org.apache.storm.serialization.SerializationFactory};
  • + *
  • {@link EwmaFeedbackRecord#fromWorkerState} against the real {@link TaskMetrics} / + * {@link StormMetricRegistry} registration path. {@code TaskMetrics} registers jitter gauges under + * suffixed names (one per {@code sourceComponent:sourceStream}), e.g. + * {@code __execute-jitter-splitter:default}, so a bare {@code get("__execute-jitter")} never matches. + * These tests assert the signal is actually discovered and aggregated — without it + * {@link org.apache.storm.grouping.JitterAwareStreamGrouping} silently degrades to round-robin.
  • + *
+ */ +public class EwmaFeedbackRecordTest { + + private static final int TASK_ID = 7; + private static final double VOID = -1.0; + + private Map conf() { + return Utils.readStormConfig(); + } + + private Map ewmaConf(boolean enabled) { + Map conf = Utils.readStormConfig(); + conf.put(Config.TOPOLOGY_STATS_EWMA_ENABLE, enabled); + return conf; + } + + private TaskMetrics newTaskMetrics(StormMetricRegistry registry, Map conf) { + WorkerTopologyContext context = mock(WorkerTopologyContext.class); + when(context.getStormId()).thenReturn("test-topo"); + when(context.getThisWorkerPort()).thenReturn(6700); + return new TaskMetrics(context, "worker", TASK_ID, registry, conf); + } + + private WorkerState workerStateFor(StormMetricRegistry registry) { + WorkerState workerState = mock(WorkerState.class); + when(workerState.getMetricRegistry()).thenReturn(registry); + return workerState; + } + + // ---- forEachMetric ---- + + @Test + public void forEachMetric_emitsAllPresentMetricsWithCorrectNames() { + EwmaFeedbackRecord record = new EwmaFeedbackRecord(1.0, 2.0, 3.0); + + Map collected = new LinkedHashMap<>(); + record.forEachMetric(collected::put); + + assertEquals(3, collected.size()); + assertEquals(1.0, collected.get(TaskMetrics.METRIC_NAME_PROCESS_JITTER)); + assertEquals(2.0, collected.get(TaskMetrics.METRIC_NAME_COMPLETE_JITTER)); + assertEquals(3.0, collected.get(TaskMetrics.METRIC_NAME_EXECUTE_JITTER)); + } + + @Test + public void forEachMetric_skipsAbsentMetrics() { + // -1 is the VOID sentinel: an absent gauge must not be reported as a real (negative) measurement. + EwmaFeedbackRecord record = new EwmaFeedbackRecord(-1, 2.0, -1); + + Map collected = new LinkedHashMap<>(); + record.forEachMetric(collected::put); + + assertEquals(1, collected.size()); + assertEquals(2.0, collected.get(TaskMetrics.METRIC_NAME_COMPLETE_JITTER)); + } + + @Test + public void forEachMetric_emitsZeroJitter() { + // Zero is a legitimate value (jitter decays to 0 under stable latency) and must be distinct from VOID. + EwmaFeedbackRecord record = new EwmaFeedbackRecord(0.0, 0.0, 0.0); + + Map collected = new LinkedHashMap<>(); + record.forEachMetric(collected::put); + + assertEquals(3, collected.size()); + assertEquals(0.0, collected.get(TaskMetrics.METRIC_NAME_EXECUTE_JITTER)); + } + + // ---- Kryo round-trip ---- + + @Test + public void kryoRoundTrip_preservesRecord() { + Map conf = conf(); + KryoValuesSerializer serializer = new KryoValuesSerializer(conf); + KryoValuesDeserializer deserializer = new KryoValuesDeserializer(conf); + + EwmaFeedbackRecord original = new EwmaFeedbackRecord(1.5, 2.5, 3.5); + Object restored = deserializer.deserializeObject(serializer.serializeObject(original)); + + assertInstanceOf(EwmaFeedbackRecord.class, restored); + assertEquals(original, restored); + } + + @Test + public void kryoRoundTrip_insideFeedbackTuple() { + // Mirrors Executor#buildUpstreamFeedbackTuple: [TaskInfo, EwmaFeedbackRecord]. Both elements + // must survive the wire so the receiving task can rebuild its child stats. + Map conf = conf(); + KryoValuesSerializer serializer = new KryoValuesSerializer(conf); + KryoValuesDeserializer deserializer = new KryoValuesDeserializer(conf); + + IMetricsConsumer.TaskInfo taskInfo = + new IMetricsConsumer.TaskInfo("host", 6700, "comp", 7, 123456, -1); + EwmaFeedbackRecord feedback = new EwmaFeedbackRecord(-1, 9.0, 4.0); + Values original = new Values(taskInfo, feedback); + + List restored = deserializer.deserialize(serializer.serialize(original)); + + assertEquals(2, restored.size()); + assertInstanceOf(IMetricsConsumer.TaskInfo.class, restored.get(0)); + assertInstanceOf(EwmaFeedbackRecord.class, restored.get(1)); + assertEquals(7, ((IMetricsConsumer.TaskInfo) restored.get(0)).srcTaskId); + assertEquals(feedback, restored.get(1)); + assertTrue(((EwmaFeedbackRecord) restored.get(1)).processJitter() < 0, "VOID sentinel preserved"); + } + + @Test + public void fromWorkerState_discoversSuffixedExecuteJitterGauge() { + StormMetricRegistry registry = new StormMetricRegistry(); + TaskMetrics taskMetrics = newTaskMetrics(registry, ewmaConf(true)); + + // Real path: registers "__execute-jitter-splitter:default" and feeds the RFC-1889 EWMA estimator. + // Varying latencies make the jitter estimate strictly positive. + for (long latency : new long[]{10, 60, 15, 90, 20}) { + taskMetrics.boltExecuteTuple("splitter", "default", latency); + } + + EwmaFeedbackRecord record = EwmaFeedbackRecord.fromWorkerState(workerStateFor(registry), TASK_ID); + + // Regression guard: a bare get("__execute-jitter") misses the suffixed key and leaves this VOID (-1). + assertTrue(record.executeJitter() > 0, + "execute-jitter must be discovered from the suffixed gauge, got " + record.executeJitter()); + + // Metrics that were never driven must stay VOID — proving the prefix filter does not cross-match + // (e.g. __execute-jitter must not be picked up when asking for __process-jitter). + assertEquals(VOID, record.processJitter()); + assertEquals(VOID, record.completeJitter()); + + // forEachMetric must surface the value under the canonical bare name consumed by ChildEwmaStats. + Map emitted = new HashMap<>(); + record.forEachMetric(emitted::put); + assertEquals(1, emitted.size()); + assertEquals(record.executeJitter(), emitted.get(TaskMetrics.METRIC_NAME_EXECUTE_JITTER)); + } + + @Test + public void fromWorkerState_aggregatesMaxAcrossMultipleSources() { + StormMetricRegistry registry = new StormMetricRegistry(); + TaskMetrics taskMetrics = newTaskMetrics(registry, ewmaConf(true)); + + // Same task consuming from two upstream sources => two distinct __execute-jitter- gauges. + for (long latency : new long[]{10, 20, 12, 18}) { + taskMetrics.boltExecuteTuple("srcA", "default", latency); + } + for (long latency : new long[]{10, 200, 5, 300}) { // far jumpier => higher jitter + taskMetrics.boltExecuteTuple("srcB", "default", latency); + } + + // Compute the expected max directly from the registry, independent of EWMA arithmetic. + Map gauges = registry.getTaskGauges(TASK_ID); + double expectedMax = gauges.entrySet().stream() + .filter(e -> e.getKey().startsWith(TaskMetrics.METRIC_NAME_EXECUTE_JITTER + "-")) + .mapToDouble(e -> ((Number) e.getValue().getValue()).doubleValue()) + .max() + .orElseThrow(() -> new AssertionError("no execute-jitter gauges registered")); + assertTrue(expectedMax > 0, "precondition: at least one source produced positive jitter"); + + EwmaFeedbackRecord record = EwmaFeedbackRecord.fromWorkerState(workerStateFor(registry), TASK_ID); + assertEquals(expectedMax, record.executeJitter(), + "fromWorkerState must report the max jitter across all of the task's source gauges"); + } + + @Test + public void fromWorkerState_allVoidWhenNoJitterGaugesRegistered() { + // EWMA disabled => boltExecuteTuple registers latency gauges but no jitter gauges at all. + StormMetricRegistry registry = new StormMetricRegistry(); + TaskMetrics taskMetrics = newTaskMetrics(registry, ewmaConf(false)); + + taskMetrics.boltExecuteTuple("splitter", "default", 42); + + EwmaFeedbackRecord record = EwmaFeedbackRecord.fromWorkerState(workerStateFor(registry), TASK_ID); + assertEquals(VOID, record.executeJitter()); + assertEquals(VOID, record.processJitter()); + assertEquals(VOID, record.completeJitter()); + } +} diff --git a/storm-client/test/jvm/org/apache/storm/grouping/JitterAwareStreamGroupingTest.java b/storm-client/test/jvm/org/apache/storm/grouping/JitterAwareStreamGroupingTest.java new file mode 100644 index 00000000000..558ef0546f7 --- /dev/null +++ b/storm-client/test/jvm/org/apache/storm/grouping/JitterAwareStreamGroupingTest.java @@ -0,0 +1,199 @@ +/** + * 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.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.storm.Config; +import org.apache.storm.executor.ChildEwmaStats; +import org.apache.storm.executor.EwmaFeedbackRecord; +import org.apache.storm.generated.NodeInfo; +import org.apache.storm.shade.com.google.common.collect.Sets; +import org.apache.storm.task.WorkerTopologyContext; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link JitterAwareStreamGrouping}: power-of-two-choices steering toward the lower-jitter + * child once {@link ChildEwmaStats} is populated, and delegation to the embedded + * {@link LoadAwareShuffleGrouping} whenever jitter cannot pick a winner (no feedback, or a tied pair). + * + *

With exactly two targets, P2C deterministically samples both, so the lower-jitter target always wins + * those tests without needing to control the random source. + */ +public class JitterAwareStreamGroupingTest { + + private static final int SOURCE_TASK = 1; + private static final List VALUES = Collections.singletonList("v"); + + private Map createConf() { + Map conf = new HashMap<>(); + conf.put(Config.STORM_NETWORK_TOPOGRAPHY_PLUGIN, "org.apache.storm.networktopography.DefaultRackDNSToSwitchMapping"); + conf.put(Config.TOPOLOGY_LOCALITYAWARE_HIGHER_BOUND, 0.8); + conf.put(Config.TOPOLOGY_LOCALITYAWARE_LOWER_BOUND, 0.2); + return conf; + } + + /** Minimal context that satisfies the embedded {@link LoadAwareShuffleGrouping#prepare}. */ + private WorkerTopologyContext mockContext(List availableTaskIds) { + WorkerTopologyContext context = mock(WorkerTopologyContext.class); + when(context.getConf()).thenReturn(createConf()); + Map taskNodeToPort = new HashMap<>(); + NodeInfo nodeInfo = new NodeInfo("node-id", Sets.newHashSet(6700L)); + availableTaskIds.forEach(e -> taskNodeToPort.put(e, nodeInfo)); + when(context.getTaskToNodePort()).thenReturn(new AtomicReference<>(taskNodeToPort)); + when(context.getAssignmentId()).thenReturn("node-id"); + when(context.getThisWorkerPort()).thenReturn(6700); + when(context.getNodeToHost()).thenReturn(new AtomicReference<>(Collections.singletonMap("node-id", "hostname1"))); + return context; + } + + private JitterAwareStreamGrouping prepared(List targets, ChildEwmaStats stats) { + JitterAwareStreamGrouping grouping = new JitterAwareStreamGrouping(); + if (stats != null) { + grouping.registerEwmaStats(stats); + } + grouping.prepare(mockContext(targets), null, targets); + return grouping; + } + + @Test + public void chooseTasks_emptyTargetsReturnsEmpty() { + JitterAwareStreamGrouping grouping = prepared(Collections.emptyList(), null); + assertTrue(grouping.chooseTasks(SOURCE_TASK, VALUES).isEmpty()); + } + + @Test + public void chooseTasks_singleTargetAlwaysReturnsIt() { + JitterAwareStreamGrouping grouping = prepared(Collections.singletonList(42), new ChildEwmaStats(true)); + for (int i = 0; i < 5; i++) { + assertEquals(Collections.singletonList(42), grouping.chooseTasks(SOURCE_TASK, VALUES)); + } + } + + @Test + public void chooseTasks_noStatsRegistered_delegatesToFallback() { + List targets = Arrays.asList(10, 11, 12); + JitterAwareStreamGrouping grouping = prepared(targets, null); + assertDelegatedSpread(grouping, targets); + } + + @Test + public void chooseTasks_statsEmptyForSource_delegatesToFallback() { + List targets = Arrays.asList(10, 11, 12); + JitterAwareStreamGrouping grouping = prepared(targets, new ChildEwmaStats(true)); + assertDelegatedSpread(grouping, targets); + } + + @Test + public void chooseTasks_steersToLowerJitterChild() { + // Two targets => P2C samples both; the lower execute-jitter (11) always wins. + List targets = Arrays.asList(10, 11); + ChildEwmaStats stats = new ChildEwmaStats(true); + stats.update(SOURCE_TASK, 10, new EwmaFeedbackRecord(5, 5, 8.0)); + stats.update(SOURCE_TASK, 11, new EwmaFeedbackRecord(5, 5, 2.0)); + JitterAwareStreamGrouping grouping = prepared(targets, stats); + + for (int i = 0; i < 5; i++) { + assertEquals(Collections.singletonList(11), grouping.chooseTasks(SOURCE_TASK, VALUES)); + } + } + + @Test + public void chooseTasks_prefersReportedOverUnreported() { + // Only 11 has reported; with two targets it is always in the pair and beats the unreported 10. + List targets = Arrays.asList(10, 11); + ChildEwmaStats stats = new ChildEwmaStats(true); + stats.update(SOURCE_TASK, 11, new EwmaFeedbackRecord(7.0, 7.0, 7.0)); + JitterAwareStreamGrouping grouping = prepared(targets, stats); + + for (int i = 0; i < 5; i++) { + assertEquals(Collections.singletonList(11), grouping.chooseTasks(SOURCE_TASK, VALUES)); + } + } + + @Test + public void chooseTasks_tiedJitter_delegatesToFallback() { + // Equal jitter on both targets => every pair ties => the load-aware fallback decides and spreads. + List targets = Arrays.asList(10, 11); + ChildEwmaStats stats = new ChildEwmaStats(true); + stats.update(SOURCE_TASK, 10, new EwmaFeedbackRecord(5.0, 5.0, 5.0)); + stats.update(SOURCE_TASK, 11, new EwmaFeedbackRecord(5.0, 5.0, 5.0)); + JitterAwareStreamGrouping grouping = prepared(targets, stats); + assertDelegatedSpread(grouping, targets); + } + + @Test + public void chooseTasks_isPerSourceTask() { + // Two targets, source 1 only: child 10 is best; the grouping must not consult another source's stats. + List targets = Arrays.asList(10, 11); + ChildEwmaStats stats = new ChildEwmaStats(true); + stats.update(1, 10, new EwmaFeedbackRecord(1, 1, 1.0)); + stats.update(1, 11, new EwmaFeedbackRecord(1, 1, 9.0)); + JitterAwareStreamGrouping grouping = prepared(targets, stats); + + for (int i = 0; i < 5; i++) { + assertEquals(Collections.singletonList(10), grouping.chooseTasks(1, VALUES)); + } + } + + @Test + public void chooseTasks_p2cAvoidsHerd() { + // Three strictly-ordered targets, real randomness: the best (10) takes the plurality but NOT 100% + // (herd avoided), the worst (12) never wins a pair, and 11 takes the remainder. + List targets = Arrays.asList(10, 11, 12); + ChildEwmaStats stats = new ChildEwmaStats(true); + stats.update(SOURCE_TASK, 10, new EwmaFeedbackRecord(5, 5, 2.0)); // best + stats.update(SOURCE_TASK, 11, new EwmaFeedbackRecord(5, 5, 5.0)); + stats.update(SOURCE_TASK, 12, new EwmaFeedbackRecord(5, 5, 8.0)); // worst + JitterAwareStreamGrouping grouping = prepared(targets, stats); + + Map hits = new HashMap<>(); + int total = 3000; + for (int i = 0; i < total; i++) { + List chosen = grouping.chooseTasks(SOURCE_TASK, VALUES); + assertEquals(1, chosen.size()); + hits.merge(chosen.get(0), 1, Integer::sum); + } + int best = hits.getOrDefault(10, 0); + int mid = hits.getOrDefault(11, 0); + int worst = hits.getOrDefault(12, 0); + assertTrue(best > 0 && best < total, "best target should take a share but not the whole herd: " + best); + assertTrue(best > mid, "best target should outweigh the middle one: " + best + " vs " + mid); + assertTrue(mid > 0, "middle target should still receive traffic: " + mid); + // The worst target is never the lower-jitter of any sampled pair, and there are no ties to delegate. + assertEquals(0, worst, "worst target should never win a P2C comparison"); + } + + /** Over a full sweep, every result is a single valid target and every target is used (fallback spread). */ + private void assertDelegatedSpread(JitterAwareStreamGrouping grouping, List targets) { + Map hits = new HashMap<>(); + for (int i = 0; i < targets.size() * 1000; i++) { + List chosen = grouping.chooseTasks(SOURCE_TASK, VALUES); + assertEquals(1, chosen.size()); + assertTrue(targets.contains(chosen.get(0))); + hits.merge(chosen.get(0), 1, Integer::sum); + } + for (Integer target : targets) { + assertTrue(hits.getOrDefault(target, 0) > 0, "target " + target + " should receive some traffic"); + } + } +} diff --git a/storm-client/test/jvm/org/apache/storm/utils/ConfigUtilsTest.java b/storm-client/test/jvm/org/apache/storm/utils/ConfigUtilsTest.java index 7bbf3941ce0..7a5bd14f5f6 100644 --- a/storm-client/test/jvm/org/apache/storm/utils/ConfigUtilsTest.java +++ b/storm-client/test/jvm/org/apache/storm/utils/ConfigUtilsTest.java @@ -23,8 +23,10 @@ import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public class ConfigUtilsTest { @@ -157,4 +159,40 @@ public void getHfdsPrincipal() throws UnknownHostException { conf.put(Config.STORM_HDFS_LOGIN_PRINCIPAL, principal); assertEquals(Config.getHdfsPrincipal(conf), principal); } + + @Test + public void upstreamFeedbackEnable_defaultsFalseWhenAbsent() { + assertFalse(ConfigUtils.upstreamFeedbackEnable(new HashMap<>())); + } + + @Test + public void upstreamFeedbackEnable_readsConfiguredValue() { + assertTrue(ConfigUtils.upstreamFeedbackEnable( + mockMap(Config.TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE, true))); + assertFalse(ConfigUtils.upstreamFeedbackEnable( + mockMap(Config.TOPOLOGY_UPSTREAM_FEEDBACK_ENABLE, false))); + } + + @Test + public void upstreamFeedbackFreqSecs_defaultsToTenWhenAbsent() { + assertEquals(10, ConfigUtils.upstreamFeedbackFreqSecs(new HashMap<>())); + } + + @Test + public void upstreamFeedbackFreqSecs_returnsConfiguredPositiveValue() { + assertEquals(5, ConfigUtils.upstreamFeedbackFreqSecs( + mockMap(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS, 5))); + } + + @Test + public void upstreamFeedbackFreqSecs_rejectsZero() { + assertThrows(IllegalArgumentException.class, () -> ConfigUtils.upstreamFeedbackFreqSecs( + mockMap(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS, 0))); + } + + @Test + public void upstreamFeedbackFreqSecs_rejectsNegative() { + assertThrows(IllegalArgumentException.class, () -> ConfigUtils.upstreamFeedbackFreqSecs( + mockMap(Config.TOPOLOGY_UPSTREAM_FEEDBACK_FREQ_SECS, -1))); + } }