You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2017/09/07 19:14:42 UTC

[01/18] storm git commit: STORM-2702: Part 1. Move files as needed

Repository: storm
Updated Branches:
  refs/heads/master 3ee9a899c -> 32389d75f


STORM-2702: Part 1.  Move files as needed


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/0d10b8af
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/0d10b8af
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/0d10b8af

Branch: refs/heads/master
Commit: 0d10b8afe7e282d04b67f1a0c1c90db801842b14
Parents: b87dcc0
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Mon Aug 21 13:29:59 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Aug 21 13:29:59 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                | 163 ++++++++
 examples/storm-loadgen/pom.xml                  | 122 ++++++
 .../loadgen/HttpForwardingMetricsConsumer.java  |  85 +++++
 .../loadgen/HttpForwardingMetricsServer.java    | 118 ++++++
 .../storm/loadgen/ThroughputVsLatency.java      | 377 +++++++++++++++++++
 examples/storm-starter/pom.xml                  |   5 -
 .../storm/starter/ThroughputVsLatency.java      | 377 -------------------
 pom.xml                                         |   2 +-
 storm-client-misc/pom.xml                       |  63 ----
 .../metric/HttpForwardingMetricsConsumer.java   |  85 -----
 .../metric/HttpForwardingMetricsServer.java     | 118 ------
 11 files changed, 866 insertions(+), 649 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
new file mode 100644
index 0000000..8414768
--- /dev/null
+++ b/examples/storm-loadgen/README.md
@@ -0,0 +1,163 @@
+# Storm Load Generation Tools
+
+A set of tools to place an artificial load on a storm cluster to compare against a different storm cluster.  This is particularly helpful when making changes to the data path in storm to see what if any impact the changes had.  This is also useful for end users that want to compare different hardware setups to see what the trade-offs are, although actually running your real topologies is going to be more accurate.
+
+## Methodology
+The idea behind all of these tools is to measure the trade-offs between latency, throughput, and cost when processing data using Apache Storm.
+
+When processing data you typically will know a few things.  First you will know about how much data you are going to be processing.  This will typically be a range of values that change throughput the day.  You also will have an idea of how quickly you need the data processed by.  Often this is measured in terms of the latency it takes to process data at the some percentile or set of percentiles.  This is because of most use cases the value of the data declines over time, and being able to react to the data quickly is more valuable.  You probably also have a budget for how much you are willing to spend to be able to process this data.  There are always trade-offs in how quickly you can process some data and how efficiently you can processes that data both in terms of resource usage (cost) and latency.  These tools are designed to help you explore that space.
+
+## Tools
+### CaptureLoad 
+
+`CaptureLoad` will look at the topologies on a running cluster and store the structure of and metrics about each of theses topologies storing them in a format that can be used later to reproduce a similar load on the cluster.
+
+#### Usage
+```
+storm jar storm-loadgen.jar org.apache.storm.loadgen.CaptureLoad [options] [topologyName]*
+```
+|Option| Description|
+|-----|-----|
+|-a,--anonymize | Strip out any possibly identifiable information|
+| -h,--help | Print a help message |
+| -o,--output-dir <file> | Where to write (defaults to ./loadgen/)|
+
+#### Limitations
+This is still a work in progress.  It does not currently capture CPU or memory usage of a topology.  Resource requests (used by RAS when scheduling) within the topology are also not captured yet, nor is the user that actually ran the topology.
+
+### GenLoad
+
+`GenLoad` will take the files produced by `CaptureLoad` and replay them in a simulated way on a cluster.  It also offers lots of ways to capture metrics about those simulated topologies to be able to compare different software versions of different hardware setups.  You can also make adjustments to the topology before submitting it to change the size or throughput of the topology.
+
+### Usage
+```
+storm jar storm-loadgen.jar org.apache.storm.loadgen.GenLoad [options] [capture_file]*
+```
+
+|Option| Description|
+|-----|-----|
+| --debug | Print debug information about the adjusted topology before submitting it. |
+|-h,--help | Print a help message |
+| --local-or-shuffle | Replace shuffle grouping with local or shuffle grouping. |
+| --parallel &lt;MULTIPLIER> | How much to scale the topology up or down in parallelism. The new parallelism will round up to the next whole number (defaults to 1.0 no scaling) The total throughput of the topology will not be scaled. |
+| -r,--report-interval &lt;INTERVAL_SECS> | How long in between reported metrics.  Will be rounded up to the next 10 sec boundary. default 30 |
+| --reporter &lt;TYPE:FILE?OPTIONS>  | Provide the config for a reporter to run. See below for more information about these |
+| -t,--test-time &lt;MINS> | How long to run the tests for in mins (defaults to 5) |
+| --throughput &lt;MULTIPLIER> | How much to scale the topology up or down in throughput. (defaults to 1.0 no scaling)|
+| -w,--report-window &lt;INTERVAL_SECS> | How long of a rolling window should be in each report.  Will be rounded up to the next report interval boundary. default 30|
+
+## ThroughputVsLatency
+This is a topology similar to `GenLoad` in most ways, except instead of simulating a load it runs a word count algorithm.
+
+### Usage
+```
+storm jar storm-loadgen.jar org.apache.storm.loadgen.ThroughputVsLatency [options]
+```
+
+|Option| Description|
+|-----|-----|
+|--counters &lt;NUM>| Number of counter bolts to use (defaults to 1)|
+| -h,--help | Print a help message |
+| --name <TOPO_NAME> | Name of the topology to run (defaults to wc-test) |
+| -r,--report-interval &lt;INTERVAL_SECS>| How long in between reported metrics.  Will be rounded up to the next 10 sec boundary. default 30 |
+| --rate &lt;SENTENCES/SEC>| How many sentences per second to run. (defaults to 500) |
+| --reporter &lt;TYPE:FILE?OPTIONS>  | Provide the config for a reporter to run. See below for more information about these |
+|--splitters &lt;NUM> | Number of splitter bolts to use (defaults to 1) |
+| --spouts &lt;NUM>| Number of spouts to use (defaults to 1) |
+| -t,--test-time &lt;MINS>| How long to run the tests for in mins (defaults to 5) |
+| -w,--report-window &lt;INTERVAL_SECS>| How long of a rolling window should be in each report.  Will be rounded up to the next report interval boundary.|
+
+# Reporters
+Reporters provide a way to store various statistics about a running topology. There are currently a few supported reporters
+
+ * legacy - report values like ThroughputVsLatency has done in the past
+ * TSV - tab separated values
+ * CSV - comma separated values
+
+All of these types can have their data written out to a file.  To do this add a path after the type.  For example `legacy:./legacy_data` or `tsv:my_run.tsv`. By default the file will be over written unless an option is given to append instead. Options are in a URL like format, with a `?` separating the type:path from the options, and all of the options separated by a `&`.  To append to the file you can do something like `csv:./my_run.csv?append` or  `csv:./my_run.csv?append=true`
+
+Not all options are supported by all reporters.
+
+|Reporter Option| Description | Supported Reporters|
+|---------------|-------------|--------------------|
+|time | Set the time unit that you want latency and CPU reported in.  This can be from nanoseconds up to seconds.  Most names are supported for the types| legacy, csv, tsv|
+|columns | A comma separated list of columns to output (see below for the metrics supported).  Defaults to "start_time", "end_time", "completion_rate", "mean", "99%ile", "99.9%ile", "cores", "mem", "failed" | csv, tsv |
+|extraColumns | Like columns but ones that should be added to the defaults instead of replacing them (this is mostly for convenience) | csv, tsv |
+|meta | An arbitrary string that will appear as a "meta" column at the end.  This helps when appending to files to keep different runs separated | csv, tsv|
+
+There are a lot of different metrics supported
+
+|Metrics Name| Description|
+|------------|------------|
+|99%ile| 99th percentile completion latency. |
+|99.9%ile| 99.9th percentile completion latency. |
+|median| Median completion latency. |
+|mean| Mean completion latency. |
+|min| Minimum completion latency. |
+|max| Maximum completion latency. |
+|stddev| Standard Deviation of completion latency. |
+|user_cpu| User space CPU time.|
+|sys_cpu| System space CPU time. |
+|gc_cpu| Amount of CPU time spent in GC as reported by the JVM. |
+|cores| The number of CPU cores used. `(user_cpu + sys_cpu) / time_window`|
+|uptime| The amount of time the oldest topology has been up for. |
+|acked| The number of tuples fully acked as reported by Storm's metrics. |
+|rate| The rate of tuples fully acked as reported by Storm's metrics. |
+|completed| The number of tuples fully acked as reported by the latency histogram metrics. |
+|completion_rate| The rate of tuples fully acked as reported by the latency histogram metrics. |
+|mem| The amount of memory used by the topology in MB, as reported by the JVM. |
+|failed| The number of failed tuples as reported by Storm's metrics. |
+|start_time| The starting time of the metrics window from when the first topology was launched.
+|end_time| The ending time of the metrics window from the the first topology was launched.
+|time_window| the length in seconds for the time window. |
+|ids| The topology ids that are being tracked |
+
+# Captured Load File Format
+The file format used with `CaptureLoad` and `GenLoad` is based off of the flux file format, but with some extensions and omissions.
+
+At a top level the supported options keys are
+
+| Config | Description |
+|--------|-------------|
+| name | The name of the topology.  If not given the base name of the file will be used. |
+| config | A map of String to Object configs to use when submitting the topology. |
+| spouts | A list of spouts for the topology. |
+| bolts | A list of bolts in the topology. |
+| streams | A list of streams connecting different components in the topology. |
+
+## Spouts and Bolts
+
+Spouts and bolts have the same format.
+
+| Config | Description |
+|--------|-------------|
+| id | The id of the bolt or spout.  This should be unique within the topology |
+| parallelism | How many instances of this component should be a part of the topology |
+| streams | The streams that are produced by this bolt or spout |
+
+### Output Streams
+
+This is not a part of flux.  It defines the output of a bolt or spout.
+
+| Config | Description |
+|--------|-------------|
+| streamId | The ID of the stream being output.  The default is "default" |
+| rate | This is a map describing the rate at which messages are output on this stream. |
+
+The rate has at least a `mean` value.  If you want the rate to vary a bit over time you can also include a Standard Deviation with `stddev` and a `min` and `max` value.  The actual rates selected will follow a Gaussian distribution within those bounds.
+
+## (Input) Streams
+
+The streams that connect components together has the form.
+
+| Config | Description |
+|--------|-------------|
+| from | the component id the stream is coming from |
+| to | the component id the stream is going to |
+| grouping | This is a map that defines the grouping used |
+| grouping.type | the type of grouping including `SHUFFLE`, `FIELDS`, `ALL`, `GLOBAL`, `LOCAL_OR_SHUFFLE`, `NONE`, or `PARTIAL_KEY`.  defaults to `SHUFFLE` |
+| grouping.streamId | the id of the stream (default is "default") |
+| execTime | a distribution of the amount of time in milliseconds that execution of this component takes (execute latency). |
+| processTime | a distribution of the amount of time in milliseconds that processing of a tuple takes (process latency). |
+
+The `execTime` and `processTime` values follow the same pattern as the `OutputStream` `rate`.  A `mean` values is required, but `stddev`, `min`, and `max` may also be given.

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/examples/storm-loadgen/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/pom.xml b/examples/storm-loadgen/pom.xml
new file mode 100644
index 0000000..f75e575
--- /dev/null
+++ b/examples/storm-loadgen/pom.xml
@@ -0,0 +1,122 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+ 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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>storm</artifactId>
+    <groupId>org.apache.storm</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+  <artifactId>storm-loadgen</artifactId>
+  <packaging>jar</packaging>
+  <name>storm-loadgen</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+  </properties>
+  <dependencies>
+    <dependency>
+      <groupId>org.hdrhistogram</groupId>
+      <artifactId>HdrHistogram</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.storm</groupId>
+      <artifactId>storm-client</artifactId>
+      <version>${project.version}</version>
+      <!--
+        Use "provided" scope to keep storm out of the jar-with-dependencies
+        For IntelliJ dev, intellij will load properly.
+      -->
+      <scope>${provided.scope}</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.storm</groupId>
+      <artifactId>storm-metrics</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <configuration>
+          <createDependencyReducedPom>true</createDependencyReducedPom>
+          <filters>
+            <filter>
+              <artifact>*:*</artifact>
+              <excludes>
+                <exclude>META-INF/*.SF</exclude>
+                <exclude>META-INF/*.sf</exclude>
+                <exclude>META-INF/*.DSA</exclude>
+                <exclude>META-INF/*.dsa</exclude>
+                <exclude>META-INF/*.RSA</exclude>
+                <exclude>META-INF/*.rsa</exclude>
+                <exclude>META-INF/*.EC</exclude>
+                <exclude>META-INF/*.ec</exclude>
+                <exclude>META-INF/MSFTSIG.SF</exclude>
+                <exclude>META-INF/MSFTSIG.RSA</exclude>
+              </excludes>
+            </filter>
+          </filters>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <transformers>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                </transformer>
+              </transformers>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <!--Note - the version would be inherited-->
+        <configuration>
+          <maxAllowedViolations>0</maxAllowedViolations>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
new file mode 100644
index 0000000..aa4579c
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
@@ -0,0 +1,85 @@
+/*
+ * 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.misc.metric;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import java.net.URL;
+import java.net.HttpURLConnection;
+
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.storm.serialization.KryoValuesSerializer;
+
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.task.IErrorReporter;
+import org.apache.storm.task.TopologyContext;
+
+/**
+ * Listens for all metrics and POSTs them serialized to a configured URL
+ *
+ * To use, add this to your topology's configuration:
+ *
+ * ```java
+ *   conf.registerMetricsConsumer(org.apache.storm.metrics.HttpForwardingMetricsConsumer.class, "http://example.com:8080/metrics/my-topology/", 1);
+ * ```
+ *
+ * The body of the post is data serialized using {@link org.apache.storm.serialization.KryoValuesSerializer}, with the data passed in
+ * as a list of `[TaskInfo, Collection<DataPoint>]`.  More things may be appended to the end of the list in the future.
+ *
+ * The values can be deserialized using the org.apache.storm.serialization.KryoValuesDeserializer, and a 
+ * correct config + classpath.
+ *
+ * @see org.apache.storm.serialization.KryoValuesSerializer
+ */
+public class HttpForwardingMetricsConsumer implements IMetricsConsumer {
+    private transient URL _url; 
+    private transient IErrorReporter _errorReporter;
+    private transient KryoValuesSerializer _serializer;
+
+    @Override
+    public void prepare(Map<String, Object> topoConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) { 
+        try {
+            _url = new URL((String)registrationArgument);
+            _errorReporter = errorReporter;
+            _serializer = new KryoValuesSerializer(topoConf);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void handleDataPoints(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
+        try {
+            HttpURLConnection con = (HttpURLConnection)_url.openConnection();
+            con.setRequestMethod("POST");
+            con.setDoOutput(true);
+            Output out = new Output(con.getOutputStream());
+            _serializer.serializeInto(Arrays.asList(taskInfo, dataPoints), out);
+            out.flush();
+            out.close();
+            //The connection is not sent unless a response is requested
+            int response = con.getResponseCode();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void cleanup() { }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
new file mode 100644
index 0000000..ef2769a
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
@@ -0,0 +1,118 @@
+/*
+ * 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.misc.metric;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.List;
+import java.net.ServerSocket;
+import java.net.InetAddress;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.ServletException;
+
+import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
+import org.apache.storm.metric.api.IMetricsConsumer.DataPoint;
+
+import com.esotericsoftware.kryo.io.Input;
+import org.apache.storm.serialization.KryoValuesDeserializer;
+import org.apache.storm.utils.Utils;
+
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+/**
+ * A server that can listen for metrics from the HttpForwardingMetricsConsumer.
+ */
+public abstract class HttpForwardingMetricsServer {
+    private Map _conf;
+    private Server _server = null;
+    private int _port = -1;
+    private String _url = null;
+
+    ThreadLocal<KryoValuesDeserializer> _des = new ThreadLocal<KryoValuesDeserializer>() {
+        @Override
+        protected KryoValuesDeserializer initialValue() {
+            return new KryoValuesDeserializer(_conf);
+        }
+    };
+
+    private class MetricsCollectionServlet extends HttpServlet
+    {
+        protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException
+        {
+            Input in = new Input(request.getInputStream());
+            List<Object> metrics = _des.get().deserializeFrom(in);
+            handle((TaskInfo)metrics.get(0), (Collection<DataPoint>)metrics.get(1));
+            response.setStatus(HttpServletResponse.SC_OK);
+        }
+    }
+
+    public HttpForwardingMetricsServer(Map<String, Object> conf) {
+        _conf = Utils.readStormConfig();
+        if (conf != null) {
+            _conf.putAll(conf);
+        }
+    }
+
+    //This needs to be thread safe
+    public abstract void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints);
+
+    public void serve(Integer port) {
+        try {
+            if (_server != null) throw new RuntimeException("The server is already running");
+    
+            if (port == null || port <= 0) {
+                ServerSocket s = new ServerSocket(0);
+                port = s.getLocalPort();
+                s.close();
+            }
+            _server = new Server(port);
+            _port = port;
+            _url = "http://"+InetAddress.getLocalHost().getHostName()+":"+_port+"/";
+ 
+            ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+            context.setContextPath("/");
+            _server.setHandler(context);
+ 
+            context.addServlet(new ServletHolder(new MetricsCollectionServlet()),"/*");
+
+            _server.start();
+         } catch (RuntimeException e) {
+             throw e;
+         } catch (Exception e) {
+             throw new RuntimeException(e);
+         }
+    }
+
+    public void serve() {
+        serve(null);
+    }
+
+    public int getPort() {
+        return _port;
+    }
+
+    public String getUrl() {
+        return _url;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
new file mode 100644
index 0000000..96c13c5
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
@@ -0,0 +1,377 @@
+/**
+ * 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.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.HdrHistogram.Histogram;
+import org.apache.storm.Config;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.generated.ClusterSummary;
+import org.apache.storm.generated.ExecutorSummary;
+import org.apache.storm.generated.KillOptions;
+import org.apache.storm.generated.Nimbus;
+import org.apache.storm.generated.SpoutStats;
+import org.apache.storm.generated.TopologyInfo;
+import org.apache.storm.generated.TopologySummary;
+import org.apache.storm.metric.api.IMetricsConsumer.DataPoint;
+import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
+import org.apache.storm.metrics.hdrhistogram.HistogramMetric;
+import org.apache.storm.misc.metric.HttpForwardingMetricsServer;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.topology.base.BaseBasicBolt;
+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.NimbusClient;
+
+/**
+ * WordCount but the spout goes at a predefined rate and we collect
+ * proper latency statistics.
+ */
+public class ThroughputVsLatency {
+    private static class SentWithTime {
+        public final String sentence;
+        public final long time;
+
+        SentWithTime(String sentence, long time) {
+            this.sentence = sentence;
+            this.time = time;
+        }
+    }
+
+    public static class FastRandomSentenceSpout extends BaseRichSpout {
+        static final String[] SENTENCES = new String[]{ "the cow jumped over the moon", "an apple a day keeps the doctor away",
+                "four score and seven years ago", "snow white and the seven dwarfs", "i am at two with nature" };
+
+        SpoutOutputCollector _collector;
+        long _periodNano;
+        long _emitAmount;
+        Random _rand;
+        long _nextEmitTime;
+        long _emitsLeft;
+        HistogramMetric _histo;
+
+        public FastRandomSentenceSpout(long ratePerSecond) {
+            if (ratePerSecond > 0) {
+                _periodNano = Math.max(1, 1000000000/ratePerSecond);
+                _emitAmount = Math.max(1, (long)((ratePerSecond / 1000000000.0) * _periodNano));
+            } else {
+                _periodNano = Long.MAX_VALUE - 1;
+                _emitAmount = 1;
+            }
+        }
+
+        @Override
+        public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
+            _collector = collector;
+            _rand = ThreadLocalRandom.current();
+            _nextEmitTime = System.nanoTime();
+            _emitsLeft = _emitAmount;
+            _histo = new HistogramMetric(3600000000000L, 3);
+            context.registerMetric("comp-lat-histo", _histo, 10); //Update every 10 seconds, so we are not too far behind
+        }
+
+        @Override
+        public void nextTuple() {
+            if (_emitsLeft <= 0 && _nextEmitTime <= System.nanoTime()) {
+                _emitsLeft = _emitAmount;
+                _nextEmitTime = _nextEmitTime + _periodNano;
+            }
+
+            if (_emitsLeft > 0) {
+                String sentence = SENTENCES[_rand.nextInt(SENTENCES.length)];
+                _collector.emit(new Values(sentence), new SentWithTime(sentence, _nextEmitTime - _periodNano));
+                _emitsLeft--;
+            }
+        }
+
+        @Override
+        public void ack(Object id) {
+            long end = System.nanoTime();
+            SentWithTime st = (SentWithTime)id;
+            _histo.recordValue(end-st.time);
+        }
+
+        @Override
+        public void fail(Object id) {
+            SentWithTime st = (SentWithTime)id;
+            _collector.emit(new Values(st.sentence), id);
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("sentence"));
+        }
+    }
+
+    public static class SplitSentence extends BaseBasicBolt {
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String sentence = tuple.getString(0);
+            for (String word: sentence.split("\\s+")) {
+                collector.emit(new Values(word, 1));
+            }
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word", "count"));
+        }
+    }
+
+    public static class WordCount extends BaseBasicBolt {
+        Map<String, Integer> counts = new HashMap<String, Integer>();
+
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String word = tuple.getString(0);
+            Integer count = counts.get(word);
+            if (count == null)
+                count = 0;
+            count++;
+            counts.put(word, count);
+            collector.emit(new Values(word, count));
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word", "count"));
+        }
+    }
+
+    private static class MemMeasure {
+        private long _mem = 0;
+        private long _time = 0;
+
+        public synchronized void update(long mem) {
+            _mem = mem;
+            _time = System.currentTimeMillis();
+        }
+
+        public synchronized long get() {
+            return isExpired() ? 0l : _mem;
+        }
+
+        public synchronized boolean isExpired() {
+            return (System.currentTimeMillis() - _time) >= 20000;
+        }
+    }
+
+    private static final Histogram _histo = new Histogram(3600000000000L, 3);
+    private static final AtomicLong _systemCPU = new AtomicLong(0);
+    private static final AtomicLong _userCPU = new AtomicLong(0);
+    private static final AtomicLong _gcCount = new AtomicLong(0);
+    private static final AtomicLong _gcMs = new AtomicLong(0);
+    private static final ConcurrentHashMap<String, MemMeasure> _memoryBytes = new ConcurrentHashMap<String, MemMeasure>();
+
+    private static long readMemory() {
+        long total = 0;
+        for (MemMeasure mem: _memoryBytes.values()) {
+            total += mem.get();
+        }
+        return total;
+    }
+
+    private static long _prev_acked = 0;
+    private static long _prev_uptime = 0;
+
+    public static void printMetrics(Nimbus.Iface client, String name) throws Exception {
+        ClusterSummary summary = client.getClusterInfo();
+        String id = null;
+        for (TopologySummary ts: summary.get_topologies()) {
+            if (name.equals(ts.get_name())) {
+                id = ts.get_id();
+            }
+        }
+        if (id == null) {
+            throw new Exception("Could not find a topology named "+name);
+        }
+        TopologyInfo info = client.getTopologyInfo(id);
+        int uptime = info.get_uptime_secs();
+        long acked = 0;
+        long failed = 0;
+        for (ExecutorSummary exec: info.get_executors()) {
+            if ("spout".equals(exec.get_component_id()) && exec.get_stats() != null && exec.get_stats().get_specific() != null) {
+                SpoutStats stats = exec.get_stats().get_specific().get_spout();
+                Map<String, Long> failedMap = stats.get_failed().get(":all-time");
+                Map<String, Long> ackedMap = stats.get_acked().get(":all-time");
+                if (ackedMap != null) {
+                    for (String key: ackedMap.keySet()) {
+                        if (failedMap != null) {
+                            Long tmp = failedMap.get(key);
+                            if (tmp != null) {
+                                failed += tmp;
+                            }
+                        }
+                        long ackVal = ackedMap.get(key);
+                        acked += ackVal;
+                    }
+                }
+            }
+        }
+        long ackedThisTime = acked - _prev_acked;
+        long thisTime = uptime - _prev_uptime;
+        long nnpct, nnnpct, min, max;
+        double mean, stddev;
+        synchronized(_histo) {
+            nnpct = _histo.getValueAtPercentile(99.0);
+            nnnpct = _histo.getValueAtPercentile(99.9);
+            min = _histo.getMinValue();
+            max = _histo.getMaxValue();
+            mean = _histo.getMean();
+            stddev = _histo.getStdDeviation();
+            _histo.reset();
+        }
+        long user = _userCPU.getAndSet(0);
+        long sys = _systemCPU.getAndSet(0);
+        long gc = _gcMs.getAndSet(0);
+        double memMB = readMemory() / (1024.0 * 1024.0);
+        System.out.printf("uptime: %,4d acked: %,9d acked/sec: %,10.2f failed: %,8d " +
+                "99%%: %,15d 99.9%%: %,15d min: %,15d max: %,15d mean: %,15.2f " +
+                "stddev: %,15.2f user: %,10d sys: %,10d gc: %,10d mem: %,10.2f\n",
+                uptime, ackedThisTime, (((double)ackedThisTime)/thisTime), failed, nnpct, nnnpct,
+                min, max, mean, stddev, user, sys, gc, memMB);
+        _prev_uptime = uptime;
+        _prev_acked = acked;
+    }
+
+    public static void kill(Nimbus.Iface client, String name) throws Exception {
+        KillOptions opts = new KillOptions();
+        opts.set_wait_secs(0);
+        client.killTopologyWithOpts(name, opts);
+    }
+
+    public static void main(String[] args) throws Exception {
+        long ratePerSecond = 500;
+        if (args != null && args.length > 0) {
+            ratePerSecond = Long.valueOf(args[0]);
+        }
+
+        int parallelism = 4;
+        if (args != null && args.length > 1) {
+            parallelism = Integer.valueOf(args[1]);
+        }
+
+        int numMins = 5;
+        if (args != null && args.length > 2) {
+            numMins = Integer.valueOf(args[2]);
+        }
+
+        String name = "wc-test";
+        if (args != null && args.length > 3) {
+            name = args[3];
+        }
+
+        Config conf = new Config();
+        HttpForwardingMetricsServer metricServer = new HttpForwardingMetricsServer(conf) {
+            @Override
+            public void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
+                String worker = taskInfo.srcWorkerHost + ":" + taskInfo.srcWorkerPort;
+                for (DataPoint dp: dataPoints) {
+                    if ("comp-lat-histo".equals(dp.name) && dp.value instanceof Histogram) {
+                        synchronized(_histo) {
+                            _histo.add((Histogram)dp.value);
+                        }
+                    } else if ("CPU".equals(dp.name) && dp.value instanceof Map) {
+                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                        Object sys = m.get("sys-ms");
+                        if (sys instanceof Number) {
+                            _systemCPU.getAndAdd(((Number)sys).longValue());
+                        }
+                        Object user = m.get("user-ms");
+                        if (user instanceof Number) {
+                            _userCPU.getAndAdd(((Number)user).longValue());
+                        }
+                    } else if (dp.name.startsWith("GC/") && dp.value instanceof Map) {
+                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                        Object count = m.get("count");
+                        if (count instanceof Number) {
+                            _gcCount.getAndAdd(((Number)count).longValue());
+                        }
+                        Object time = m.get("timeMs");
+                        if (time instanceof Number) {
+                            _gcMs.getAndAdd(((Number)time).longValue());
+                        }
+                    } else if (dp.name.startsWith("memory/") && dp.value instanceof Map) {
+                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                        Object val = m.get("usedBytes");
+                        if (val instanceof Number) {
+                            MemMeasure mm = _memoryBytes.get(worker);
+                            if (mm == null) {
+                                mm = new MemMeasure();
+                                MemMeasure tmp = _memoryBytes.putIfAbsent(worker, mm);
+                                mm = tmp == null ? mm : tmp; 
+                            }
+                            mm.update(((Number)val).longValue());
+                        }
+                    }
+                }
+            }
+        };
+
+        metricServer.serve();
+        String url = metricServer.getUrl();
+
+        NimbusClient client = NimbusClient.getConfiguredClient(conf);
+        conf.setNumWorkers(parallelism);
+        conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class);
+        conf.registerMetricsConsumer(org.apache.storm.misc.metric.HttpForwardingMetricsConsumer.class, url, 1);
+        Map<String, String> workerMetrics = new HashMap<String, String>();
+        if (!NimbusClient.isLocalOverride()) {
+            //sigar uses JNI and does not work in local mode
+            workerMetrics.put("CPU", "org.apache.storm.metrics.sigar.CPUMetric");
+        }
+        conf.put(Config.TOPOLOGY_WORKER_METRICS, workerMetrics);
+        conf.put(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS, 10);
+        conf.put(Config.TOPOLOGY_WORKER_GC_CHILDOPTS,
+                "-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:NewSize=128m -XX:CMSInitiatingOccupancyFraction=70 -XX:-CMSConcurrentMTEnabled");
+        conf.put(Config.TOPOLOGY_WORKER_CHILDOPTS, "-Xmx2g");
+
+        TopologyBuilder builder = new TopologyBuilder();
+
+        int numEach = 4 * parallelism;
+        builder.setSpout("spout", new FastRandomSentenceSpout(ratePerSecond/numEach), numEach);
+
+        builder.setBolt("split", new SplitSentence(), numEach).shuffleGrouping("spout");
+        builder.setBolt("count", new WordCount(), numEach).fieldsGrouping("split", new Fields("word"));
+
+        try {
+            StormSubmitter.submitTopology(name, conf, builder.createTopology());
+
+            for (int i = 0; i < numMins * 2; i++) {
+                Thread.sleep(30 * 1000);
+                printMetrics(client.getClient(), name);
+            }
+        } finally {
+            kill(client.getClient(), name);
+            System.exit(0);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/examples/storm-starter/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml
index 2ef18f2..e95958d 100644
--- a/examples/storm-starter/pom.xml
+++ b/examples/storm-starter/pom.xml
@@ -90,11 +90,6 @@
     </dependency>
       <dependency>
           <groupId>org.apache.storm</groupId>
-          <artifactId>storm-client-misc</artifactId>
-          <version>${project.version}</version>
-      </dependency>
-      <dependency>
-          <groupId>org.apache.storm</groupId>
           <artifactId>storm-client</artifactId>
           <version>${project.version}</version>
           <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
deleted file mode 100644
index 96c13c5..0000000
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
+++ /dev/null
@@ -1,377 +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.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.HdrHistogram.Histogram;
-import org.apache.storm.Config;
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.ClusterSummary;
-import org.apache.storm.generated.ExecutorSummary;
-import org.apache.storm.generated.KillOptions;
-import org.apache.storm.generated.Nimbus;
-import org.apache.storm.generated.SpoutStats;
-import org.apache.storm.generated.TopologyInfo;
-import org.apache.storm.generated.TopologySummary;
-import org.apache.storm.metric.api.IMetricsConsumer.DataPoint;
-import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
-import org.apache.storm.metrics.hdrhistogram.HistogramMetric;
-import org.apache.storm.misc.metric.HttpForwardingMetricsServer;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.topology.base.BaseBasicBolt;
-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.NimbusClient;
-
-/**
- * WordCount but the spout goes at a predefined rate and we collect
- * proper latency statistics.
- */
-public class ThroughputVsLatency {
-    private static class SentWithTime {
-        public final String sentence;
-        public final long time;
-
-        SentWithTime(String sentence, long time) {
-            this.sentence = sentence;
-            this.time = time;
-        }
-    }
-
-    public static class FastRandomSentenceSpout extends BaseRichSpout {
-        static final String[] SENTENCES = new String[]{ "the cow jumped over the moon", "an apple a day keeps the doctor away",
-                "four score and seven years ago", "snow white and the seven dwarfs", "i am at two with nature" };
-
-        SpoutOutputCollector _collector;
-        long _periodNano;
-        long _emitAmount;
-        Random _rand;
-        long _nextEmitTime;
-        long _emitsLeft;
-        HistogramMetric _histo;
-
-        public FastRandomSentenceSpout(long ratePerSecond) {
-            if (ratePerSecond > 0) {
-                _periodNano = Math.max(1, 1000000000/ratePerSecond);
-                _emitAmount = Math.max(1, (long)((ratePerSecond / 1000000000.0) * _periodNano));
-            } else {
-                _periodNano = Long.MAX_VALUE - 1;
-                _emitAmount = 1;
-            }
-        }
-
-        @Override
-        public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
-            _collector = collector;
-            _rand = ThreadLocalRandom.current();
-            _nextEmitTime = System.nanoTime();
-            _emitsLeft = _emitAmount;
-            _histo = new HistogramMetric(3600000000000L, 3);
-            context.registerMetric("comp-lat-histo", _histo, 10); //Update every 10 seconds, so we are not too far behind
-        }
-
-        @Override
-        public void nextTuple() {
-            if (_emitsLeft <= 0 && _nextEmitTime <= System.nanoTime()) {
-                _emitsLeft = _emitAmount;
-                _nextEmitTime = _nextEmitTime + _periodNano;
-            }
-
-            if (_emitsLeft > 0) {
-                String sentence = SENTENCES[_rand.nextInt(SENTENCES.length)];
-                _collector.emit(new Values(sentence), new SentWithTime(sentence, _nextEmitTime - _periodNano));
-                _emitsLeft--;
-            }
-        }
-
-        @Override
-        public void ack(Object id) {
-            long end = System.nanoTime();
-            SentWithTime st = (SentWithTime)id;
-            _histo.recordValue(end-st.time);
-        }
-
-        @Override
-        public void fail(Object id) {
-            SentWithTime st = (SentWithTime)id;
-            _collector.emit(new Values(st.sentence), id);
-        }
-
-        @Override
-        public void declareOutputFields(OutputFieldsDeclarer declarer) {
-            declarer.declare(new Fields("sentence"));
-        }
-    }
-
-    public static class SplitSentence extends BaseBasicBolt {
-        @Override
-        public void execute(Tuple tuple, BasicOutputCollector collector) {
-            String sentence = tuple.getString(0);
-            for (String word: sentence.split("\\s+")) {
-                collector.emit(new Values(word, 1));
-            }
-        }
-
-        @Override
-        public void declareOutputFields(OutputFieldsDeclarer declarer) {
-            declarer.declare(new Fields("word", "count"));
-        }
-    }
-
-    public static class WordCount extends BaseBasicBolt {
-        Map<String, Integer> counts = new HashMap<String, Integer>();
-
-        @Override
-        public void execute(Tuple tuple, BasicOutputCollector collector) {
-            String word = tuple.getString(0);
-            Integer count = counts.get(word);
-            if (count == null)
-                count = 0;
-            count++;
-            counts.put(word, count);
-            collector.emit(new Values(word, count));
-        }
-
-        @Override
-        public void declareOutputFields(OutputFieldsDeclarer declarer) {
-            declarer.declare(new Fields("word", "count"));
-        }
-    }
-
-    private static class MemMeasure {
-        private long _mem = 0;
-        private long _time = 0;
-
-        public synchronized void update(long mem) {
-            _mem = mem;
-            _time = System.currentTimeMillis();
-        }
-
-        public synchronized long get() {
-            return isExpired() ? 0l : _mem;
-        }
-
-        public synchronized boolean isExpired() {
-            return (System.currentTimeMillis() - _time) >= 20000;
-        }
-    }
-
-    private static final Histogram _histo = new Histogram(3600000000000L, 3);
-    private static final AtomicLong _systemCPU = new AtomicLong(0);
-    private static final AtomicLong _userCPU = new AtomicLong(0);
-    private static final AtomicLong _gcCount = new AtomicLong(0);
-    private static final AtomicLong _gcMs = new AtomicLong(0);
-    private static final ConcurrentHashMap<String, MemMeasure> _memoryBytes = new ConcurrentHashMap<String, MemMeasure>();
-
-    private static long readMemory() {
-        long total = 0;
-        for (MemMeasure mem: _memoryBytes.values()) {
-            total += mem.get();
-        }
-        return total;
-    }
-
-    private static long _prev_acked = 0;
-    private static long _prev_uptime = 0;
-
-    public static void printMetrics(Nimbus.Iface client, String name) throws Exception {
-        ClusterSummary summary = client.getClusterInfo();
-        String id = null;
-        for (TopologySummary ts: summary.get_topologies()) {
-            if (name.equals(ts.get_name())) {
-                id = ts.get_id();
-            }
-        }
-        if (id == null) {
-            throw new Exception("Could not find a topology named "+name);
-        }
-        TopologyInfo info = client.getTopologyInfo(id);
-        int uptime = info.get_uptime_secs();
-        long acked = 0;
-        long failed = 0;
-        for (ExecutorSummary exec: info.get_executors()) {
-            if ("spout".equals(exec.get_component_id()) && exec.get_stats() != null && exec.get_stats().get_specific() != null) {
-                SpoutStats stats = exec.get_stats().get_specific().get_spout();
-                Map<String, Long> failedMap = stats.get_failed().get(":all-time");
-                Map<String, Long> ackedMap = stats.get_acked().get(":all-time");
-                if (ackedMap != null) {
-                    for (String key: ackedMap.keySet()) {
-                        if (failedMap != null) {
-                            Long tmp = failedMap.get(key);
-                            if (tmp != null) {
-                                failed += tmp;
-                            }
-                        }
-                        long ackVal = ackedMap.get(key);
-                        acked += ackVal;
-                    }
-                }
-            }
-        }
-        long ackedThisTime = acked - _prev_acked;
-        long thisTime = uptime - _prev_uptime;
-        long nnpct, nnnpct, min, max;
-        double mean, stddev;
-        synchronized(_histo) {
-            nnpct = _histo.getValueAtPercentile(99.0);
-            nnnpct = _histo.getValueAtPercentile(99.9);
-            min = _histo.getMinValue();
-            max = _histo.getMaxValue();
-            mean = _histo.getMean();
-            stddev = _histo.getStdDeviation();
-            _histo.reset();
-        }
-        long user = _userCPU.getAndSet(0);
-        long sys = _systemCPU.getAndSet(0);
-        long gc = _gcMs.getAndSet(0);
-        double memMB = readMemory() / (1024.0 * 1024.0);
-        System.out.printf("uptime: %,4d acked: %,9d acked/sec: %,10.2f failed: %,8d " +
-                "99%%: %,15d 99.9%%: %,15d min: %,15d max: %,15d mean: %,15.2f " +
-                "stddev: %,15.2f user: %,10d sys: %,10d gc: %,10d mem: %,10.2f\n",
-                uptime, ackedThisTime, (((double)ackedThisTime)/thisTime), failed, nnpct, nnnpct,
-                min, max, mean, stddev, user, sys, gc, memMB);
-        _prev_uptime = uptime;
-        _prev_acked = acked;
-    }
-
-    public static void kill(Nimbus.Iface client, String name) throws Exception {
-        KillOptions opts = new KillOptions();
-        opts.set_wait_secs(0);
-        client.killTopologyWithOpts(name, opts);
-    }
-
-    public static void main(String[] args) throws Exception {
-        long ratePerSecond = 500;
-        if (args != null && args.length > 0) {
-            ratePerSecond = Long.valueOf(args[0]);
-        }
-
-        int parallelism = 4;
-        if (args != null && args.length > 1) {
-            parallelism = Integer.valueOf(args[1]);
-        }
-
-        int numMins = 5;
-        if (args != null && args.length > 2) {
-            numMins = Integer.valueOf(args[2]);
-        }
-
-        String name = "wc-test";
-        if (args != null && args.length > 3) {
-            name = args[3];
-        }
-
-        Config conf = new Config();
-        HttpForwardingMetricsServer metricServer = new HttpForwardingMetricsServer(conf) {
-            @Override
-            public void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
-                String worker = taskInfo.srcWorkerHost + ":" + taskInfo.srcWorkerPort;
-                for (DataPoint dp: dataPoints) {
-                    if ("comp-lat-histo".equals(dp.name) && dp.value instanceof Histogram) {
-                        synchronized(_histo) {
-                            _histo.add((Histogram)dp.value);
-                        }
-                    } else if ("CPU".equals(dp.name) && dp.value instanceof Map) {
-                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
-                        Object sys = m.get("sys-ms");
-                        if (sys instanceof Number) {
-                            _systemCPU.getAndAdd(((Number)sys).longValue());
-                        }
-                        Object user = m.get("user-ms");
-                        if (user instanceof Number) {
-                            _userCPU.getAndAdd(((Number)user).longValue());
-                        }
-                    } else if (dp.name.startsWith("GC/") && dp.value instanceof Map) {
-                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
-                        Object count = m.get("count");
-                        if (count instanceof Number) {
-                            _gcCount.getAndAdd(((Number)count).longValue());
-                        }
-                        Object time = m.get("timeMs");
-                        if (time instanceof Number) {
-                            _gcMs.getAndAdd(((Number)time).longValue());
-                        }
-                    } else if (dp.name.startsWith("memory/") && dp.value instanceof Map) {
-                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
-                        Object val = m.get("usedBytes");
-                        if (val instanceof Number) {
-                            MemMeasure mm = _memoryBytes.get(worker);
-                            if (mm == null) {
-                                mm = new MemMeasure();
-                                MemMeasure tmp = _memoryBytes.putIfAbsent(worker, mm);
-                                mm = tmp == null ? mm : tmp; 
-                            }
-                            mm.update(((Number)val).longValue());
-                        }
-                    }
-                }
-            }
-        };
-
-        metricServer.serve();
-        String url = metricServer.getUrl();
-
-        NimbusClient client = NimbusClient.getConfiguredClient(conf);
-        conf.setNumWorkers(parallelism);
-        conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class);
-        conf.registerMetricsConsumer(org.apache.storm.misc.metric.HttpForwardingMetricsConsumer.class, url, 1);
-        Map<String, String> workerMetrics = new HashMap<String, String>();
-        if (!NimbusClient.isLocalOverride()) {
-            //sigar uses JNI and does not work in local mode
-            workerMetrics.put("CPU", "org.apache.storm.metrics.sigar.CPUMetric");
-        }
-        conf.put(Config.TOPOLOGY_WORKER_METRICS, workerMetrics);
-        conf.put(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS, 10);
-        conf.put(Config.TOPOLOGY_WORKER_GC_CHILDOPTS,
-                "-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:NewSize=128m -XX:CMSInitiatingOccupancyFraction=70 -XX:-CMSConcurrentMTEnabled");
-        conf.put(Config.TOPOLOGY_WORKER_CHILDOPTS, "-Xmx2g");
-
-        TopologyBuilder builder = new TopologyBuilder();
-
-        int numEach = 4 * parallelism;
-        builder.setSpout("spout", new FastRandomSentenceSpout(ratePerSecond/numEach), numEach);
-
-        builder.setBolt("split", new SplitSentence(), numEach).shuffleGrouping("spout");
-        builder.setBolt("count", new WordCount(), numEach).fieldsGrouping("split", new Fields("word"));
-
-        try {
-            StormSubmitter.submitTopology(name, conf, builder.createTopology());
-
-            for (int i = 0; i < numMins * 2; i++) {
-                Thread.sleep(30 * 1000);
-                printMetrics(client.getClient(), name);
-            }
-        } finally {
-            kill(client.getClient(), name);
-            System.exit(0);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 006387b..02a2ba6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -326,7 +326,6 @@
         <module>storm-buildtools/maven-shade-clojure-transformer</module>
         <module>storm-buildtools/storm-maven-plugins</module>
         <module>storm-client</module>
-        <module>storm-client-misc</module>
         <module>storm-server</module>
         <module>storm-core</module>
         <module>storm-webapp</module>
@@ -362,6 +361,7 @@
 
         <!-- examples -->
         <module>examples/storm-starter</module>
+	<module>examples/storm-loadgen</module>
         <module>examples/storm-mongodb-examples</module>
         <module>examples/storm-redis-examples</module>
         <module>examples/storm-opentsdb-examples</module>

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/storm-client-misc/pom.xml
----------------------------------------------------------------------
diff --git a/storm-client-misc/pom.xml b/storm-client-misc/pom.xml
deleted file mode 100644
index 276e2c8..0000000
--- a/storm-client-misc/pom.xml
+++ /dev/null
@@ -1,63 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ 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.
-  -->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <parent>
-        <artifactId>storm</artifactId>
-        <groupId>org.apache.storm</groupId>
-        <version>2.0.0-SNAPSHOT</version>
-        <relativePath>..</relativePath>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-
-    <artifactId>storm-client-misc</artifactId>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-client</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.eclipse.jetty</groupId>
-            <artifactId>jetty-server</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.eclipse.jetty</groupId>
-            <artifactId>jetty-servlet</artifactId>
-        </dependency>
-    </dependencies>
-
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-checkstyle-plugin</artifactId>
-                <!--Note - the version would be inherited-->
-                <configuration>
-                    <maxAllowedViolations>39</maxAllowedViolations>
-                </configuration>
-            </plugin>
-        </plugins>
-    </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsConsumer.java
----------------------------------------------------------------------
diff --git a/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsConsumer.java b/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsConsumer.java
deleted file mode 100644
index aa4579c..0000000
--- a/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsConsumer.java
+++ /dev/null
@@ -1,85 +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.misc.metric;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Map;
-import java.net.URL;
-import java.net.HttpURLConnection;
-
-import com.esotericsoftware.kryo.io.Output;
-import org.apache.storm.serialization.KryoValuesSerializer;
-
-import org.apache.storm.metric.api.IMetricsConsumer;
-import org.apache.storm.task.IErrorReporter;
-import org.apache.storm.task.TopologyContext;
-
-/**
- * Listens for all metrics and POSTs them serialized to a configured URL
- *
- * To use, add this to your topology's configuration:
- *
- * ```java
- *   conf.registerMetricsConsumer(org.apache.storm.metrics.HttpForwardingMetricsConsumer.class, "http://example.com:8080/metrics/my-topology/", 1);
- * ```
- *
- * The body of the post is data serialized using {@link org.apache.storm.serialization.KryoValuesSerializer}, with the data passed in
- * as a list of `[TaskInfo, Collection<DataPoint>]`.  More things may be appended to the end of the list in the future.
- *
- * The values can be deserialized using the org.apache.storm.serialization.KryoValuesDeserializer, and a 
- * correct config + classpath.
- *
- * @see org.apache.storm.serialization.KryoValuesSerializer
- */
-public class HttpForwardingMetricsConsumer implements IMetricsConsumer {
-    private transient URL _url; 
-    private transient IErrorReporter _errorReporter;
-    private transient KryoValuesSerializer _serializer;
-
-    @Override
-    public void prepare(Map<String, Object> topoConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) { 
-        try {
-            _url = new URL((String)registrationArgument);
-            _errorReporter = errorReporter;
-            _serializer = new KryoValuesSerializer(topoConf);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public void handleDataPoints(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
-        try {
-            HttpURLConnection con = (HttpURLConnection)_url.openConnection();
-            con.setRequestMethod("POST");
-            con.setDoOutput(true);
-            Output out = new Output(con.getOutputStream());
-            _serializer.serializeInto(Arrays.asList(taskInfo, dataPoints), out);
-            out.flush();
-            out.close();
-            //The connection is not sent unless a response is requested
-            int response = con.getResponseCode();
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public void cleanup() { }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/0d10b8af/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsServer.java
----------------------------------------------------------------------
diff --git a/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsServer.java b/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsServer.java
deleted file mode 100644
index ef2769a..0000000
--- a/storm-client-misc/src/main/java/org/apache/storm/misc/metric/HttpForwardingMetricsServer.java
+++ /dev/null
@@ -1,118 +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.misc.metric;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.List;
-import java.net.ServerSocket;
-import java.net.InetAddress;
-
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.servlet.ServletException;
-
-import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
-import org.apache.storm.metric.api.IMetricsConsumer.DataPoint;
-
-import com.esotericsoftware.kryo.io.Input;
-import org.apache.storm.serialization.KryoValuesDeserializer;
-import org.apache.storm.utils.Utils;
-
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.ServletHolder;
-
-/**
- * A server that can listen for metrics from the HttpForwardingMetricsConsumer.
- */
-public abstract class HttpForwardingMetricsServer {
-    private Map _conf;
-    private Server _server = null;
-    private int _port = -1;
-    private String _url = null;
-
-    ThreadLocal<KryoValuesDeserializer> _des = new ThreadLocal<KryoValuesDeserializer>() {
-        @Override
-        protected KryoValuesDeserializer initialValue() {
-            return new KryoValuesDeserializer(_conf);
-        }
-    };
-
-    private class MetricsCollectionServlet extends HttpServlet
-    {
-        protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException
-        {
-            Input in = new Input(request.getInputStream());
-            List<Object> metrics = _des.get().deserializeFrom(in);
-            handle((TaskInfo)metrics.get(0), (Collection<DataPoint>)metrics.get(1));
-            response.setStatus(HttpServletResponse.SC_OK);
-        }
-    }
-
-    public HttpForwardingMetricsServer(Map<String, Object> conf) {
-        _conf = Utils.readStormConfig();
-        if (conf != null) {
-            _conf.putAll(conf);
-        }
-    }
-
-    //This needs to be thread safe
-    public abstract void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints);
-
-    public void serve(Integer port) {
-        try {
-            if (_server != null) throw new RuntimeException("The server is already running");
-    
-            if (port == null || port <= 0) {
-                ServerSocket s = new ServerSocket(0);
-                port = s.getLocalPort();
-                s.close();
-            }
-            _server = new Server(port);
-            _port = port;
-            _url = "http://"+InetAddress.getLocalHost().getHostName()+":"+_port+"/";
- 
-            ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
-            context.setContextPath("/");
-            _server.setHandler(context);
- 
-            context.addServlet(new ServletHolder(new MetricsCollectionServlet()),"/*");
-
-            _server.start();
-         } catch (RuntimeException e) {
-             throw e;
-         } catch (Exception e) {
-             throw new RuntimeException(e);
-         }
-    }
-
-    public void serve() {
-        serve(null);
-    }
-
-    public int getPort() {
-        return _port;
-    }
-
-    public String getUrl() {
-        return _url;
-    }
-}


[02/18] storm git commit: STORM-2702: storm-loadgen

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
new file mode 100644
index 0000000..1a45ccc
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
@@ -0,0 +1,432 @@
+/**
+ * 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.loadgen;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.storm.Config;
+import org.apache.storm.generated.GlobalStreamId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.Yaml;
+import org.yaml.snakeyaml.constructor.SafeConstructor;
+
+/**
+ * Configuration for a simulated topology.
+ */
+public class TopologyLoadConf {
+    private static final Logger LOG = LoggerFactory.getLogger(TopologyLoadConf.class);
+    static final Set<String> IMPORTANT_CONF_KEYS = Collections.unmodifiableSet(new HashSet(Arrays.asList(
+        Config.TOPOLOGY_WORKERS,
+        Config.TOPOLOGY_ACKER_EXECUTORS,
+        Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT,
+        Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB,
+        Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
+        Config.TOPOLOGY_DISABLE_LOADAWARE_MESSAGING,
+        Config.TOPOLOGY_DEBUG,
+        Config.TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE,
+        Config.TOPOLOGY_ISOLATED_MACHINES,
+        Config.TOPOLOGY_MAX_SPOUT_PENDING,
+        Config.TOPOLOGY_MAX_TASK_PARALLELISM,
+        Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS,
+        Config.TOPOLOGY_PRIORITY,
+        Config.TOPOLOGY_SCHEDULER_STRATEGY,
+        Config.TOPOLOGY_SHELLBOLT_MAX_PENDING,
+        Config.TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS,
+        Config.TOPOLOGY_SPOUT_WAIT_STRATEGY,
+        Config.TOPOLOGY_WORKER_CHILDOPTS,
+        Config.TOPOLOGY_WORKER_GC_CHILDOPTS,
+        Config.TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE,
+        Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB
+    )));
+    private static AtomicInteger topoUniquifier = new AtomicInteger(0);
+
+    public final String name;
+    public final Map<String, Object> topoConf;
+    public final List<LoadCompConf> spouts;
+    public final List<LoadCompConf> bolts;
+    public final List<InputStream> streams;
+    private final AtomicInteger boltUniquifier = new AtomicInteger(0);
+    private final AtomicInteger spoutUniquifier = new AtomicInteger(0);
+    private final AtomicInteger streamUniquifier = new AtomicInteger(0);
+
+    /**
+     * Parse the TopologyLoadConf from a file in YAML format.
+     * @param file the file to read from
+     * @return the parsed conf
+     * @throws IOException if there is an issue reading the file.
+     */
+    public static TopologyLoadConf fromConf(File file) throws IOException {
+        Yaml yaml = new Yaml(new SafeConstructor());
+        Map<String, Object> yamlConf = (Map<String, Object>)yaml.load(new FileReader(file));
+        return TopologyLoadConf.fromConf(yamlConf);
+    }
+
+    /**
+     * Parse the TopologyLoadConf from a config map.
+     * @param conf the config with the TopologyLoadConf in it
+     * @return the parsed instance.
+     */
+    public static TopologyLoadConf fromConf(Map<String, Object> conf) {
+        Map<String, Object> topoConf = null;
+        if (conf.containsKey("config")) {
+            topoConf = new HashMap<>((Map<String, Object>)conf.get("config"));
+        }
+
+        List<LoadCompConf> spouts = new ArrayList<>();
+        for (Map<String, Object> spoutInfo: (List<Map<String, Object>>) conf.get("spouts")) {
+            spouts.add(LoadCompConf.fromConf(spoutInfo));
+        }
+
+        List<LoadCompConf> bolts = new ArrayList<>();
+        List<Map<String, Object>> boltInfos = (List<Map<String, Object>>) conf.get("bolts");
+        if (boltInfos != null) {
+            for (Map<String, Object> boltInfo : boltInfos) {
+                bolts.add(LoadCompConf.fromConf(boltInfo));
+            }
+        }
+
+        List<InputStream> streams = new ArrayList<>();
+        List<Map<String, Object>> streamInfos = (List<Map<String, Object>>) conf.get("streams");
+        if (streamInfos != null) {
+            for (Map<String, Object> streamInfo: streamInfos) {
+                streams.add(InputStream.fromConf(streamInfo));
+            }
+        }
+
+        return new TopologyLoadConf((String)conf.get("name"), topoConf, spouts, bolts, streams);
+    }
+
+    /**
+     * Write this out to a file in YAML format.
+     * @param file the file to write to.
+     * @throws IOException if there is an error writing to the file.
+     */
+    public void writeTo(File file) throws IOException {
+        Yaml yaml = new Yaml(new SafeConstructor());
+        try (FileWriter writer = new FileWriter(file)) {
+            yaml.dump(toConf(), writer);
+        }
+    }
+
+    /**
+     * Convert this into a YAML String.
+     * @return this as a YAML String.
+     */
+    public String toYamlString() {
+        Yaml yaml = new Yaml(new SafeConstructor());
+        StringWriter writer = new StringWriter();
+        yaml.dump(toConf(), writer);
+        return writer.toString();
+    }
+
+    /**
+     * Covert this into a Map config.
+     * @return this as a Map config.
+     */
+    public Map<String, Object> toConf() {
+        Map<String, Object> ret = new HashMap<>();
+        if (name != null) {
+            ret.put("name", name);
+        }
+        if (topoConf != null) {
+            ret.put("config", topoConf);
+        }
+        if (spouts != null && !spouts.isEmpty()) {
+            ret.put("spouts", spouts.stream().map(LoadCompConf::toConf)
+                .collect(Collectors.toList()));
+        }
+
+        if (bolts != null && !bolts.isEmpty()) {
+            ret.put("bolts", bolts.stream().map(LoadCompConf::toConf)
+                .collect(Collectors.toList()));
+        }
+
+        if (streams != null && !streams.isEmpty()) {
+            ret.put("streams", streams.stream().map(InputStream::toConf)
+                .collect(Collectors.toList()));
+        }
+        return ret;
+    }
+
+    /**
+     * Constructor.
+     * @param name the name of the topology.
+     * @param topoConf the config for the topology
+     * @param spouts the spouts for the topology
+     * @param bolts the bolts for the topology
+     * @param streams the streams for the topology
+     */
+    public TopologyLoadConf(String name, Map<String, Object> topoConf,
+                            List<LoadCompConf> spouts, List<LoadCompConf> bolts, List<InputStream> streams) {
+        this.name = name;
+        this.topoConf = topoConf;
+        this.spouts = spouts;
+        this.bolts = bolts;
+        this.streams = streams;
+    }
+
+    private static String getUniqueTopoName() {
+        return "topology_" + asCharString(topoUniquifier.getAndIncrement());
+    }
+
+    private String getUniqueBoltName() {
+        return "bolt_" + asCharString(boltUniquifier.getAndIncrement());
+    }
+
+    private String getUniqueSpoutName() {
+        return "spout_" + asCharString(spoutUniquifier.getAndIncrement());
+    }
+
+    private String getUniqueStreamName() {
+        return "stream_" + asCharString(spoutUniquifier.getAndIncrement());
+    }
+
+    private static String asCharString(int value) {
+        int div = value / 26;
+        int remainder = value % 26;
+        String ret = "";
+        if (div > 0) {
+            ret = asCharString(div);
+        }
+        ret += (char)((int)'a' + remainder);
+        return ret;
+    }
+
+    public TopologyLoadConf withName(String baseName) {
+        return new TopologyLoadConf(baseName, topoConf, spouts, bolts, streams);
+    }
+
+    /**
+     * Scale all of the components in the topology by a percentage (but keep the throughput the same).
+     * @param v the amount to scale them by.  1.0 is nothing, 0.5 cuts them in half, 2.0 doubles them.
+     * @return a copy of this with the needed adjustments made.
+     */
+    public TopologyLoadConf scaleParallel(double v) {
+        List<LoadCompConf> scaledSpouts = spouts.stream().map((c) -> c.scaleParallel(v)).collect(Collectors.toList());
+        List<LoadCompConf> scaledBolts = bolts.stream().map((c) -> c.scaleParallel(v)).collect(Collectors.toList());
+        return new TopologyLoadConf(name, topoConf, scaledSpouts, scaledBolts, streams);
+    }
+
+    /**
+     * Scale the throughput of the entire topology by a percentage.
+     * @param v the amount to scale it by 1.0 is nothing 0.5 cuts it in half and 2.0 doubles it.
+     * @return a copy of this with the needed adjustments made.
+     */
+    public TopologyLoadConf scaleThroughput(double v) {
+        List<LoadCompConf> scaledSpouts = spouts.stream()
+            .map((c) -> c.scaleThroughput(v)).collect(Collectors.toList());
+        List<LoadCompConf> scaledBolts = bolts.stream()
+            .map((c) -> c.scaleThroughput(v)).collect(Collectors.toList());
+        return new TopologyLoadConf(name, topoConf, scaledSpouts, scaledBolts, streams);
+    }
+
+    /**
+     * Create a new version of this topology with identifiable information removed.
+     * @return the anonymized version of the TopologyLoadConf.
+     */
+    public TopologyLoadConf anonymize() {
+        Map<String, String> remappedComponents = new HashMap<>();
+        Map<GlobalStreamId, GlobalStreamId> remappedStreams = new HashMap<>();
+        for (LoadCompConf comp: bolts) {
+            String newId = getUniqueBoltName();
+            remappedComponents.put(comp.id, newId);
+            if (comp.streams != null) {
+                for (OutputStream out : comp.streams) {
+                    GlobalStreamId orig = new GlobalStreamId(comp.id, out.id);
+                    GlobalStreamId remapped = new GlobalStreamId(newId, getUniqueStreamName());
+                    remappedStreams.put(orig, remapped);
+                }
+            }
+        }
+
+        for (LoadCompConf comp: spouts) {
+            remappedComponents.put(comp.id, getUniqueSpoutName());
+            String newId = getUniqueSpoutName();
+            remappedComponents.put(comp.id, newId);
+            if (comp.streams != null) {
+                for (OutputStream out : comp.streams) {
+                    GlobalStreamId orig = new GlobalStreamId(comp.id, out.id);
+                    GlobalStreamId remapped = new GlobalStreamId(newId, getUniqueStreamName());
+                    remappedStreams.put(orig, remapped);
+                }
+            }
+        }
+
+        for (InputStream in : streams) {
+            if (!remappedComponents.containsKey(in.toComponent)) {
+                remappedComponents.put(in.toComponent, getUniqueSpoutName());
+            }
+            GlobalStreamId orig = in.gsid();
+            if (!remappedStreams.containsKey(orig)) {
+                //Even if the topology is not valid we still need to remap it all
+                String remappedComp = remappedComponents.computeIfAbsent(in.fromComponent, (key) -> {
+                    LOG.warn("stream's {} from is not defined {}", in.id, in.fromComponent);
+                    return getUniqueBoltName();
+                });
+                remappedStreams.put(orig, new GlobalStreamId(remappedComp, getUniqueStreamName()));
+            }
+        }
+
+        //Now we need to map them all back again
+        List<LoadCompConf> remappedSpouts = spouts.stream()
+            .map((orig) -> orig.remap(remappedComponents, remappedStreams))
+            .collect(Collectors.toList());
+        List<LoadCompConf> remappedBolts = bolts.stream()
+            .map((orig) -> orig.remap(remappedComponents, remappedStreams))
+            .collect(Collectors.toList());
+        List<InputStream> remappedInputStreams = streams.stream()
+            .map((orig) -> orig.remap(remappedComponents, remappedStreams))
+            .collect(Collectors.toList());
+        return new TopologyLoadConf(getUniqueTopoName(), anonymizeTopoConf(topoConf), remappedSpouts, remappedBolts, remappedInputStreams);
+    }
+
+    private static Map<String,Object> anonymizeTopoConf(Map<String, Object> topoConf) {
+        //Only keep important conf keys
+        Map<String, Object> ret = new HashMap<>();
+        for (Map.Entry<String, Object> entry: topoConf.entrySet()) {
+            String key = entry.getKey();
+            Object value = entry.getValue();
+            if (IMPORTANT_CONF_KEYS.contains(key)) {
+                if (Config.TOPOLOGY_WORKER_CHILDOPTS.equals(key)
+                    || Config.TOPOLOGY_WORKER_GC_CHILDOPTS.equals(key)) {
+                    value = cleanupChildOpts(value);
+                }
+                ret.put(key, value);
+            }
+        }
+        return ret;
+    }
+
+    private static Object cleanupChildOpts(Object value) {
+        if (value instanceof String) {
+            String sv = (String) value;
+            StringBuffer ret = new StringBuffer();
+            for (String part: sv.split("\\s+")) {
+                if (part.startsWith("-X")) {
+                    ret.append(part).append(" ");
+                }
+            }
+            return ret.toString();
+        } else {
+            List<String> ret = new ArrayList<>();
+            for (String subValue: (Collection<String>)value) {
+                ret.add((String)cleanupChildOpts(subValue));
+            }
+            return ret.stream().filter((item) -> item != null && !item.isEmpty()).collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * Try to see if this looks like a trident topology.
+     * NOTE: this will not work for anonymized configs
+     * @return true if it does else false.
+     */
+    public boolean looksLikeTrident() {
+        for (LoadCompConf spout: spouts) {
+            if (spout.id.startsWith("$mastercoord")) {
+                return true;
+            }
+        }
+
+        for (LoadCompConf bolt: bolts) {
+            if (bolt.id.startsWith("$spoutcoord")) {
+                return true;
+            }
+        }
+
+        for (InputStream in: streams) {
+            if (in.id.equals("$batch")) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Get the messages emitted per second in aggregate across all streams in the topology.
+     * @return messages per second.
+     */
+    public double getAllEmittedAggregate() {
+        double ret = getSpoutEmittedAggregate();
+        for (LoadCompConf bolt: bolts) {
+            ret += bolt.getAllEmittedAggregate() * bolt.parallelism;
+        }
+        return ret;
+    }
+
+    /**
+     * Get the messages emitted per second in aggregate for all of the spouts in the topology.
+     * @return messages per second.
+     */
+    public double getSpoutEmittedAggregate() {
+        double ret = 0;
+        for (LoadCompConf spout: spouts) {
+            ret += spout.getAllEmittedAggregate() * spout.parallelism;
+        }
+        return ret;
+    }
+
+    /**
+     * Try and guess at the actual number of messages emitted per second by a trident topology, not the number of batches.
+     * This does not work on an anonymized conf.
+     * @return messages per second or 0 if this does not look like a trident topology.
+     */
+    public double getTridentEstimatedEmittedAggregate() {
+        //In this case we are ignoring the coord stuff, and only looking at
+        double ret = 0;
+        if (looksLikeTrident()) {
+            List<LoadCompConf> all = new ArrayList<>(bolts);
+            all.addAll(spouts);
+            for (LoadCompConf comp : all) {
+                if (comp.id.startsWith("spout-")) {
+                    if (comp.streams != null) {
+                        for (OutputStream out: comp.streams) {
+                            if (!out.id.startsWith("$")
+                                && !out.id.startsWith("__")
+                                && out.rate != null) {
+                                ret += out.rate.mean * comp.parallelism;
+                            }
+                        }
+                    }
+                }
+            }
+        }
+        return ret;
+    }
+
+    public TopologyLoadConf replaceShuffleWithLocalOrShuffle() {
+        List<InputStream> modified = streams.stream().map((in) -> in.replaceShuffleWithLocalOrShuffle()).collect(Collectors.toList());
+        return new TopologyLoadConf(name, topoConf, spouts, bolts, modified);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/LoadCompConfTest.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/LoadCompConfTest.java b/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/LoadCompConfTest.java
new file mode 100644
index 0000000..27c7389
--- /dev/null
+++ b/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/LoadCompConfTest.java
@@ -0,0 +1,57 @@
+/**
+ * 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.loadgen;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class LoadCompConfTest {
+    @Test
+    public void scaleParallel() throws Exception {
+        LoadCompConf orig = new LoadCompConf.Builder()
+            .withId("SOME_SPOUT")
+            .withParallelism(1)
+            .withStream(new OutputStream("default", new NormalDistStats(500.0, 100.0, 300.0, 600.0), false))
+            .build();
+        assertEquals(500.0, orig.getAllEmittedAggregate(), 0.001);
+        LoadCompConf scaled = orig.scaleParallel(2);
+        //Parallelism is double
+        assertEquals(2, scaled.parallelism);
+        assertEquals("SOME_SPOUT", scaled.id);
+        //But throughput is the same
+        assertEquals(500.0, scaled.getAllEmittedAggregate(), 0.001);
+    }
+
+    @Test
+    public void scaleThroughput() throws Exception {
+        LoadCompConf orig = new LoadCompConf.Builder()
+            .withId("SOME_SPOUT")
+            .withParallelism(1)
+            .withStream(new OutputStream("default", new NormalDistStats(500.0, 100.0, 300.0, 600.0), false))
+            .build();
+        assertEquals(500.0, orig.getAllEmittedAggregate(), 0.001);
+        LoadCompConf scaled = orig.scaleThroughput(2.0);
+        //Parallelism is same
+        assertEquals(1, scaled.parallelism);
+        assertEquals("SOME_SPOUT", scaled.id);
+        //But throughput is the same
+        assertEquals(1000.0, scaled.getAllEmittedAggregate(), 0.001);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/LoadMetricsServerTest.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/LoadMetricsServerTest.java b/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/LoadMetricsServerTest.java
new file mode 100644
index 0000000..ca58d7e
--- /dev/null
+++ b/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/LoadMetricsServerTest.java
@@ -0,0 +1,36 @@
+/**
+ * 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.loadgen;
+
+import java.util.concurrent.TimeUnit;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+import static org.apache.storm.loadgen.LoadMetricsServer.convert;
+
+public class LoadMetricsServerTest {
+    @Test
+    public void convertTest() throws Exception {
+        for (TimeUnit from : TimeUnit.values()) {
+            for (TimeUnit to : TimeUnit.values()) {
+                assertEquals(from + " to " + to + " and back", 1.0, convert(convert(1.0, from, to), to, from), 0.0001);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/NormalDistStatsTest.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/NormalDistStatsTest.java b/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/NormalDistStatsTest.java
new file mode 100644
index 0000000..10cc18b
--- /dev/null
+++ b/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/NormalDistStatsTest.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.loadgen;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class NormalDistStatsTest {
+    public static void assertNDSEquals(NormalDistStats a, NormalDistStats b) {
+        assertEquals("mean", a.mean, b.mean, 0.0001);
+        assertEquals("min", a.min, b.min, 0.0001);
+        assertEquals("max", a.max, b.max, 0.0001);
+        assertEquals("stddev", a.stddev, b.stddev, 0.0001);
+    }
+
+    @Test
+    public void scaleBy() throws Exception {
+        NormalDistStats orig = new NormalDistStats(1.0, 0.5, 0.0, 2.0);
+        assertNDSEquals(orig, orig.scaleBy(1.0));
+        NormalDistStats expectedDouble = new NormalDistStats(2.0, 0.5, 1.0, 3.0);
+        assertNDSEquals(expectedDouble, orig.scaleBy(2.0));
+        NormalDistStats expectedHalf = new NormalDistStats(0.5, 0.5, 0.0, 1.5);
+        assertNDSEquals(expectedHalf, orig.scaleBy(0.5));
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/OutputStreamTest.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/OutputStreamTest.java b/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/OutputStreamTest.java
new file mode 100644
index 0000000..9200104
--- /dev/null
+++ b/examples/storm-loadgen/src/test/java/org/apache/storm/loadgen/OutputStreamTest.java
@@ -0,0 +1,37 @@
+/**
+ * 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.loadgen;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class OutputStreamTest {
+    @Test
+    public void scaleThroughput() throws Exception {
+        OutputStream orig = new OutputStream("ID", new NormalDistStats(100.0, 1.0, 99.0, 101.0), false);
+        OutputStream scaled = orig.scaleThroughput(2.0);
+        assertEquals(orig.id, scaled.id);
+        assertEquals(orig.areKeysSkewed, scaled.areKeysSkewed);
+        assertEquals(scaled.rate.mean, 200.0, 0.0001);
+        assertEquals(scaled.rate.stddev, 1.0, 0.0001);
+        assertEquals(scaled.rate.min, 199.0, 0.0001);
+        assertEquals(scaled.rate.max, 201.0, 0.0001);
+    }
+}
\ No newline at end of file


[12/18] storm git commit: Addressed review comments

Posted by bo...@apache.org.
Addressed review comments


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/381387b2
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/381387b2
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/381387b2

Branch: refs/heads/master
Commit: 381387b2c89f11cc6206219634af53df0769850e
Parents: 0ef492e
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Aug 25 12:10:41 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Aug 25 12:10:41 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                      |  6 +++---
 .../java/org/apache/storm/loadgen/CaptureLoad.java    | 14 ++++++--------
 .../org/apache/storm/loadgen/EstimateThroughput.java  | 10 ++++------
 .../org/apache/storm/loadgen/TopologyLoadConf.java    |  4 ++--
 4 files changed, 15 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/381387b2/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index 29d27ab..52f5358 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -5,14 +5,14 @@ A set of tools to place an artificial load on a storm cluster to compare against
 ## Methodology
 The idea behind all of these tools is to measure the trade-offs between latency, throughput, and cost when processing data using Apache Storm.
 
-When processing data you typically will know a few things.  First you will know about how much data you are going to be processing.  This will typically be a range of values that change throughput the day.  You also will have an idea of how quickly you need the data processed by.  Often this is measured in terms of the latency it takes to process data at the some percentile or set of percentiles.  This is because of most use cases the value of the data declines over time, and being able to react to the data quickly is more valuable.  You probably also have a budget for how much you are willing to spend to be able to process this data.  There are always trade-offs in how quickly you can process some data and how efficiently you can processes that data both in terms of resource usage (cost) and latency.  These tools are designed to help you explore that space.
+When processing data you typically will know a few things.  First you will know about how much data you are going to be processing.  This will typically be a range of values that change throughput the day.  You also will have an idea of how quickly you need the data processed by.  Often this is measured in terms of the latency it takes to process data at some percentile or set of percentiles.  This is because in most use cases the value of the data declines over time, and being able to react to the data quickly is more valuable.  You probably also have a budget for how much you are willing to spend to be able to process this data.  There are always trade-offs in how quickly you can process some data and how efficiently you can processes that data both in terms of resource usage (cost) and latency.  These tools are designed to help you explore that space.
 
 A note on how latency is measured.  Storm typically measures latency from when a message is emitted by a spout until the point it is fully acked or failed (in many versions of storm it actually does this in the acker instead of the spout so it is trying to be a measure of how long it takes for the actual processing, removing as much of the acker overhead as possible).  For these tools we do it differently.  We simulate a throughput and measure the start time of the tuple from when it would have been emitted if the topology could keep up with the load.  In the normal case this should not be an issue, but if the topology cannot keep up with the throughput you will see the latency grow very high compared to the latency reported by storm.
 
 ## Tools
 ### CaptureLoad 
 
-`CaptureLoad` will look at the topologies on a running cluster and store the structure of and metrics about each of theses topologies storing them in a format that can be used later to reproduce a similar load on the cluster.
+`CaptureLoad` will look at the topologies on a running cluster and store the structure of and metrics about each in a format described below that can be used later to reproduce a similar load on the cluster.
 
 #### Usage
 ```
@@ -49,7 +49,7 @@ storm jar storm-loadgen.jar org.apache.storm.loadgen.GenLoad [options] [capture_
 | -w,--report-window &lt;INTERVAL_SECS> | How long of a rolling window should be in each report.  Will be rounded up to the next report interval boundary. default 30|
 
 ## ThroughputVsLatency
-This is a topology similar to `GenLoad` in most ways, except instead of simulating a load it runs a word count algorithm.
+A word count topology with metrics reporting like the `GenLoad` command.
 
 ### Usage
 ```

http://git-wip-us.apache.org/repos/asf/storm/blob/381387b2/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
index 649a4c0..11d64a1 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.function.Function;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.DefaultParser;
@@ -76,8 +77,7 @@ public class CaptureLoad {
                         List<Double> subvalues = data.values().stream()
                             .map((subMap) -> subMap.get(id))
                             .filter((value) -> value != null)
-                            .mapToDouble((value) -> value.doubleValue())
-                            .boxed().collect(Collectors.toList());
+                            .collect(Collectors.toList());
                         ret.addAll(subvalues);
                     }
                 }
@@ -325,16 +325,14 @@ public class CaptureLoad {
             .build());
         CommandLineParser parser = new DefaultParser();
         CommandLine cmd = null;
-        ParseException pe = null;
+        boolean printHelp = false;
         try {
             cmd = parser.parse(options, args);
         } catch (ParseException e) {
-            pe = e;
+            System.err.println("ERROR " + e.getMessage());
+            printHelp = true;
         }
-        if (pe != null || cmd.hasOption('h')) {
-            if (pe != null) {
-                System.err.println("ERROR " + pe.getMessage());
-            }
+        if (printHelp || cmd.hasOption('h')) {
             new HelpFormatter().printHelp("CaptureLoad [options] [topologyName]*", options);
             return;
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/381387b2/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/EstimateThroughput.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/EstimateThroughput.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/EstimateThroughput.java
index 80ede37..fc0b4c7 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/EstimateThroughput.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/EstimateThroughput.java
@@ -55,16 +55,14 @@ public class EstimateThroughput {
             .build());
         CommandLineParser parser = new DefaultParser();
         CommandLine cmd = null;
-        ParseException pe = null;
+        boolean printHelp = false;
         try {
             cmd = parser.parse(options, args);
         } catch (ParseException e) {
-            pe = e;
+            System.err.println("ERROR " + e.getMessage());
+            printHelp = true;
         }
-        if (pe != null || cmd.hasOption('h')) {
-            if (pe != null) {
-                System.err.println("ERROR " + pe.getMessage());
-            }
+        if (printHelp || cmd.hasOption('h')) {
             new HelpFormatter().printHelp("EstimateThroughput [options] [topologyName]*", options);
             return;
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/381387b2/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
index 38458c6..b171972 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
@@ -427,7 +427,7 @@ public class TopologyLoadConf {
     public double getAllEmittedAggregate() {
         double ret = getSpoutEmittedAggregate();
         for (LoadCompConf bolt: bolts) {
-            ret += bolt.getAllEmittedAggregate() * bolt.parallelism;
+            ret += bolt.getAllEmittedAggregate();
         }
         return ret;
     }
@@ -439,7 +439,7 @@ public class TopologyLoadConf {
     public double getSpoutEmittedAggregate() {
         double ret = 0;
         for (LoadCompConf spout: spouts) {
-            ret += spout.getAllEmittedAggregate() * spout.parallelism;
+            ret += spout.getAllEmittedAggregate();
         }
         return ret;
     }


[05/18] storm git commit: STORM-2702: fixed a bug and added in more metrics

Posted by bo...@apache.org.
STORM-2702: fixed a bug and added in more metrics


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/211f8a94
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/211f8a94
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/211f8a94

Branch: refs/heads/master
Commit: 211f8a944c096b04083c211306687daeb16c7adc
Parents: 6c2dcbe
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Aug 23 12:49:41 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Aug 23 14:30:38 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                |  73 ++++++---
 .../java/org/apache/storm/loadgen/GenLoad.java  |   8 +-
 .../apache/storm/loadgen/LoadMetricsServer.java | 157 ++++++++++++++-----
 .../storm/loadgen/ThroughputVsLatency.java      |   9 +-
 4 files changed, 182 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/211f8a94/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index 8414768..e91409c 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -81,36 +81,57 @@ Not all options are supported by all reporters.
 |Reporter Option| Description | Supported Reporters|
 |---------------|-------------|--------------------|
 |time | Set the time unit that you want latency and CPU reported in.  This can be from nanoseconds up to seconds.  Most names are supported for the types| legacy, csv, tsv|
-|columns | A comma separated list of columns to output (see below for the metrics supported).  Defaults to "start_time", "end_time", "completion_rate", "mean", "99%ile", "99.9%ile", "cores", "mem", "failed" | csv, tsv |
-|extraColumns | Like columns but ones that should be added to the defaults instead of replacing them (this is mostly for convenience) | csv, tsv |
+|columns | A comma separated list of columns to output (see below for the metrics supported).  A `*` is replaced by all metrics. Defaults to "start_time", "end_time", "completion_rate", "mean", "99%ile", "99.9%ile", "cores", "mem", "failed" | csv, tsv |
+|extraColumns | Like columns but ones that should be added to the defaults instead of replacing them. A `*` is replaced by all metrics. | csv, tsv |
 |meta | An arbitrary string that will appear as a "meta" column at the end.  This helps when appending to files to keep different runs separated | csv, tsv|
 
 There are a lot of different metrics supported
 
-|Metrics Name| Description|
-|------------|------------|
-|99%ile| 99th percentile completion latency. |
-|99.9%ile| 99.9th percentile completion latency. |
-|median| Median completion latency. |
-|mean| Mean completion latency. |
-|min| Minimum completion latency. |
-|max| Maximum completion latency. |
-|stddev| Standard Deviation of completion latency. |
-|user_cpu| User space CPU time.|
-|sys_cpu| System space CPU time. |
-|gc_cpu| Amount of CPU time spent in GC as reported by the JVM. |
-|cores| The number of CPU cores used. `(user_cpu + sys_cpu) / time_window`|
-|uptime| The amount of time the oldest topology has been up for. |
-|acked| The number of tuples fully acked as reported by Storm's metrics. |
-|rate| The rate of tuples fully acked as reported by Storm's metrics. |
-|completed| The number of tuples fully acked as reported by the latency histogram metrics. |
-|completion_rate| The rate of tuples fully acked as reported by the latency histogram metrics. |
-|mem| The amount of memory used by the topology in MB, as reported by the JVM. |
-|failed| The number of failed tuples as reported by Storm's metrics. |
-|start_time| The starting time of the metrics window from when the first topology was launched.
-|end_time| The ending time of the metrics window from the the first topology was launched.
-|time_window| the length in seconds for the time window. |
-|ids| The topology ids that are being tracked |
+|Metrics Name| Description| In |
+|------------|------------|----|
+|99%ile| 99th percentile completion latency. | all
+|99.9%ile| 99.9th percentile completion latency. | all
+|median| Median completion latency. | all
+|mean| Mean completion latency. | all
+|min| Minimum completion latency. | all
+|max| Maximum completion latency. | all
+|stddev| Standard Deviation of completion latency. | all
+|user_cpu| User space CPU time.| all
+|sys_cpu| System space CPU time. | all
+|gc_cpu| Amount of CPU time spent in GC as reported by the JVM. | all
+|cores| The number of CPU cores used. `(user_cpu + sys_cpu) / time_window`| all
+|uptime| The amount of time the oldest topology has been up for. | all
+|acked| The number of tuples fully acked as reported by Storm's metrics. | all
+|acked_rate| The rate of tuples fully acked as reported by Storm's metrics. | all
+|completed| The number of tuples fully acked as reported by the latency histogram metrics. | all
+|completion_rate| The rate of tuples fully acked as reported by the latency histogram metrics. | all
+|mem| The amount of memory used by the topology in MB, as reported by the JVM. | all
+|failed| The number of failed tuples as reported by Storm's metrics. | all
+|start_time| The starting time of the metrics window from when the first topology was launched. | all
+|end_time| The ending time of the metrics window from the the first topology was launched. | all
+|time_window| the length in seconds for the time window. | all
+|ids| The topology ids that are being tracked | all
+|storm_version| The version of storm as reported by the client | all
+|java_version| The version of java as reported by the client | all
+|os_arch| The OS architecture as reported by the client | all
+|os_name| The name of the OS as reported by the client | all
+|os_version| The version of the OS as reported by the client | all
+|config_override| And command line overrides to storm config values | all
+|hosts| The number of hosts the monitored topologies are running on| all
+|executors| The number of running executors in the monitored topologies | all
+|workers| The number of workers the monitored topologies are running on | all
+|target_rate| The target rate in sentenses per second for the ThroughputVsLatency topology | ThroughputVsLatency
+|spout_parallel| The parallelism of the spout for the `ThroughputVsLatency` topology. | ThroughputVsLatency
+|split_parallel| The parallelism of the split bolt for the `ThroughputVsLatency` topology. | ThroughputVsLatency
+|count_parallel| The parallelism of the count bolt for the `ThroughputVsLatency` topology. | ThroughputVsLatency
+|parallel\_adjust| The adjustment to the parallelism in `GenLoad`. | GenLoad
+|throughput_adjust| The adjustment to the throughput in `GenLoad`. | GenLoad
+|local\_or\_shuffle| true if shuffles were replaced with local or shuffle in GenLoad. | GenLoad
+
+There are also some generic rules that you can use for some metrics.  Any metric that starts with `"conf:"` will be the config for that.  It does not include config overrides from the `GenLoad` file.
+
+In addition any metric that ends with `"%ile"` will be the latency at that percentile.
+
 
 # Captured Load File Format
 The file format used with `CaptureLoad` and `GenLoad` is based off of the flux file format, but with some extensions and omissions.

http://git-wip-us.apache.org/repos/asf/storm/blob/211f8a94/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
index 7998fdc..8821b2a 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
@@ -21,6 +21,7 @@ package org.apache.storm.loadgen;
 import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.Map;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -117,8 +118,13 @@ public class GenLoad {
             new HelpFormatter().printHelp("GenLoad [options] [captured_file]*", options);
             return;
         }
+        Map<String, Object> metrics = new LinkedHashMap<>();
+        metrics.put("parallel_adjust", parallel);
+        metrics.put("throughput_adjust", throughput);
+        metrics.put("local_or_shuffle", cmd.hasOption("local-or-shuffle"));
+
         Config conf = new Config();
-        LoadMetricsServer metricServer = new LoadMetricsServer(conf, cmd);
+        LoadMetricsServer metricServer = new LoadMetricsServer(conf, cmd, metrics);
 
         metricServer.serve();
         String url = metricServer.getUrl();

http://git-wip-us.apache.org/repos/asf/storm/blob/211f8a94/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
index c126ca5..fd12247 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -29,6 +29,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -50,6 +51,8 @@ import org.apache.storm.generated.SpoutStats;
 import org.apache.storm.generated.TopologyInfo;
 import org.apache.storm.generated.TopologySummary;
 import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.VersionInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -96,6 +99,9 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         private long acked;
         private long failed;
         private Set<String> topologyIds;
+        private long workers;
+        private long executors;
+        private long hosts;
 
         /**
          * Constructor.
@@ -105,7 +111,8 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
          * @param gcMs GC CPU in ms.
          */
         public Measurements(long uptimeSecs, long acked, long timeWindow, long failed, Histogram histo,
-                            double userMs, double sysMs, double gcMs, long memBytes, Set<String> topologyIds) {
+                            double userMs, double sysMs, double gcMs, long memBytes, Set<String> topologyIds,
+                            long workers, long executors, long hosts) {
             this.uptimeSecs = uptimeSecs;
             this.acked = acked;
             this.timeWindow = timeWindow;
@@ -116,6 +123,9 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             this.histo = histo;
             this.memBytes = memBytes;
             this.topologyIds = topologyIds;
+            this.workers = workers;
+            this.executors = executors;
+            this.hosts = hosts;
         }
 
         /**
@@ -132,6 +142,9 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             acked = 0;
             failed = 0;
             topologyIds = new HashSet<>();
+            workers = 0;
+            executors = 0;
+            hosts = 0;
         }
 
         /**
@@ -149,6 +162,9 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             uptimeSecs = Math.max(uptimeSecs, other.uptimeSecs);
             timeWindow += other.timeWindow;
             topologyIds.addAll(other.topologyIds);
+            workers = Math.max(workers, other.workers);
+            executors = Math.max(executors, other.executors);
+            hosts = Math.max(hosts, other.hosts);
         }
 
         public double getLatencyAtPercentile(double percential, TimeUnit unit) {
@@ -227,6 +243,18 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             return topologyIds;
         }
 
+        public long getWorkers() {
+            return workers;
+        }
+
+        public long getHosts() {
+            return hosts;
+        }
+
+        public long getExecutors() {
+            return executors;
+        }
+
         static Measurements combine(List<Measurements> measurements, Integer start, Integer count) {
             if (count == null) {
                 count = measurements.size();
@@ -257,12 +285,14 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
     abstract static class FileReporter implements MetricResultsReporter {
         protected final PrintStream out;
         private final boolean needsClose;
+        protected final Map<String, MetricExtractor> allExtractors;
 
-        public FileReporter() throws FileNotFoundException {
-            this(null, null);
+        public FileReporter(Map<String, MetricExtractor> allExtractors) throws FileNotFoundException {
+            this(null, Collections.emptyMap(), allExtractors);
         }
 
-        public FileReporter(String path, Map<String, String> query) throws FileNotFoundException {
+        public FileReporter(String path, Map<String, String> query,  Map<String, MetricExtractor> allExtractors)
+            throws FileNotFoundException {
             boolean append = Boolean.parseBoolean(query.getOrDefault("append", "false"));
 
             if (path == null || "/dev/stdout".equals(path)) {
@@ -275,6 +305,8 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                 out = new PrintStream(new FileOutputStream(path, append));
                 needsClose = true;
             }
+            //Copy it in case we want to modify it
+            this.allExtractors = new LinkedHashMap<>(allExtractors);
         }
 
         @Override
@@ -337,31 +369,42 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
 
     static {
         //Perhaps there is a better way to do this???
-        HashMap<String, MetricExtractor> tmp = new HashMap<>();
+        LinkedHashMap<String, MetricExtractor> tmp = new LinkedHashMap<>();
+        tmp.put("start_time",  new MetricExtractor((m, unit) -> m.startTime(),"s"));
+        tmp.put("end_time",  new MetricExtractor((m, unit) -> m.endTime(), "s"));
+        tmp.put("completion_rate",  new MetricExtractor((m, unit) -> m.getCompletedPerSec(), "tuple/s"));
+        tmp.put("mean", new MetricExtractor((m, unit) -> m.getMeanLatency(unit)));
         tmp.put("99%ile", new MetricExtractor((m, unit) -> m.getLatencyAtPercentile(99.0, unit)));
         tmp.put("99.9%ile", new MetricExtractor((m, unit) -> m.getLatencyAtPercentile(99.9, unit)));
+        tmp.put("cores", new MetricExtractor(
+            (m, unit) -> (m.getSysTime(TimeUnit.SECONDS) + m.getUserTime(TimeUnit.SECONDS)) / m.getTimeWindow(),
+            ""));
+        tmp.put("mem",  new MetricExtractor((m, unit) -> m.getMemMb(), "MB"));
+        tmp.put("failed",  new MetricExtractor((m, unit) -> m.getFailed(), ""));
         tmp.put("median", new MetricExtractor((m, unit) -> m.getLatencyAtPercentile(50, unit)));
-        tmp.put("mean", new MetricExtractor((m, unit) -> m.getMeanLatency(unit)));
         tmp.put("min", new MetricExtractor((m, unit) -> m.getMinLatency(unit)));
         tmp.put("max", new MetricExtractor((m, unit) -> m.getMaxLatency(unit)));
         tmp.put("stddev", new MetricExtractor((m, unit) -> m.getLatencyStdDeviation(unit)));
         tmp.put("user_cpu", new MetricExtractor((m, unit) -> m.getUserTime(unit)));
         tmp.put("sys_cpu", new MetricExtractor((m, unit) -> m.getSysTime(unit)));
         tmp.put("gc_cpu", new MetricExtractor((m, unit) -> m.getGc(unit)));
-        tmp.put("cores", new MetricExtractor(
-            (m, unit) -> (m.getSysTime(TimeUnit.SECONDS) + m.getUserTime(TimeUnit.SECONDS)) / m.getTimeWindow(),
-            ""));
-        tmp.put("uptime",  new MetricExtractor((m, unit) -> m.getUptimeSecs(), "s"));
         tmp.put("acked",  new MetricExtractor((m, unit) -> m.getAcked(), ""));
-        tmp.put("rate",  new MetricExtractor((m, unit) -> m.getAckedPerSec(), "tuple/s"));
+        tmp.put("acked_rate",  new MetricExtractor((m, unit) -> m.getAckedPerSec(), "tuple/s"));
         tmp.put("completed",  new MetricExtractor((m, unit) -> m.getCompleted(), ""));
-        tmp.put("completion_rate",  new MetricExtractor((m, unit) -> m.getCompletedPerSec(), "tuple/s"));
-        tmp.put("mem",  new MetricExtractor((m, unit) -> m.getMemMb(), "MB"));
-        tmp.put("failed",  new MetricExtractor((m, unit) -> m.getFailed(), ""));
-        tmp.put("start_time",  new MetricExtractor((m, unit) -> m.startTime(),"s"));
-        tmp.put("end_time",  new MetricExtractor((m, unit) -> m.endTime(), "s"));
+        tmp.put("uptime",  new MetricExtractor((m, unit) -> m.getUptimeSecs(), "s"));
         tmp.put("time_window",  new MetricExtractor((m, unit) -> m.getTimeWindow(), "s"));
         tmp.put("ids",  new MetricExtractor((m, unit) -> m.getTopologyIds(), ""));
+        tmp.put("workers",  new MetricExtractor((m, unit) -> m.getWorkers(), ""));
+        tmp.put("hosts",  new MetricExtractor((m, unit) -> m.getHosts(), ""));
+        tmp.put("executors",  new MetricExtractor((m, unit) -> m.getExecutors(), ""));
+        String buildVersion = VersionInfo.getBuildVersion();
+        tmp.put("storm_version", new MetricExtractor((m, unit) -> buildVersion, ""));
+        tmp.put("java_version", new MetricExtractor((m, unit) -> System.getProperty("java.vendor")
+            + " " + System.getProperty("java.version"),""));
+        tmp.put("os_arch", new MetricExtractor((m, unit) -> System.getProperty("os.arch"), ""));
+        tmp.put("os_name", new MetricExtractor((m, unit) -> System.getProperty("os.name"), ""));
+        tmp.put("os_version", new MetricExtractor((m, unit) -> System.getProperty("os.version"), ""));
+        tmp.put("config_override", new MetricExtractor((m, unit) -> Utils.readCommandLineOpts(), ""));
         NAMED_EXTRACTORS = Collections.unmodifiableMap(tmp);
     }
 
@@ -405,20 +448,23 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         private final List<String> extractors;
         private final String meta;
 
-        public SepValReporter(String separator, String path, Map<String, String> query) throws FileNotFoundException {
-            super(path, query);
+        public SepValReporter(String separator, String path, Map<String, String> query, Map<String, MetricExtractor> extractorsMap)
+            throws FileNotFoundException {
+            super(path, query, extractorsMap);
             this.separator = separator;
             targetUnit = UNIT_MAP.get(query.getOrDefault("time", "MILLISECONDS").toUpperCase());
             if (targetUnit == null) {
                 throw new IllegalArgumentException(query.get("time") + " is not a supported time unit");
             }
             if (query.containsKey("columns")) {
-                extractors = Arrays.asList(query.get("columns").split("\\s*,\\s*"));
+                List<String> extractors = handleExtractorCleanup(Arrays.asList(query.get("columns").split("\\s*,\\s*")));
+
                 HashSet<String> notFound = new HashSet<>(extractors);
-                notFound.removeAll(NAMED_EXTRACTORS.keySet());
+                notFound.removeAll(allExtractors.keySet());
                 if (notFound.size() > 0) {
                     throw new IllegalArgumentException(notFound + " columns are not supported");
                 }
+                this.extractors = extractors;
             } else {
                 //Wrapping it makes it mutable
                 extractors = new ArrayList<>(Arrays.asList("start_time", "end_time", "completion_rate",
@@ -426,9 +472,10 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             }
 
             if (query.containsKey("extraColumns")) {
-                List<String> moreExtractors = Arrays.asList(query.get("extraColumns").split("\\s*,\\s*"));
+                List<String> moreExtractors =
+                    handleExtractorCleanup(Arrays.asList(query.get("extraColumns").split("\\s*,\\s*")));
                 for (String extractor: moreExtractors) {
-                    if (!NAMED_EXTRACTORS.containsKey(extractor)) {
+                    if (!allExtractors.containsKey(extractor)) {
                         throw new IllegalArgumentException(extractor + " is not a supported column");
                     }
                     if (!extractors.contains(extractor)) {
@@ -440,6 +487,28 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             meta = query.get("meta");
         }
 
+        private List<String> handleExtractorCleanup(List<String> orig) {
+            Map<String, Object> stormConfig = Utils.readStormConfig();
+            List<String> ret = new ArrayList<>(orig.size());
+            for (String extractor: orig) {
+                if (extractor.startsWith("conf:")) {
+                    String confKey = extractor.substring("conf:".length());
+                    Object confValue = stormConfig.get(confKey);
+                    allExtractors.put(extractor, new MetricExtractor((m, t) -> confValue, ""));
+                    ret.add(extractor);
+                } else if (extractor.endsWith("%ile")) {
+                    double number = Double.valueOf(extractor.substring(0, extractor.length() - "%ile".length()));
+                    allExtractors.put(extractor, new MetricExtractor((m, t) -> m.getLatencyAtPercentile(number, t)));
+                    ret.add(extractor);
+                } else if ("*".equals(extractor)) {
+                    ret.addAll(allExtractors.keySet());
+                } else {
+                    ret.add(extractor);
+                }
+            }
+            return ret;
+        }
+
         @Override
         public void start() {
             boolean first = true;
@@ -448,7 +517,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                     out.print(separator);
                 }
                 first = false;
-                out.print(NAMED_EXTRACTORS.get(name).formatName(name, targetUnit));
+                out.print(allExtractors.get(name).formatName(name, targetUnit));
             }
             if (meta != null) {
                 out.print(separator);
@@ -465,7 +534,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                     out.print(separator);
                 }
                 first = false;
-                Object value = NAMED_EXTRACTORS.get(name).get(m, targetUnit);
+                Object value = allExtractors.get(name).get(m, targetUnit);
                 String svalue = value == null ? "" : value.toString();
                 out.print(escape(svalue));
             }
@@ -484,13 +553,14 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
     static class LegacyReporter extends FileReporter {
         private final TimeUnit targetUnitOverride;
 
-        public LegacyReporter() throws FileNotFoundException {
-            super();
+        public LegacyReporter(Map<String, MetricExtractor> allExtractors) throws FileNotFoundException {
+            super(allExtractors);
             targetUnitOverride = null;
         }
 
-        public LegacyReporter(String path, Map<String, String> query) throws FileNotFoundException {
-            super(path, query);
+        public LegacyReporter(String path, Map<String, String> query, Map<String, MetricExtractor> allExtractors)
+            throws FileNotFoundException {
+            super(path, query, allExtractors);
             if (query.containsKey("time")) {
                 targetUnitOverride = UNIT_MAP.get(query.get("time").toUpperCase());
                 if (targetUnitOverride == null) {
@@ -540,7 +610,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         options.addOption(Option.builder("r")
             .longOpt("report-interval")
             .hasArg()
-            .argName("INTERVAL_SECS")
+            .argName("SECS")
             .desc("How long in between reported metrics.  Will be rounded up to the next 10 sec boundary.\n"
                 + "default " + DEFAULT_REPORT_INTERVAL)
             .build());
@@ -548,7 +618,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         options.addOption(Option.builder("w")
             .longOpt("report-window")
             .hasArg()
-            .argName("INTERVAL_SECS")
+            .argName("SECS")
             .desc("How long of a rolling window should be in each report.  Will be rounded up to the next report interval boundary.\n"
                 + "default " + DEFAULT_WINDOW_INTERVAL)
             .build());
@@ -585,8 +655,14 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
 
     private final LinkedList<Measurements> allCombined = new LinkedList<>();
 
-    LoadMetricsServer(Map<String, Object> conf, CommandLine commandLine) throws URISyntaxException, FileNotFoundException {
+    LoadMetricsServer(Map<String, Object> conf, CommandLine commandLine, Map<String, Object> parameterMetrics) throws URISyntaxException,
+        FileNotFoundException {
         super(conf);
+        Map<String, MetricExtractor> allExtractors = new LinkedHashMap<>(NAMED_EXTRACTORS);
+        for (Map.Entry<String, Object> entry: parameterMetrics.entrySet()) {
+            final Object value = entry.getValue();
+            allExtractors.put(entry.getKey(), new MetricExtractor((m, unit) -> value, ""));
+        }
         if (commandLine.hasOption("r")) {
             reportIntervalSecs = Long.parseLong(commandLine.getOptionValue("r"));
             reportIntervalSecs = ((reportIntervalSecs + 1) / 10) * 10;
@@ -600,7 +676,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             for (String reporterString: commandLine.getOptionValues("reporter")) {
                 Matcher m = REPORTER_PATTERN.matcher(reporterString);
                 if (!m.matches()) {
-                    throw new IllegalArgumentException(reporterString + " does nto look like it is a reporter");
+                    throw new IllegalArgumentException(reporterString + " does not look like it is a reporter");
                 }
                 String type = m.group("type");
                 String path = m.group("path");
@@ -617,20 +693,20 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                 type = type.toUpperCase();
                 switch (type) {
                     case "LEGACY":
-                        reporters.add(new LegacyReporter(path, query));
+                        reporters.add(new LegacyReporter(path, query, allExtractors));
                         break;
                     case "TSV":
-                        reporters.add(new SepValReporter("\t", path, query));
+                        reporters.add(new SepValReporter("\t", path, query, allExtractors));
                         break;
                     case "CSV":
-                        reporters.add(new SepValReporter(",", path, query));
+                        reporters.add(new SepValReporter(",", path, query, allExtractors));
                         break;
                     default:
                         throw new RuntimeException(type + " is not a supported reporter type");
                 }
             }
         } else {
-            reporters.add(new LegacyReporter());
+            reporters.add(new LegacyReporter(allExtractors));
         }
     }
 
@@ -682,6 +758,9 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         if (ids.size() != names.size()) {
             throw new Exception("Could not find all topologies: " + names);
         }
+        HashSet<String> workers = new HashSet<>();
+        HashSet<String> hosts = new HashSet<>();
+        int executors = 0;
         int uptime = 0;
         long acked = 0;
         long failed = 0;
@@ -689,6 +768,9 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             TopologyInfo info = client.getTopologyInfo(id);
             uptime = Math.max(uptime, info.get_uptime_secs());
             for (ExecutorSummary exec : info.get_executors()) {
+                hosts.add(exec.get_host());
+                workers.add(exec.get_host() + exec.get_port());
+                executors++;
                 if (exec.get_stats() != null && exec.get_stats().get_specific() != null
                     && exec.get_stats().get_specific().is_set_spout()) {
                     SpoutStats stats = exec.get_stats().get_specific().get_spout();
@@ -729,7 +811,8 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         long gc = gcMs.getAndSet(0);
         long memBytes = readMemory();
 
-        allCombined.add(new Measurements(uptime, ackedThisTime, thisTime, failedThisTime, copy, user, sys, gc, memBytes, ids));
+        allCombined.add(new Measurements(uptime, ackedThisTime, thisTime, failedThisTime, copy, user, sys, gc, memBytes,
+            ids, workers.size(), executors, hosts.size()));
         Measurements inWindow = Measurements.combine(allCombined, null, windowLength);
         for (MetricResultsReporter reporter: reporters) {
             reporter.reportWindow(inWindow, allCombined);

http://git-wip-us.apache.org/repos/asf/storm/blob/211f8a94/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
index 2c22b42..3d3a271 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
@@ -19,6 +19,7 @@
 package org.apache.storm.loadgen;
 
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.Map;
 
 import org.apache.commons.cli.CommandLine;
@@ -208,8 +209,14 @@ public class ThroughputVsLatency {
             return;
         }
 
+        Map<String, Object> metrics = new LinkedHashMap<>();
+        metrics.put("target_rate", ratePerSecond);
+        metrics.put("spout_parallel", numSpouts);
+        metrics.put("split_parallel", numSplits);
+        metrics.put("count_parallel", numCounts);
+
         Config conf = new Config();
-        LoadMetricsServer metricServer = new LoadMetricsServer(conf, cmd);
+        LoadMetricsServer metricServer = new LoadMetricsServer(conf, cmd, metrics);
         metricServer.serve();
         String url = metricServer.getUrl();
 


[10/18] storm git commit: STORM-2702: added in simple congestion detection

Posted by bo...@apache.org.
STORM-2702: added in simple congestion detection


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/a4c372c9
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/a4c372c9
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/a4c372c9

Branch: refs/heads/master
Commit: a4c372c95e5ad97beedc534e55d35cc09a962520
Parents: 0e3fc5a
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Thu Aug 24 17:13:50 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Thu Aug 24 17:13:50 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                |  3 +-
 .../loadgen/HttpForwardingMetricsConsumer.java  |  4 ++-
 .../loadgen/HttpForwardingMetricsServer.java    |  4 +--
 .../apache/storm/loadgen/LoadMetricsServer.java | 34 ++++++++++++++++----
 4 files changed, 34 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a4c372c9/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index 1e5c69d..29d27ab 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -86,7 +86,7 @@ Not all options are supported by all reporters.
 |Reporter Option| Description | Supported Reporters|
 |---------------|-------------|--------------------|
 |time | Set the time unit that you want latency and CPU reported in.  This can be from nanoseconds up to seconds.  Most names are supported for the types| legacy, csv, tsv, fixed|
-|columns | A comma separated list of columns to output (see below for the metrics supported).  A `*` is replaced by all metrics. Defaults to "start_time", "end_time", "rate", "mean", "99%ile", "99.9%ile", "cores", "mem", "failed", "ids" | csv, tsv, fixed |
+|columns | A comma separated list of columns to output (see below for the metrics supported).  A `*` is replaced by all metrics. Defaults to "start_time", "end_time", "rate", "mean", "99%ile", "99.9%ile", "cores", "mem", "failed", "ids", "congested" | csv, tsv, fixed |
 |extraColumns | Like columns but ones that should be added to the defaults instead of replacing them. A `*` is replaced by all metrics. | csv, tsv, fixed |
 |meta | An arbitrary string that will appear as a "meta" column at the end.  This helps when appending to files to keep different runs separated | csv, tsv, fixed|
 |columnWidth | The width of each field | fixed|
@@ -118,6 +118,7 @@ There are a lot of different metrics supported
 |end_time| The ending time of the metrics window from the the first topology was launched. | all
 |time_window| the length in seconds for the time window. | all
 |ids| The topology ids that are being tracked | all
+|congested| Componets that appear to be congested | all
 |storm_version| The version of storm as reported by the client | all
 |java_version| The version of java as reported by the client | all
 |os_arch| The OS architecture as reported by the client | all

http://git-wip-us.apache.org/repos/asf/storm/blob/a4c372c9/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
index 5829e9d..645000d 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
@@ -48,6 +48,7 @@ public class HttpForwardingMetricsConsumer implements IMetricsConsumer {
     private transient URL url;
     private transient IErrorReporter errorReporter;
     private transient KryoValuesSerializer serializer;
+    private transient String topologyId;
 
     @Override
     public void prepare(Map<String, Object> topoConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) { 
@@ -55,6 +56,7 @@ public class HttpForwardingMetricsConsumer implements IMetricsConsumer {
             url = new URL((String)registrationArgument);
             this.errorReporter = errorReporter;
             serializer = new KryoValuesSerializer(topoConf);
+            topologyId = context.getStormId();
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
@@ -67,7 +69,7 @@ public class HttpForwardingMetricsConsumer implements IMetricsConsumer {
             con.setRequestMethod("POST");
             con.setDoOutput(true);
             try (Output out = new Output(con.getOutputStream())) {
-                serializer.serializeInto(Arrays.asList(taskInfo, dataPoints), out);
+                serializer.serializeInto(Arrays.asList(taskInfo, dataPoints, topologyId), out);
                 out.flush();
             }
             //The connection is not sent unless a response is requested

http://git-wip-us.apache.org/repos/asf/storm/blob/a4c372c9/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
index 99a980b..247d017 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
@@ -57,7 +57,7 @@ public abstract class HttpForwardingMetricsServer {
         protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
             Input in = new Input(request.getInputStream());
             List<Object> metrics = des.get().deserializeFrom(in);
-            handle((TaskInfo)metrics.get(0), (Collection<DataPoint>)metrics.get(1));
+            handle((TaskInfo)metrics.get(0), (Collection<DataPoint>)metrics.get(1), (String)metrics.get(2));
             response.setStatus(HttpServletResponse.SC_OK);
         }
     }
@@ -74,7 +74,7 @@ public abstract class HttpForwardingMetricsServer {
     }
 
     //This needs to be thread safe
-    public abstract void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints);
+    public abstract void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints, String topologyId);
 
     /**
      * Start the server.

http://git-wip-us.apache.org/repos/asf/storm/blob/a4c372c9/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
index 457dedf..36050ae 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -19,11 +19,9 @@
 package org.apache.storm.loadgen;
 
 import com.google.common.annotations.VisibleForTesting;
-import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
-import java.io.PrintWriter;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -39,6 +37,7 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiFunction;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -104,6 +103,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         private long workers;
         private long executors;
         private long hosts;
+        private Map<String, String> congested;
 
         /**
          * Constructor.
@@ -114,7 +114,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
          */
         public Measurements(long uptimeSecs, long acked, long timeWindow, long failed, Histogram histo,
                             double userMs, double sysMs, double gcMs, long memBytes, Set<String> topologyIds,
-                            long workers, long executors, long hosts) {
+                            long workers, long executors, long hosts, Map<String, String> congested) {
             this.uptimeSecs = uptimeSecs;
             this.acked = acked;
             this.timeWindow = timeWindow;
@@ -128,6 +128,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             this.workers = workers;
             this.executors = executors;
             this.hosts = hosts;
+            this.congested = congested;
         }
 
         /**
@@ -147,6 +148,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             workers = 0;
             executors = 0;
             hosts = 0;
+            congested = new HashMap<>();
         }
 
         /**
@@ -167,6 +169,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             workers = Math.max(workers, other.workers);
             executors = Math.max(executors, other.executors);
             hosts = Math.max(hosts, other.hosts);
+            congested.putAll(other.congested);
         }
 
         public double getLatencyAtPercentile(double percential, TimeUnit unit) {
@@ -257,6 +260,10 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             return executors;
         }
 
+        public Map<String, String> getCongested() {
+            return congested;
+        }
+
         static Measurements combine(List<Measurements> measurements, Integer start, Integer count) {
             if (count == null) {
                 count = measurements.size();
@@ -396,6 +403,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         tmp.put("uptime",  new MetricExtractor((m, unit) -> m.getUptimeSecs(), "s"));
         tmp.put("time_window",  new MetricExtractor((m, unit) -> m.getTimeWindow(), "s"));
         tmp.put("ids",  new MetricExtractor((m, unit) -> m.getTopologyIds(), ""));
+        tmp.put("congested",  new MetricExtractor((m, unit) -> m.getCongested(), ""));
         tmp.put("workers",  new MetricExtractor((m, unit) -> m.getWorkers(), ""));
         tmp.put("hosts",  new MetricExtractor((m, unit) -> m.getHosts(), ""));
         tmp.put("executors",  new MetricExtractor((m, unit) -> m.getExecutors(), ""));
@@ -468,7 +476,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             } else {
                 //Wrapping it makes it mutable
                 extractors = new ArrayList<>(Arrays.asList("start_time", "end_time", "rate",
-                    "mean", "99%ile", "99.9%ile", "cores", "mem", "failed", "ids"));
+                    "mean", "99%ile", "99.9%ile", "cores", "mem", "failed", "ids", "congested"));
             }
 
             if (query.containsKey("extraColumns")) {
@@ -723,6 +731,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
     private final AtomicLong gcCount = new AtomicLong(0);
     private final AtomicLong gcMs = new AtomicLong(0);
     private final ConcurrentHashMap<String, MemMeasure> memoryBytes = new ConcurrentHashMap<>();
+    private final AtomicReference<ConcurrentHashMap<String, String>> congested = new AtomicReference<>(new ConcurrentHashMap<>());
     private final List<MetricResultsReporter> reporters;
     private long prevAcked = 0;
     private long prevFailed = 0;
@@ -903,7 +912,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         long memBytes = readMemory();
 
         allCombined.add(new Measurements(uptime, ackedThisTime, thisTime, failedThisTime, copy, user, sys, gc, memBytes,
-            ids, workers.size(), executors, hosts.size()));
+            ids, workers.size(), executors, hosts.size(), congested.getAndSet(new ConcurrentHashMap<>())));
         Measurements inWindow = Measurements.combine(allCombined, null, windowLength);
         for (MetricResultsReporter reporter: reporters) {
             reporter.reportWindow(inWindow, allCombined);
@@ -912,8 +921,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
 
     @Override
     @SuppressWarnings("unchecked")
-    public void handle(IMetricsConsumer.TaskInfo taskInfo, Collection<IMetricsConsumer.DataPoint> dataPoints) {
-        //crud no simple way to tie this to a given topology :(
+    public void handle(IMetricsConsumer.TaskInfo taskInfo, Collection<IMetricsConsumer.DataPoint> dataPoints, String topologyId) {
         String worker = taskInfo.srcWorkerHost + ":" + taskInfo.srcWorkerPort;
         for (IMetricsConsumer.DataPoint dp: dataPoints) {
             if (dp.name.startsWith("comp-lat-histo") && dp.value instanceof Histogram) {
@@ -952,6 +960,18 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                     }
                     mm.update(((Number)val).longValue());
                 }
+            } else if (dp.name.equals("__receive")) {
+                Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                Object pop = m.get("population");
+                Object cap = m.get("capacity");
+                if (pop instanceof Number && cap instanceof Number) {
+                    double full = ((Number) pop).doubleValue() / ((Number) cap).doubleValue();
+                    if (full >= 0.8) {
+                        congested.get().put(
+                            topologyId + ":" + taskInfo.srcComponentId + ":" + taskInfo.srcTaskId,
+                            "receive " + pop + "/" + cap);
+                    }
+                }
             }
         }
     }


[18/18] storm git commit: Merge branch 'STORM-2702' of https://github.com/revans2/incubator-storm into STORM-2702

Posted by bo...@apache.org.
Merge branch 'STORM-2702' of https://github.com/revans2/incubator-storm into STORM-2702

STORM-2702: storm-loadgen

This closes #2289


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/32389d75
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/32389d75
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/32389d75

Branch: refs/heads/master
Commit: 32389d75f5e2e4aab0c2699a2d5b842d20358340
Parents: 3ee9a89 59213c1
Author: Robert Evans <ev...@yahoo-inc.com>
Authored: Thu Sep 7 13:43:40 2017 -0500
Committer: Robert Evans <ev...@yahoo-inc.com>
Committed: Thu Sep 7 13:43:40 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                |  199 ++++
 examples/storm-loadgen/pom.xml                  |  119 ++
 .../org/apache/storm/loadgen/CaptureLoad.java   |  466 ++++++++
 .../storm/loadgen/EstimateThroughput.java       |  106 ++
 .../java/org/apache/storm/loadgen/GenLoad.java  |  307 +++++
 .../org/apache/storm/loadgen/GroupingType.java  |   91 ++
 .../loadgen/HttpForwardingMetricsConsumer.java  |   84 ++
 .../loadgen/HttpForwardingMetricsServer.java    |  123 ++
 .../org/apache/storm/loadgen/InputStream.java   |  263 +++++
 .../java/org/apache/storm/loadgen/LoadBolt.java |  146 +++
 .../org/apache/storm/loadgen/LoadCompConf.java  |  230 ++++
 .../apache/storm/loadgen/LoadMetricsServer.java | 1066 ++++++++++++++++++
 .../org/apache/storm/loadgen/LoadSpout.java     |  137 +++
 .../apache/storm/loadgen/NormalDistStats.java   |  151 +++
 .../org/apache/storm/loadgen/OutputStream.java  |  121 ++
 .../storm/loadgen/OutputStreamEngine.java       |  122 ++
 .../apache/storm/loadgen/ScopedTopologySet.java |  107 ++
 .../storm/loadgen/ThroughputVsLatency.java      |  257 +++++
 .../apache/storm/loadgen/TopologyLoadConf.java  |  479 ++++++++
 .../apache/storm/loadgen/LoadCompConfTest.java  |   57 +
 .../storm/loadgen/LoadMetricsServerTest.java    |   36 +
 .../storm/loadgen/NormalDistStatsTest.java      |   43 +
 .../apache/storm/loadgen/OutputStreamTest.java  |   37 +
 examples/storm-starter/pom.xml                  |    5 -
 .../storm/starter/ThroughputVsLatency.java      |  377 -------
 pom.xml                                         |    2 +-
 storm-client-misc/pom.xml                       |   63 --
 .../metric/HttpForwardingMetricsConsumer.java   |   85 --
 .../metric/HttpForwardingMetricsServer.java     |  118 --
 29 files changed, 4748 insertions(+), 649 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/32389d75/pom.xml
----------------------------------------------------------------------


[17/18] storm git commit: Added in the option to tee output to stdout

Posted by bo...@apache.org.
Added in the option to tee output to stdout


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/59213c16
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/59213c16
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/59213c16

Branch: refs/heads/master
Commit: 59213c16865023bad72036ecf4719fdc6ecb6c67
Parents: 113413c
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Sep 1 11:15:41 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Sep 1 11:15:41 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                |  1 +
 .../apache/storm/loadgen/LoadMetricsServer.java | 46 +++++++++++++++-----
 2 files changed, 36 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/59213c16/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index b24b8ed..5a6a030 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -90,6 +90,7 @@ Not all options are supported by all reporters.
 |extraColumns | Like columns but ones that should be added to the defaults instead of replacing them. A `*` is replaced by all metrics. | csv, tsv, fixed |
 |meta | An arbitrary string that will appear as a "meta" column at the end.  This helps when appending to files to keep different runs separated | csv, tsv, fixed|
 |precision | The number of places after the decimal point to print out.  The default for fixed is 3, all others it is unlimited. | csv, tsv, fixed|
+|tee | A boolean saying if in addition to writing to a file should the output be written to stdout too. | csv, tsv, fixed|
 |columnWidth | The width of each field | fixed|
 
 There are a lot of different metrics supported

http://git-wip-us.apache.org/repos/asf/storm/blob/59213c16/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
index 546b3ca..10ec698 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -21,6 +21,8 @@ package org.apache.storm.loadgen;
 import com.google.common.annotations.VisibleForTesting;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+import java.io.OutputStream;
 import java.io.PrintStream;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
@@ -45,6 +47,7 @@ import org.HdrHistogram.Histogram;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
+import org.apache.commons.io.output.TeeOutputStream;
 import org.apache.storm.generated.ClusterSummary;
 import org.apache.storm.generated.ExecutorSummary;
 import org.apache.storm.generated.Nimbus;
@@ -309,10 +312,21 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         void finish(List<Measurements> allTime) throws Exception;
     }
 
+    private static class NoCloseOutputStream extends FilterOutputStream {
+        public NoCloseOutputStream(OutputStream out) {
+            super(out);
+        }
+
+        @Override
+        public void close() {
+            //NOOP on purpose
+        }
+    }
+
     abstract static class FileReporter implements MetricResultsReporter {
         protected final PrintStream out;
-        private final boolean needsClose;
         protected final Map<String, MetricExtractor> allExtractors;
+        public final boolean includesSysOutOrError;
 
         public FileReporter(Map<String, MetricExtractor> allExtractors) throws FileNotFoundException {
             this(null, Collections.emptyMap(), allExtractors);
@@ -321,19 +335,30 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         public FileReporter(String path, Map<String, String> query,  Map<String, MetricExtractor> allExtractors)
             throws FileNotFoundException {
             boolean append = Boolean.parseBoolean(query.getOrDefault("append", "false"));
+            boolean tee = Boolean.parseBoolean(query.getOrDefault("tee", "false"));
+            boolean includesSysOutOrError = false;
 
+            OutputStream out = null;
             if (path == null || "/dev/stdout".equals(path)) {
-                out = System.out;
-                needsClose = false;
+                out = new NoCloseOutputStream(System.out);
+                includesSysOutOrError = true;
+                tee = false;
             } else if ("/dev/stderr".equals(path)) {
-                out = System.err;
-                needsClose = false;
+                out = new NoCloseOutputStream(System.err);
+                includesSysOutOrError = true;
+                tee = false;
             } else {
-                out = new PrintStream(new FileOutputStream(path, append));
-                needsClose = true;
+                out = new FileOutputStream(path, append);
+            }
+
+            if (tee) {
+                out = new TeeOutputStream(new NoCloseOutputStream(System.out), out);
+                includesSysOutOrError = true;
             }
+            this.out = new PrintStream(out);
             //Copy it in case we want to modify it
             this.allExtractors = new LinkedHashMap<>(allExtractors);
+            this.includesSysOutOrError = includesSysOutOrError;
         }
 
         @Override
@@ -343,7 +368,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
 
         @Override
         public void finish(List<Measurements> allTime) throws Exception {
-            if (needsClose && out != null) {
+            if (out != null) {
                 out.close();
             }
         }
@@ -840,9 +865,8 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         boolean foundStdOutOrErr = false;
         for (MetricResultsReporter rep : reporters) {
             if (rep instanceof FileReporter) {
-                PrintStream ps = ((FileReporter) rep).out;
-                if (ps == System.out || ps == System.err) {
-                    foundStdOutOrErr = true;
+                foundStdOutOrErr = ((FileReporter) rep).includesSysOutOrError;
+                if (foundStdOutOrErr) {
                     break;
                 }
             }


[13/18] storm git commit: Added in shutdown hook to kill topos on exit

Posted by bo...@apache.org.
Added in shutdown hook to kill topos on exit


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/2f5cde87
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/2f5cde87
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/2f5cde87

Branch: refs/heads/master
Commit: 2f5cde873a4fc3733b72cafb9761e844340c88ff
Parents: 381387b
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Sep 1 09:29:06 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Sep 1 09:29:06 2017 -0500

----------------------------------------------------------------------
 .../apache/storm/loadgen/ScopedTopologySet.java | 22 +++++++++++++++++---
 1 file changed, 19 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/2f5cde87/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ScopedTopologySet.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ScopedTopologySet.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ScopedTopologySet.java
index 1b6ed74..f7e7912 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ScopedTopologySet.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ScopedTopologySet.java
@@ -29,7 +29,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * A set of topology names that will be killed when this is closed.
+ * A set of topology names that will be killed when this is closed, or when the
+ * program exits.
  */
 public class ScopedTopologySet extends HashSet<String> implements AutoCloseable {
     private static final Logger LOG = LoggerFactory.getLogger(ScopedTopologySet.class);
@@ -40,11 +41,21 @@ public class ScopedTopologySet extends HashSet<String> implements AutoCloseable
     }
 
     private final Nimbus.Iface client;
-    private final Set<String> unmodWrapper;
+    private boolean closed = false;
 
+    /**
+     * Constructor.
+     * @param client the client used to kill the topologies when this exist.
+     */
     public ScopedTopologySet(Nimbus.Iface client) {
         this.client = client;
-        unmodWrapper = Collections.unmodifiableSet(this);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            try {
+                close();
+            } catch (Exception e) {
+                LOG.error("Error trying to shutdown topologies on exit", e);
+            }
+        }));
     }
 
     @Override
@@ -69,11 +80,15 @@ public class ScopedTopologySet extends HashSet<String> implements AutoCloseable
 
     @Override
     public void close() {
+        if (closed) {
+            return;
+        }
         RuntimeException saved = null;
         for (Iterator<String> it = super.iterator(); it.hasNext();) {
             String name = it.next();
             try {
                 client.killTopologyWithOpts(name, NO_WAIT_KILL);
+                it.remove();
             } catch (Exception e) {
                 RuntimeException wrapped = new RuntimeException("Error trying to kill " + name, e);
                 if (saved != null) {
@@ -87,5 +102,6 @@ public class ScopedTopologySet extends HashSet<String> implements AutoCloseable
         if (saved != null) {
             throw saved;
         }
+        closed = true;
     }
 }


[06/18] storm git commit: STORM-2702: Added in component specific adjustments to GenLoad

Posted by bo...@apache.org.
STORM-2702: Added in component specific adjustments to GenLoad


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b4cd98fa
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b4cd98fa
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b4cd98fa

Branch: refs/heads/master
Commit: b4cd98fa9f2f263aee3eaf7df53b385e62a34386
Parents: 211f8a9
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Thu Aug 24 09:09:34 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Thu Aug 24 09:09:34 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                |  6 +-
 .../java/org/apache/storm/loadgen/GenLoad.java  | 88 ++++++++++++++++----
 .../apache/storm/loadgen/LoadMetricsServer.java |  3 +-
 .../apache/storm/loadgen/TopologyLoadConf.java  | 63 ++++++++++++--
 4 files changed, 131 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b4cd98fa/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index e91409c..827fd77 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -39,11 +39,11 @@ storm jar storm-loadgen.jar org.apache.storm.loadgen.GenLoad [options] [capture_
 | --debug | Print debug information about the adjusted topology before submitting it. |
 |-h,--help | Print a help message |
 | --local-or-shuffle | Replace shuffle grouping with local or shuffle grouping. |
-| --parallel &lt;MULTIPLIER> | How much to scale the topology up or down in parallelism. The new parallelism will round up to the next whole number (defaults to 1.0 no scaling) The total throughput of the topology will not be scaled. |
+| --parallel &lt;MULTIPLIER(:TOPO:COMP)?> | How much to scale the topology up or down in parallelism. The new parallelism will round up to the next whole number. If a topology + component is supplied only that component will be scaled. If topo or component is blank or a `'*'` all topologies or components matched the other part will be scaled. Only 1 scaling rule, the most specific, will be applied to a component. Providing a topology name is considered more specific than not providing one. (defaults to 1.0 no scaling) |
 | -r,--report-interval &lt;INTERVAL_SECS> | How long in between reported metrics.  Will be rounded up to the next 10 sec boundary. default 30 |
 | --reporter &lt;TYPE:FILE?OPTIONS>  | Provide the config for a reporter to run. See below for more information about these |
 | -t,--test-time &lt;MINS> | How long to run the tests for in mins (defaults to 5) |
-| --throughput &lt;MULTIPLIER> | How much to scale the topology up or down in throughput. (defaults to 1.0 no scaling)|
+| --throughput &lt;MULTIPLIER(:TOPO:COMP)?> | How much to scale the topology up or down in throughput. If a topology + component is supplied only that component will be scaled. If topo or component is blank or a `'*'` all topologies or components matched will be scaled. Only 1 scaling rule, the most specific, will be applied to a component. Providing a topology name is considered more specific than not providing one.(defaults to 1.0 no scaling)|
 | -w,--report-window &lt;INTERVAL_SECS> | How long of a rolling window should be in each report.  Will be rounded up to the next report interval boundary. default 30|
 
 ## ThroughputVsLatency
@@ -125,7 +125,9 @@ There are a lot of different metrics supported
 |split_parallel| The parallelism of the split bolt for the `ThroughputVsLatency` topology. | ThroughputVsLatency
 |count_parallel| The parallelism of the count bolt for the `ThroughputVsLatency` topology. | ThroughputVsLatency
 |parallel\_adjust| The adjustment to the parallelism in `GenLoad`. | GenLoad
+|topo_parallel| A list of topology/component specfic adjustment rules to the parallelism in `GenLoad`. | GenLoad
 |throughput_adjust| The adjustment to the throughput in `GenLoad`. | GenLoad
+|topo_throughput| A list of topology/component specfic adjustment rules to the throughput in `GenLoad`. | GenLoad
 |local\_or\_shuffle| true if shuffles were replaced with local or shuffle in GenLoad. | GenLoad
 
 There are also some generic rules that you can use for some metrics.  Any metric that starts with `"conf:"` will be the config for that.  It does not include config overrides from the `GenLoad` file.

http://git-wip-us.apache.org/repos/asf/storm/blob/b4cd98fa/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
index 8821b2a..95ba8dd 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
@@ -23,6 +23,9 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.DefaultParser;
@@ -48,6 +51,8 @@ import org.slf4j.LoggerFactory;
 public class GenLoad {
     private static final Logger LOG = LoggerFactory.getLogger(GenLoad.class);
     private static final int TEST_EXECUTE_TIME_DEFAULT = 5;
+    private static final Pattern MULTI_PATTERN = Pattern.compile(
+        "(?<value>[^:?]+)(?::(?<topo>[^:]*):(?<comp>.*))?");
 
     /**
      * Main entry point for GenLoad application.
@@ -68,18 +73,25 @@ public class GenLoad {
             .build());
         options.addOption(Option.builder()
             .longOpt("parallel")
-            .argName("MULTIPLIER")
+            .argName("MULTIPLIER(:TOPO:COMP)?")
             .hasArg()
-            .desc("How much to scale the topology up or down in parallelism.\n"
-                + "The new parallelism will round up to the next whole number\n"
+            .desc("How much to scale the topology up or down in parallelism. "
+                + "The new parallelism will round up to the next whole number. "
+                + "If a topology + component is supplied only that component will be scaled. "
+                + "If topo or component is blank or a '*' all topologies or components matched will be scaled. "
+                + "Only 1 scaling rule, the most specific, will be applied to a component. Providing a topology name is considered more "
+                + "specific than not providing one."
                 + "(defaults to 1.0 no scaling)")
             .build());
         options.addOption(Option.builder()
             .longOpt("throughput")
-            .argName("MULTIPLIER")
+            .argName("MULTIPLIER(:TOPO:COMP)?")
             .hasArg()
-            .desc("How much to scale the topology up or down in throughput.\n"
-                + "Note this is applied after and build on any parallelism changes.\n"
+            .desc("How much to scale the topology up or down in throughput. "
+                + "If a topology + component is supplied only that component will be scaled. "
+                + "If topo or component is blank or a '*' all topologies or components matched will be scaled. "
+                + "Only 1 scaling rule, the most specific, will be applied to a component. Providing a topology name is considered more "
+                + "specific than not providing one."
                 + "(defaults to 1.0 no scaling)")
             .build());
         options.addOption(Option.builder()
@@ -95,18 +107,58 @@ public class GenLoad {
         CommandLine cmd = null;
         Exception commandLineException = null;
         double executeTime = TEST_EXECUTE_TIME_DEFAULT;
-        double parallel = 1.0;
-        double throughput = 1.0;
+        double globalParallel = 1.0;
+        Map<String, Double> topoSpecificParallel = new HashMap<>();
+        double globalThroughput = 1.0;
+        Map<String, Double> topoSpecificThroughput = new HashMap<>();
         try {
             cmd = parser.parse(options, args);
             if (cmd.hasOption("t")) {
                 executeTime = Double.valueOf(cmd.getOptionValue("t"));
             }
             if (cmd.hasOption("parallel")) {
-                parallel = Double.parseDouble(cmd.getOptionValue("parallel"));
+                for (String stringParallel : cmd.getOptionValues("parallel")) {
+                    Matcher m = MULTI_PATTERN.matcher(stringParallel);
+                    if (!m.matches()) {
+                        throw new ParseException("--parallel " + stringParallel + " is not in the format MULTIPLIER(:TOPO:COMP)?");
+                    }
+                    double parallel = Double.parseDouble(m.group("value"));
+                    String topo = m.group("topo");
+                    if (topo == null || topo.isEmpty()) {
+                        topo = "*";
+                    }
+                    String comp = m.group("comp");
+                    if (comp == null || comp.isEmpty()) {
+                        comp = "*";
+                    }
+                    if ("*".equals(topo) && "*".equals(comp)) {
+                        globalParallel = parallel;
+                    } else {
+                        topoSpecificParallel.put(topo + ":" + comp, parallel);
+                    }
+                }
             }
             if (cmd.hasOption("throughput")) {
-                throughput = Double.parseDouble(cmd.getOptionValue("throughput"));
+                for (String stringThroughput : cmd.getOptionValues("throughput")) {
+                    Matcher m = MULTI_PATTERN.matcher(stringThroughput);
+                    if (!m.matches()) {
+                        throw new ParseException("--throughput " + stringThroughput + " is not in the format MULTIPLIER(:TOPO:COMP)?");
+                    }
+                    double throughput = Double.parseDouble(m.group("value"));
+                    String topo = m.group("topo");
+                    if (topo == null || topo.isEmpty()) {
+                        topo = "*";
+                    }
+                    String comp = m.group("comp");
+                    if (comp == null || comp.isEmpty()) {
+                        comp = "*";
+                    }
+                    if ("*".equals(topo) && "*".equals(comp)) {
+                        globalThroughput = throughput;
+                    } else {
+                        topoSpecificThroughput.put(topo + ":" + comp, throughput);
+                    }
+                }
             }
         } catch (ParseException | NumberFormatException e) {
             commandLineException = e;
@@ -119,9 +171,13 @@ public class GenLoad {
             return;
         }
         Map<String, Object> metrics = new LinkedHashMap<>();
-        metrics.put("parallel_adjust", parallel);
-        metrics.put("throughput_adjust", throughput);
+        metrics.put("parallel_adjust", globalParallel);
+        metrics.put("throughput_adjust", globalThroughput);
         metrics.put("local_or_shuffle", cmd.hasOption("local-or-shuffle"));
+        metrics.put("topo_parallel", topoSpecificParallel.entrySet().stream().map((entry) -> entry.getValue() + ":" + entry.getKey())
+            .collect(Collectors.toList()));
+        metrics.put("topo_throuhgput", topoSpecificThroughput.entrySet().stream().map((entry) -> entry.getValue() + ":" + entry.getKey())
+            .collect(Collectors.toList()));
 
         Config conf = new Config();
         LoadMetricsServer metricServer = new LoadMetricsServer(conf, cmd, metrics);
@@ -134,12 +190,8 @@ public class GenLoad {
             for (String topoFile : cmd.getArgList()) {
                 try {
                     TopologyLoadConf tlc = readTopology(topoFile);
-                    if (parallel != 1.0) {
-                        tlc = tlc.scaleParallel(parallel);
-                    }
-                    if (throughput != 1.0) {
-                        tlc = tlc.scaleThroughput(throughput);
-                    }
+                    tlc = tlc.scaleParallel(globalParallel, topoSpecificParallel);
+                    tlc = tlc.scaleThroughput(globalThroughput, topoSpecificThroughput);
                     if (cmd.hasOption("local-or-shuffle")) {
                         tlc = tlc.replaceShuffleWithLocalOrShuffle();
                     }

http://git-wip-us.apache.org/repos/asf/storm/blob/b4cd98fa/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
index fd12247..69adabc 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -630,7 +630,8 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             .desc("Provide the config for a reporter to run.  Supported types are:\n"
                 + "LEGACY - (write things out in the legacy format)\n"
                 + "TSV - tab separated values\n"
-                + "CSV - comma separated values")
+                + "CSV - comma separated values\n"
+                + "PATH and OPTIONS are each optional but must be marked with a ':' or '?' separator respectively.")
             .build());
 
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/b4cd98fa/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
index 1a45ccc..38458c6 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/TopologyLoadConf.java
@@ -228,13 +228,57 @@ public class TopologyLoadConf {
     }
 
     /**
+     * The first one that is not null
+     * @param rest all the other somethings
+     * @param <V> whatever type you want.
+     * @return the first one that is not null
+     */
+    static <V> V or(V...rest) {
+        for (V i: rest) {
+            if (i != null) {
+                return i;
+            }
+        }
+        return null;
+    }
+
+    LoadCompConf scaleCompParallel(LoadCompConf comp, double v, Map<String, Double> topoSpecificParallel) {
+        LoadCompConf ret = comp;
+        double scale = or(topoSpecificParallel.get(name + ":" + comp.id),
+            topoSpecificParallel.get(name + ":*"),
+            topoSpecificParallel.get("*:" + comp.id),
+            v);
+        if (scale != 1.0) {
+            ret = ret.scaleParallel(scale);
+        }
+        return ret;
+    }
+
+    LoadCompConf scaleCompThroughput(LoadCompConf comp, double v, Map<String, Double> topoSpecificParallel) {
+        LoadCompConf ret = comp;
+        double scale = or(topoSpecificParallel.get(name + ":" + comp.id),
+            topoSpecificParallel.get(name + ":*"),
+            topoSpecificParallel.get("*:" + comp.id),
+            v);
+        if (scale != 1.0) {
+            ret = ret.scaleThroughput(scale);
+        }
+        return ret;
+    }
+
+    /**
      * Scale all of the components in the topology by a percentage (but keep the throughput the same).
      * @param v the amount to scale them by.  1.0 is nothing, 0.5 cuts them in half, 2.0 doubles them.
      * @return a copy of this with the needed adjustments made.
      */
-    public TopologyLoadConf scaleParallel(double v) {
-        List<LoadCompConf> scaledSpouts = spouts.stream().map((c) -> c.scaleParallel(v)).collect(Collectors.toList());
-        List<LoadCompConf> scaledBolts = bolts.stream().map((c) -> c.scaleParallel(v)).collect(Collectors.toList());
+    public TopologyLoadConf scaleParallel(double v, Map<String, Double> topoSpecific) {
+        if (v == 1.0 && (topoSpecific == null || topoSpecific.isEmpty())) {
+            return this;
+        }
+        List<LoadCompConf> scaledSpouts = spouts.stream().map((s) -> scaleCompParallel(s, v, topoSpecific))
+            .collect(Collectors.toList());
+        List<LoadCompConf> scaledBolts = bolts.stream().map((s) -> scaleCompParallel(s, v, topoSpecific))
+            .collect(Collectors.toList());
         return new TopologyLoadConf(name, topoConf, scaledSpouts, scaledBolts, streams);
     }
 
@@ -243,11 +287,14 @@ public class TopologyLoadConf {
      * @param v the amount to scale it by 1.0 is nothing 0.5 cuts it in half and 2.0 doubles it.
      * @return a copy of this with the needed adjustments made.
      */
-    public TopologyLoadConf scaleThroughput(double v) {
-        List<LoadCompConf> scaledSpouts = spouts.stream()
-            .map((c) -> c.scaleThroughput(v)).collect(Collectors.toList());
-        List<LoadCompConf> scaledBolts = bolts.stream()
-            .map((c) -> c.scaleThroughput(v)).collect(Collectors.toList());
+    public TopologyLoadConf scaleThroughput(double v, Map<String, Double> topoSpecific) {
+        if (v == 1.0 && (topoSpecific == null || topoSpecific.isEmpty())) {
+            return this;
+        }
+        List<LoadCompConf> scaledSpouts = spouts.stream().map((s) -> scaleCompThroughput(s, v, topoSpecific))
+            .collect(Collectors.toList());
+        List<LoadCompConf> scaledBolts = bolts.stream().map((s) -> scaleCompThroughput(s, v, topoSpecific))
+            .collect(Collectors.toList());
         return new TopologyLoadConf(name, topoConf, scaledSpouts, scaledBolts, streams);
     }
 


[03/18] storm git commit: STORM-2702: storm-loadgen

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
new file mode 100644
index 0000000..c126ca5
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -0,0 +1,784 @@
+/**
+ * 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.loadgen;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.PrintStream;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.HdrHistogram.Histogram;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.storm.generated.ClusterSummary;
+import org.apache.storm.generated.ExecutorSummary;
+import org.apache.storm.generated.Nimbus;
+import org.apache.storm.generated.SpoutStats;
+import org.apache.storm.generated.TopologyInfo;
+import org.apache.storm.generated.TopologySummary;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A metrics server that records and reports metrics for a set of running topologies.
+ */
+public class LoadMetricsServer extends HttpForwardingMetricsServer {
+    private static final Logger LOG = LoggerFactory.getLogger(HttpForwardingMetricsServer.class);
+
+    private static class MemMeasure {
+        private long mem = 0;
+        private long time = 0;
+
+        synchronized void update(long mem) {
+            this.mem = mem;
+            time = System.currentTimeMillis();
+        }
+
+        public synchronized long get() {
+            return isExpired() ? 0L : mem;
+        }
+
+        synchronized boolean isExpired() {
+            return (System.currentTimeMillis() - time) >= 20000;
+        }
+    }
+
+    @VisibleForTesting
+    static double convert(double value, TimeUnit from, TimeUnit target) {
+        if (target.compareTo(from) > 0) {
+            return value / from.convert(1, target);
+        }
+        return value * target.convert(1, from);
+    }
+
+    public static class Measurements {
+        private final Histogram histo;
+        private double userMs;
+        private double sysMs;
+        private double gcMs;
+        private long memBytes;
+        private long uptimeSecs;
+        private long timeWindow;
+        private long acked;
+        private long failed;
+        private Set<String> topologyIds;
+
+        /**
+         * Constructor.
+         * @param histo latency histogram.
+         * @param userMs user CPU in ms.
+         * @param sysMs system CPU in ms.
+         * @param gcMs GC CPU in ms.
+         */
+        public Measurements(long uptimeSecs, long acked, long timeWindow, long failed, Histogram histo,
+                            double userMs, double sysMs, double gcMs, long memBytes, Set<String> topologyIds) {
+            this.uptimeSecs = uptimeSecs;
+            this.acked = acked;
+            this.timeWindow = timeWindow;
+            this.failed = failed;
+            this.userMs = userMs;
+            this.sysMs = sysMs;
+            this.gcMs = gcMs;
+            this.histo = histo;
+            this.memBytes = memBytes;
+            this.topologyIds = topologyIds;
+        }
+
+        /**
+         * Default Constructor.
+         */
+        public Measurements() {
+            histo = new Histogram(3600000000000L, 3);
+            sysMs = 0;
+            userMs = 0;
+            gcMs = 0;
+            memBytes = 0;
+            uptimeSecs = 0;
+            timeWindow = 0;
+            acked = 0;
+            failed = 0;
+            topologyIds = new HashSet<>();
+        }
+
+        /**
+         * Add other to this.
+         * @param other meaurements to add in.
+         */
+        public void add(Measurements other) {
+            histo.add(other.histo);
+            sysMs += other.sysMs;
+            userMs += other.userMs;
+            gcMs += other.gcMs;
+            memBytes = Math.max(memBytes, other.memBytes);
+            acked += other.acked;
+            failed += other.failed;
+            uptimeSecs = Math.max(uptimeSecs, other.uptimeSecs);
+            timeWindow += other.timeWindow;
+            topologyIds.addAll(other.topologyIds);
+        }
+
+        public double getLatencyAtPercentile(double percential, TimeUnit unit) {
+            return convert(histo.getValueAtPercentile(percential), TimeUnit.NANOSECONDS, unit);
+        }
+
+        public double getMinLatency(TimeUnit unit) {
+            return convert(histo.getMinValue(), TimeUnit.NANOSECONDS, unit);
+        }
+
+        public double getMaxLatency(TimeUnit unit) {
+            return convert(histo.getMaxValue(), TimeUnit.NANOSECONDS, unit);
+        }
+
+        public double getMeanLatency(TimeUnit unit) {
+            return convert(histo.getMean(), TimeUnit.NANOSECONDS, unit);
+        }
+
+        public double getLatencyStdDeviation(TimeUnit unit) {
+            return convert(histo.getStdDeviation(), TimeUnit.NANOSECONDS, unit);
+        }
+
+        public double getUserTime(TimeUnit unit) {
+            return convert(userMs, TimeUnit.MILLISECONDS, unit);
+        }
+
+        public double getSysTime(TimeUnit unit) {
+            return convert(sysMs, TimeUnit.MILLISECONDS, unit);
+        }
+
+        public double getGc(TimeUnit unit) {
+            return convert(gcMs, TimeUnit.MILLISECONDS, unit);
+        }
+
+        public double getMemMb() {
+            return memBytes / (1024.0 * 1024.0);
+        }
+
+        public long getUptimeSecs() {
+            return uptimeSecs;
+        }
+
+        public long getCompleted() {
+            return histo.getTotalCount();
+        }
+
+        public double getCompletedPerSec() {
+            return getCompleted() / (double)timeWindow;
+        }
+
+        public long getAcked() {
+            return acked;
+        }
+
+        public double getAckedPerSec() {
+            return acked / (double)timeWindow;
+        }
+
+        public long getFailed() {
+            return failed;
+        }
+
+        public long startTime() {
+            return uptimeSecs - timeWindow;
+        }
+
+        public long endTime() {
+            return uptimeSecs;
+        }
+
+        public double getTimeWindow() {
+            return timeWindow;
+        }
+
+        public Set<String> getTopologyIds() {
+            return topologyIds;
+        }
+
+        static Measurements combine(List<Measurements> measurements, Integer start, Integer count) {
+            if (count == null) {
+                count = measurements.size();
+            }
+
+            if (start == null) {
+                start = measurements.size() - count;
+            }
+            start = Math.max(0, start);
+            count = Math.min(count, measurements.size() - start);
+
+            Measurements ret = new Measurements();
+            for (int i = start; i < start + count; i ++) {
+                ret.add(measurements.get(i));
+            }
+            return ret;
+        }
+    }
+
+    interface MetricResultsReporter {
+        void start();
+
+        void reportWindow(Measurements inWindow, List<Measurements> allTime);
+
+        void finish(List<Measurements> allTime) throws Exception;
+    }
+
+    abstract static class FileReporter implements MetricResultsReporter {
+        protected final PrintStream out;
+        private final boolean needsClose;
+
+        public FileReporter() throws FileNotFoundException {
+            this(null, null);
+        }
+
+        public FileReporter(String path, Map<String, String> query) throws FileNotFoundException {
+            boolean append = Boolean.parseBoolean(query.getOrDefault("append", "false"));
+
+            if (path == null || "/dev/stdout".equals(path)) {
+                out = System.out;
+                needsClose = false;
+            } else if ("/dev/stderr".equals(path)) {
+                out = System.err;
+                needsClose = false;
+            } else {
+                out = new PrintStream(new FileOutputStream(path, append));
+                needsClose = true;
+            }
+        }
+
+        @Override
+        public void start() {
+            //NOOP
+        }
+
+        @Override
+        public void finish(List<Measurements> allTime) throws Exception {
+            if (needsClose && out != null) {
+                out.close();
+            }
+        }
+    }
+
+    private static final Map<String, TimeUnit> UNIT_MAP;
+
+    static {
+        HashMap<String, TimeUnit> tmp = new HashMap<>();
+        tmp.put("NS", TimeUnit.NANOSECONDS);
+        tmp.put("NANO", TimeUnit.NANOSECONDS);
+        tmp.put("NANOSEC", TimeUnit.NANOSECONDS);
+        tmp.put("NANOSECOND", TimeUnit.NANOSECONDS);
+        tmp.put("NANOSECONDS", TimeUnit.NANOSECONDS);
+        tmp.put("μS", TimeUnit.MICROSECONDS);
+        tmp.put("US", TimeUnit.MICROSECONDS);
+        tmp.put("MICRO", TimeUnit.MICROSECONDS);
+        tmp.put("MICROSEC", TimeUnit.MICROSECONDS);
+        tmp.put("MICROSECOND", TimeUnit.MICROSECONDS);
+        tmp.put("MICROSECONDS", TimeUnit.MICROSECONDS);
+        tmp.put("MS", TimeUnit.MILLISECONDS);
+        tmp.put("MILLI", TimeUnit.MILLISECONDS);
+        tmp.put("MILLISEC", TimeUnit.MILLISECONDS);
+        tmp.put("MILLISECOND", TimeUnit.MILLISECONDS);
+        tmp.put("MILLISECONDS", TimeUnit.MILLISECONDS);
+        tmp.put("S", TimeUnit.SECONDS);
+        tmp.put("SEC", TimeUnit.SECONDS);
+        tmp.put("SECOND", TimeUnit.SECONDS);
+        tmp.put("SECONDS", TimeUnit.SECONDS);
+        tmp.put("M", TimeUnit.MINUTES);
+        tmp.put("MIN", TimeUnit.MINUTES);
+        tmp.put("MINUTE", TimeUnit.MINUTES);
+        tmp.put("MINUTES", TimeUnit.MINUTES);
+        UNIT_MAP = Collections.unmodifiableMap(tmp);
+    }
+
+    private static final Map<TimeUnit, String> TIME_UNIT_NAME;
+
+    static {
+        HashMap<TimeUnit, String> tmp = new HashMap<>();
+        tmp.put(TimeUnit.NANOSECONDS, "ns");
+        tmp.put(TimeUnit.MICROSECONDS, "μs");
+        tmp.put(TimeUnit.MILLISECONDS, "ms");
+        tmp.put(TimeUnit.SECONDS, "s");
+        tmp.put(TimeUnit.MINUTES, "m");
+        TIME_UNIT_NAME = Collections.unmodifiableMap(tmp);
+    }
+
+    private static final Map<String, MetricExtractor> NAMED_EXTRACTORS;
+
+    static {
+        //Perhaps there is a better way to do this???
+        HashMap<String, MetricExtractor> tmp = new HashMap<>();
+        tmp.put("99%ile", new MetricExtractor((m, unit) -> m.getLatencyAtPercentile(99.0, unit)));
+        tmp.put("99.9%ile", new MetricExtractor((m, unit) -> m.getLatencyAtPercentile(99.9, unit)));
+        tmp.put("median", new MetricExtractor((m, unit) -> m.getLatencyAtPercentile(50, unit)));
+        tmp.put("mean", new MetricExtractor((m, unit) -> m.getMeanLatency(unit)));
+        tmp.put("min", new MetricExtractor((m, unit) -> m.getMinLatency(unit)));
+        tmp.put("max", new MetricExtractor((m, unit) -> m.getMaxLatency(unit)));
+        tmp.put("stddev", new MetricExtractor((m, unit) -> m.getLatencyStdDeviation(unit)));
+        tmp.put("user_cpu", new MetricExtractor((m, unit) -> m.getUserTime(unit)));
+        tmp.put("sys_cpu", new MetricExtractor((m, unit) -> m.getSysTime(unit)));
+        tmp.put("gc_cpu", new MetricExtractor((m, unit) -> m.getGc(unit)));
+        tmp.put("cores", new MetricExtractor(
+            (m, unit) -> (m.getSysTime(TimeUnit.SECONDS) + m.getUserTime(TimeUnit.SECONDS)) / m.getTimeWindow(),
+            ""));
+        tmp.put("uptime",  new MetricExtractor((m, unit) -> m.getUptimeSecs(), "s"));
+        tmp.put("acked",  new MetricExtractor((m, unit) -> m.getAcked(), ""));
+        tmp.put("rate",  new MetricExtractor((m, unit) -> m.getAckedPerSec(), "tuple/s"));
+        tmp.put("completed",  new MetricExtractor((m, unit) -> m.getCompleted(), ""));
+        tmp.put("completion_rate",  new MetricExtractor((m, unit) -> m.getCompletedPerSec(), "tuple/s"));
+        tmp.put("mem",  new MetricExtractor((m, unit) -> m.getMemMb(), "MB"));
+        tmp.put("failed",  new MetricExtractor((m, unit) -> m.getFailed(), ""));
+        tmp.put("start_time",  new MetricExtractor((m, unit) -> m.startTime(),"s"));
+        tmp.put("end_time",  new MetricExtractor((m, unit) -> m.endTime(), "s"));
+        tmp.put("time_window",  new MetricExtractor((m, unit) -> m.getTimeWindow(), "s"));
+        tmp.put("ids",  new MetricExtractor((m, unit) -> m.getTopologyIds(), ""));
+        NAMED_EXTRACTORS = Collections.unmodifiableMap(tmp);
+    }
+
+    static class MetricExtractor {
+        private final String unit;
+        private final BiFunction<Measurements, TimeUnit, Object> func;
+
+        public MetricExtractor(BiFunction<Measurements, TimeUnit, Object> func) {
+            this.func = func;
+            this.unit = null;
+        }
+
+        public MetricExtractor(BiFunction<Measurements, TimeUnit, Object> func, String unit) {
+            this.func = func;
+            this.unit = unit;
+        }
+
+        public Object get(Measurements m, TimeUnit unit) {
+            return func.apply(m, unit);
+        }
+
+        public String formatName(String name, TimeUnit targetUnit) {
+            StringBuilder ret = new StringBuilder();
+            ret.append(name);
+            if (unit == null || !unit.isEmpty()) {
+                ret.append("(");
+                if (unit == null) {
+                    ret.append(TIME_UNIT_NAME.get(targetUnit));
+                } else {
+                    ret.append(unit);
+                }
+                ret.append(")");
+            }
+            return ret.toString();
+        }
+    }
+
+    static class SepValReporter extends  FileReporter {
+        private final TimeUnit targetUnit;
+        private final String separator;
+        private final List<String> extractors;
+        private final String meta;
+
+        public SepValReporter(String separator, String path, Map<String, String> query) throws FileNotFoundException {
+            super(path, query);
+            this.separator = separator;
+            targetUnit = UNIT_MAP.get(query.getOrDefault("time", "MILLISECONDS").toUpperCase());
+            if (targetUnit == null) {
+                throw new IllegalArgumentException(query.get("time") + " is not a supported time unit");
+            }
+            if (query.containsKey("columns")) {
+                extractors = Arrays.asList(query.get("columns").split("\\s*,\\s*"));
+                HashSet<String> notFound = new HashSet<>(extractors);
+                notFound.removeAll(NAMED_EXTRACTORS.keySet());
+                if (notFound.size() > 0) {
+                    throw new IllegalArgumentException(notFound + " columns are not supported");
+                }
+            } else {
+                //Wrapping it makes it mutable
+                extractors = new ArrayList<>(Arrays.asList("start_time", "end_time", "completion_rate",
+                    "mean", "99%ile", "99.9%ile", "cores", "mem", "failed"));
+            }
+
+            if (query.containsKey("extraColumns")) {
+                List<String> moreExtractors = Arrays.asList(query.get("extraColumns").split("\\s*,\\s*"));
+                for (String extractor: moreExtractors) {
+                    if (!NAMED_EXTRACTORS.containsKey(extractor)) {
+                        throw new IllegalArgumentException(extractor + " is not a supported column");
+                    }
+                    if (!extractors.contains(extractor)) {
+                        extractors.add(extractor);
+                    }
+                }
+            }
+
+            meta = query.get("meta");
+        }
+
+        @Override
+        public void start() {
+            boolean first = true;
+            for (String name: extractors) {
+                if (!first) {
+                    out.print(separator);
+                }
+                first = false;
+                out.print(NAMED_EXTRACTORS.get(name).formatName(name, targetUnit));
+            }
+            if (meta != null) {
+                out.print(separator);
+                out.print("meta");
+            }
+            out.println();
+        }
+
+        @Override
+        public void reportWindow(Measurements m, List<Measurements> allTime) {
+            boolean first = true;
+            for (String name: extractors) {
+                if (!first) {
+                    out.print(separator);
+                }
+                first = false;
+                Object value = NAMED_EXTRACTORS.get(name).get(m, targetUnit);
+                String svalue = value == null ? "" : value.toString();
+                out.print(escape(svalue));
+            }
+            if (meta != null) {
+                out.print(separator);
+                out.print(escape(meta));
+            }
+            out.println();
+        }
+
+        private String escape(String svalue) {
+            return svalue.replace("\\", "\\\\").replace(separator, "\\" + separator);
+        }
+    }
+
+    static class LegacyReporter extends FileReporter {
+        private final TimeUnit targetUnitOverride;
+
+        public LegacyReporter() throws FileNotFoundException {
+            super();
+            targetUnitOverride = null;
+        }
+
+        public LegacyReporter(String path, Map<String, String> query) throws FileNotFoundException {
+            super(path, query);
+            if (query.containsKey("time")) {
+                targetUnitOverride = UNIT_MAP.get(query.get("time").toUpperCase());
+                if (targetUnitOverride == null) {
+                    throw new IllegalArgumentException(query.get("time") + " is not a supported time unit");
+                }
+            } else {
+                targetUnitOverride = null;
+            }
+        }
+
+        @Override
+        public void reportWindow(Measurements m, List<Measurements> allTime) {
+            TimeUnit nsOr = TimeUnit.NANOSECONDS;
+            TimeUnit msOr = TimeUnit.MILLISECONDS;
+            if (targetUnitOverride != null) {
+                nsOr = targetUnitOverride;
+                msOr = targetUnitOverride;
+            }
+
+            Measurements total = Measurements.combine(allTime, null, null);
+            out.printf("uptime: %,4d acked: %,9d acked/sec: %,10.2f failed: %,8d "
+                    + "99%%: %,15.0f 99.9%%: %,15.0f min: %,15.0f max: %,15.0f mean: %,15.2f "
+                    + "stddev: %,15.2f user: %,10.0f sys: %,10.0f gc: %,10.0f mem: %,10.2f\n",
+                m.getUptimeSecs(), m.getAcked(), m.getAckedPerSec(), total.getFailed(),
+                m.getLatencyAtPercentile(99.0, nsOr),
+                m.getLatencyAtPercentile(99.9, nsOr),
+                m.getMinLatency(nsOr),
+                m.getMaxLatency(nsOr),
+                m.getMeanLatency(nsOr),
+                m.getLatencyStdDeviation(nsOr),
+                m.getUserTime(msOr),
+                m.getSysTime(msOr),
+                m.getGc(msOr),
+                m.getMemMb());
+        }
+    }
+
+    /**
+     * Add Command line options for configuring the output of this.
+     * @param options command line options to update
+     */
+    public static void addCommandLineOptions(Options options) {
+        //We want to be able to select the measurement interval
+        // reporting window (We don't need 3 different reports)
+        // We want to be able to specify format (and configs specific to the format)
+        // With perhaps defaults overall
+        options.addOption(Option.builder("r")
+            .longOpt("report-interval")
+            .hasArg()
+            .argName("INTERVAL_SECS")
+            .desc("How long in between reported metrics.  Will be rounded up to the next 10 sec boundary.\n"
+                + "default " + DEFAULT_REPORT_INTERVAL)
+            .build());
+
+        options.addOption(Option.builder("w")
+            .longOpt("report-window")
+            .hasArg()
+            .argName("INTERVAL_SECS")
+            .desc("How long of a rolling window should be in each report.  Will be rounded up to the next report interval boundary.\n"
+                + "default " + DEFAULT_WINDOW_INTERVAL)
+            .build());
+
+        options.addOption(Option.builder()
+            .longOpt("reporter")
+            .hasArg()
+            .argName("TYPE:PATH?OPTIONS")
+            .desc("Provide the config for a reporter to run.  Supported types are:\n"
+                + "LEGACY - (write things out in the legacy format)\n"
+                + "TSV - tab separated values\n"
+                + "CSV - comma separated values")
+            .build());
+
+    }
+
+    public static final long DEFAULT_REPORT_INTERVAL = 30;
+    public static final long DEFAULT_WINDOW_INTERVAL = DEFAULT_REPORT_INTERVAL;
+    private static final Pattern REPORTER_PATTERN = Pattern.compile(
+        "(?<type>[^:?]+)(?::(?<path>[^?]+))?(?:\\?(?<query>.*))?");
+
+    private final Histogram histo = new Histogram(3600000000000L, 3);
+    private final AtomicLong systemCpu = new AtomicLong(0);
+    private final AtomicLong userCpu = new AtomicLong(0);
+    private final AtomicLong gcCount = new AtomicLong(0);
+    private final AtomicLong gcMs = new AtomicLong(0);
+    private final ConcurrentHashMap<String, MemMeasure> memoryBytes = new ConcurrentHashMap<>();
+    private final List<MetricResultsReporter> reporters;
+    private long prevAcked = 0;
+    private long prevFailed = 0;
+    private long prevUptime = 0;
+    private int windowLength = 1;
+    private long reportIntervalSecs = DEFAULT_REPORT_INTERVAL;
+
+    private final LinkedList<Measurements> allCombined = new LinkedList<>();
+
+    LoadMetricsServer(Map<String, Object> conf, CommandLine commandLine) throws URISyntaxException, FileNotFoundException {
+        super(conf);
+        if (commandLine.hasOption("r")) {
+            reportIntervalSecs = Long.parseLong(commandLine.getOptionValue("r"));
+            reportIntervalSecs = ((reportIntervalSecs + 1) / 10) * 10;
+        }
+        if (commandLine.hasOption("w")) {
+            long window = Long.parseLong(commandLine.getOptionValue("w"));
+            windowLength = (int) ((window + 1) / reportIntervalSecs);
+        }
+        reporters = new ArrayList<>();
+        if (commandLine.hasOption("reporter")) {
+            for (String reporterString: commandLine.getOptionValues("reporter")) {
+                Matcher m = REPORTER_PATTERN.matcher(reporterString);
+                if (!m.matches()) {
+                    throw new IllegalArgumentException(reporterString + " does nto look like it is a reporter");
+                }
+                String type = m.group("type");
+                String path = m.group("path");
+                Map<String, String> query = new HashMap<>();
+                String queryString = m.group("query");
+                if (queryString != null) {
+                    for (String param : queryString.split("&")) {
+                        String[] pair = param.split("=");
+                        String key = pair[0];
+                        String value = pair.length > 1 ? pair[1] : "true";
+                        query.put(key, value);
+                    }
+                }
+                type = type.toUpperCase();
+                switch (type) {
+                    case "LEGACY":
+                        reporters.add(new LegacyReporter(path, query));
+                        break;
+                    case "TSV":
+                        reporters.add(new SepValReporter("\t", path, query));
+                        break;
+                    case "CSV":
+                        reporters.add(new SepValReporter(",", path, query));
+                        break;
+                    default:
+                        throw new RuntimeException(type + " is not a supported reporter type");
+                }
+            }
+        } else {
+            reporters.add(new LegacyReporter());
+        }
+    }
+
+    private long readMemory() {
+        long total = 0;
+        for (MemMeasure mem: memoryBytes.values()) {
+            total += mem.get();
+        }
+        return total;
+    }
+
+    private void startMetricsOutput() {
+        for (MetricResultsReporter reporter: reporters) {
+            reporter.start();
+        }
+    }
+
+    private void finishMetricsOutput() throws Exception {
+        for (MetricResultsReporter reporter: reporters) {
+            reporter.finish(allCombined);
+        }
+    }
+
+    /**
+     * Monitor the list of topologies for the given time frame.
+     * @param execTimeMins how long to monitor for
+     * @param client the client to use when monitoring
+     * @param topoNames the names of the topologies to monitor
+     * @throws Exception on any error
+     */
+    public void monitorFor(double execTimeMins, Nimbus.Iface client, Collection<String> topoNames) throws Exception {
+        startMetricsOutput();
+        long iterations = (long) ((execTimeMins * 60) / reportIntervalSecs);
+        for (int i = 0; i < iterations; i++) {
+            Thread.sleep(reportIntervalSecs * 1000);
+            outputMetrics(client, topoNames);
+        }
+        finishMetricsOutput();
+    }
+
+    private void outputMetrics(Nimbus.Iface client, Collection<String> names) throws Exception {
+        ClusterSummary summary = client.getClusterInfo();
+        Set<String> ids = new HashSet<>();
+        for (TopologySummary ts: summary.get_topologies()) {
+            if (names.contains(ts.get_name())) {
+                ids.add(ts.get_id());
+            }
+        }
+        if (ids.size() != names.size()) {
+            throw new Exception("Could not find all topologies: " + names);
+        }
+        int uptime = 0;
+        long acked = 0;
+        long failed = 0;
+        for (String id: ids) {
+            TopologyInfo info = client.getTopologyInfo(id);
+            uptime = Math.max(uptime, info.get_uptime_secs());
+            for (ExecutorSummary exec : info.get_executors()) {
+                if (exec.get_stats() != null && exec.get_stats().get_specific() != null
+                    && exec.get_stats().get_specific().is_set_spout()) {
+                    SpoutStats stats = exec.get_stats().get_specific().get_spout();
+                    Map<String, Long> failedMap = stats.get_failed().get(":all-time");
+                    Map<String, Long> ackedMap = stats.get_acked().get(":all-time");
+                    if (ackedMap != null) {
+                        for (String key : ackedMap.keySet()) {
+                            if (failedMap != null) {
+                                Long tmp = failedMap.get(key);
+                                if (tmp != null) {
+                                    failed += tmp;
+                                }
+                            }
+                            long ackVal = ackedMap.get(key);
+                            acked += ackVal;
+                        }
+                    }
+                }
+            }
+        }
+        @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
+        long failedThisTime = failed - prevFailed;
+        @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
+        long ackedThisTime = acked - prevAcked;
+        @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
+        long thisTime = uptime - prevUptime;
+        prevUptime = uptime;
+        prevAcked = acked;
+        prevFailed = failed;
+
+        Histogram copy = new Histogram(3600000000000L, 3);;
+        synchronized (histo) {
+            copy.add(histo);
+            histo.reset();
+        }
+        long user = userCpu.getAndSet(0);
+        long sys = systemCpu.getAndSet(0);
+        long gc = gcMs.getAndSet(0);
+        long memBytes = readMemory();
+
+        allCombined.add(new Measurements(uptime, ackedThisTime, thisTime, failedThisTime, copy, user, sys, gc, memBytes, ids));
+        Measurements inWindow = Measurements.combine(allCombined, null, windowLength);
+        for (MetricResultsReporter reporter: reporters) {
+            reporter.reportWindow(inWindow, allCombined);
+        }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void handle(IMetricsConsumer.TaskInfo taskInfo, Collection<IMetricsConsumer.DataPoint> dataPoints) {
+        //crud no simple way to tie this to a given topology :(
+        String worker = taskInfo.srcWorkerHost + ":" + taskInfo.srcWorkerPort;
+        for (IMetricsConsumer.DataPoint dp: dataPoints) {
+            if (dp.name.startsWith("comp-lat-histo") && dp.value instanceof Histogram) {
+                synchronized (histo) {
+                    histo.add((Histogram)dp.value);
+                }
+            } else if ("CPU".equals(dp.name) && dp.value instanceof Map) {
+                Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                Object sys = m.get("sys-ms");
+                if (sys instanceof Number) {
+                    systemCpu.getAndAdd(((Number)sys).longValue());
+                }
+                Object user = m.get("user-ms");
+                if (user instanceof Number) {
+                    userCpu.getAndAdd(((Number)user).longValue());
+                }
+            } else if (dp.name.startsWith("GC/") && dp.value instanceof Map) {
+                Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                Object count = m.get("count");
+                if (count instanceof Number) {
+                    gcCount.getAndAdd(((Number)count).longValue());
+                }
+                Object time = m.get("timeMs");
+                if (time instanceof Number) {
+                    gcMs.getAndAdd(((Number)time).longValue());
+                }
+            } else if (dp.name.startsWith("memory/") && dp.value instanceof Map) {
+                Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                Object val = m.get("usedBytes");
+                if (val instanceof Number) {
+                    MemMeasure mm = memoryBytes.get(worker);
+                    if (mm == null) {
+                        mm = new MemMeasure();
+                        MemMeasure tmp = memoryBytes.putIfAbsent(worker, mm);
+                        mm = tmp == null ? mm : tmp;
+                    }
+                    mm.update(((Number)val).longValue());
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadSpout.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadSpout.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadSpout.java
new file mode 100644
index 0000000..95ade59
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadSpout.java
@@ -0,0 +1,137 @@
+/**
+ * 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.loadgen;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.storm.metrics.hdrhistogram.HistogramMetric;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+
+/**
+ * A spout that simulates a real world spout based off of statistics about it.
+ */
+public class LoadSpout  extends BaseRichSpout {
+    private static class OutputStreamEngineWithHisto extends OutputStreamEngine {
+        public final HistogramMetric histogram;
+
+        public OutputStreamEngineWithHisto(OutputStream stats, TopologyContext context) {
+            super(stats);
+            histogram = new HistogramMetric(3600000000000L, 3);
+            //TODO perhaps we can adjust the frequency later...
+            context.registerMetric("comp-lat-histo-" + stats.id, histogram, 10);
+        }
+    }
+
+    private static class SentWithTime {
+        public final String streamName;
+        public final Values keyValue;
+        public final long time;
+        public final HistogramMetric histogram;
+
+        SentWithTime(String streamName, Values keyValue, long time, HistogramMetric histogram) {
+            this.streamName = streamName;
+            this.keyValue = keyValue;
+            this.time = time;
+            this.histogram = histogram;
+        }
+
+        public void done() {
+            histogram.recordValue(Math.max(0, System.nanoTime() - time));
+        }
+    }
+
+    private final List<OutputStream> streamStats;
+    private List<OutputStreamEngineWithHisto> streams;
+    private SpoutOutputCollector collector;
+    //This is an attempt to give all of the streams an equal opportunity to emit something.
+    private long nextStreamCounter = 0;
+    private final int numStreams;
+
+    /**
+     * Create a simple load spout with just a set rate per second on the default stream.
+     * @param ratePerSecond the rate to send messages at.
+     */
+    public LoadSpout(double ratePerSecond) {
+        OutputStream test = new OutputStream.Builder()
+            .withId("default")
+            .withRate(new NormalDistStats(ratePerSecond, 0.0, ratePerSecond, ratePerSecond))
+            .build();
+        streamStats = Arrays.asList(test);
+        numStreams = 1;
+    }
+
+    public LoadSpout(LoadCompConf conf) {
+        this.streamStats = Collections.unmodifiableList(new ArrayList<>(conf.streams));
+        numStreams = streamStats.size();
+    }
+
+    @Override
+    public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
+        streams = Collections.unmodifiableList(streamStats.stream()
+            .map((ss) -> new OutputStreamEngineWithHisto(ss, context)).collect(Collectors.toList()));
+        this.collector = collector;
+    }
+
+    @Override
+    public void nextTuple() {
+        int size = numStreams;
+        for (int tries = 0; tries < size; tries++) {
+            int index = Math.abs((int) (nextStreamCounter++ % size));
+            OutputStreamEngineWithHisto se = streams.get(index);
+            Long emitTupleTime = se.shouldEmit();
+            if (emitTupleTime != null) {
+                SentWithTime swt =
+                    new SentWithTime(se.streamName, getNextValues(se), emitTupleTime, se.histogram);
+                collector.emit(swt.streamName, swt.keyValue, swt);
+                break;
+            }
+        }
+    }
+
+    protected Values getNextValues(OutputStreamEngine se) {
+        return new Values(se.nextKey(), "JUST_SOME_VALUE");
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        for (OutputStream s: streamStats) {
+            declarer.declareStream(s.id, new Fields("key", "value"));
+        }
+    }
+
+    @Override
+    public void ack(Object id) {
+        ((SentWithTime)id).done();
+    }
+
+    @Override
+    public void fail(Object id) {
+        SentWithTime swt = (SentWithTime)id;
+        collector.emit(swt.streamName, swt.keyValue, swt);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/NormalDistStats.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/NormalDistStats.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/NormalDistStats.java
new file mode 100644
index 0000000..20b2926
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/NormalDistStats.java
@@ -0,0 +1,151 @@
+/**
+ * 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.loadgen;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import org.apache.storm.utils.ObjectReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Stats related to something with a normal distribution, and a way to randomly simulate it.
+ */
+public class NormalDistStats implements Serializable {
+    private static final Logger LOG = LoggerFactory.getLogger(NormalDistStats.class);
+    public final double mean;
+    public final double stddev;
+    public final double min;
+    public final double max;
+
+    /**
+     * Read the stats from a config.
+     * @param conf the config.
+     * @return the corresponding stats.
+     */
+    public static NormalDistStats fromConf(Map<String, Object> conf) {
+        return fromConf(conf, null);
+    }
+
+    /**
+     * Read the stats from a config.
+     * @param conf the config.
+     * @param def the default mean.
+     * @return the corresponding stats.
+     */
+    public static NormalDistStats fromConf(Map<String, Object> conf, Double def) {
+        if (conf == null) {
+            conf = Collections.emptyMap();
+        }
+        double mean = ObjectReader.getDouble(conf.get("mean"), def);
+        double stddev = ObjectReader.getDouble(conf.get("stddev"), mean / 4);
+        double min = ObjectReader.getDouble(conf.get("min"), 0.0);
+        double max = ObjectReader.getDouble(conf.get("max"), Double.MAX_VALUE);
+        return new NormalDistStats(mean, stddev, min, max);
+    }
+
+    /**
+     * Return this as a config.
+     * @return the config version of this.
+     */
+    public Map<String, Object> toConf() {
+        Map<String, Object> ret = new HashMap<>();
+        ret.put("mean", mean);
+        ret.put("stddev", stddev);
+        ret.put("min", min);
+        ret.put("max", max);
+        return ret;
+    }
+
+    /**
+     * Create an instance of this from a list of values.  The metrics will be computed from the values.
+     * @param values the values to compute metrics from.
+     */
+    public NormalDistStats(List<Double> values) {
+        //Compute the stats for these and save them
+        double min = values.isEmpty() ? 0.0 : values.get(0);
+        double max = values.isEmpty() ? 0.0 : values.get(0);
+        double sum = 0.0;
+        long count = values.size();
+        for (Double v: values) {
+            sum += v;
+            min = Math.min(min, v);
+            max = Math.max(max,v);
+        }
+        double mean = sum / Math.max(count, 1);
+        double sdPartial = 0;
+        for (Double v: values) {
+            sdPartial += Math.pow(v - mean, 2);
+        }
+        double stddev = 0.0;
+        if (count >= 2) {
+            stddev = Math.sqrt(sdPartial / (count - 1));
+        }
+        this.min = min;
+        this.max = max;
+        this.mean = mean;
+        this.stddev = stddev;
+        LOG.debug("Stats for {} are {}", values, this);
+    }
+
+    /**
+     * A Constructor for the pre computed stats.
+     * @param mean the mean of the values.
+     * @param stddev the standard deviation of the values.
+     * @param min the min of the values.
+     * @param max the max of the values.
+     */
+    public NormalDistStats(double mean, double stddev, double min, double max) {
+        this.mean = mean;
+        this.stddev = stddev;
+        this.min = min;
+        this.max = max;
+    }
+
+    /**
+     * Generate a random number that follows the statistical distribution
+     * @param rand the random number generator to use
+     * @return the next number that should follow the statistical distribution.
+     */
+    public double nextRandom(Random rand) {
+        return Math.max(Math.min((rand.nextGaussian() * stddev) + mean, max), min);
+    }
+
+    @Override
+    public String toString() {
+        return "mean: " + mean + " min: " + min + " max: " + max + " stddev: " + stddev;
+    }
+
+    /**
+     * Scale the stats by v. This is not scaling everything proportionally.  We don't want the stddev to increase
+     * so instead we scale the mean and shift everything up or down by the same amount.
+     * @param v the amount to scale by 1.0 is nothing 0.5 is half.
+     * @return a copy of this with the needed adjustments.
+     */
+    public NormalDistStats scaleBy(double v) {
+        double newMean = mean * v;
+        double shiftAmount = newMean - mean;
+        return new NormalDistStats(Math.max(0, mean + shiftAmount), stddev,
+            Math.max(0, min + shiftAmount), Math.max(0, max + shiftAmount));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/OutputStream.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/OutputStream.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/OutputStream.java
new file mode 100644
index 0000000..99357d9
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/OutputStream.java
@@ -0,0 +1,121 @@
+/**
+ * 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.loadgen;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import jdk.nashorn.internal.objects.Global;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.loadgen.NormalDistStats;
+
+/**
+ * A set of measurements about a stream so we can statistically reproduce it.
+ */
+public class OutputStream implements Serializable {
+    //The global stream id is this + the from component it must be a part of.
+    public final String id;
+    public final NormalDistStats rate;
+    public final boolean areKeysSkewed;
+
+    /**
+     * Create an output stream from a config.
+     * @param conf the config to read from.
+     * @return the read OutputStream.
+     */
+    public static OutputStream fromConf(Map<String, Object> conf) {
+        String streamId = (String) conf.getOrDefault("streamId", "default");
+        NormalDistStats rate = NormalDistStats.fromConf((Map<String, Object>) conf.get("rate"));
+        boolean areKeysSkewed = (Boolean) conf.getOrDefault("areKeysSkewed", false);
+        return new OutputStream(streamId, rate, areKeysSkewed);
+    }
+
+    /**
+     * Convert this to a conf.
+     * @return the conf.
+     */
+    public Map<String, Object> toConf() {
+        Map<String, Object> ret = new HashMap<>();
+        ret.put("streamId", id);
+        ret.put("rate", rate.toConf());
+        ret.put("areKeysSkewed", areKeysSkewed);
+        return ret;
+    }
+
+    public OutputStream remap(String origId, Map<GlobalStreamId, GlobalStreamId> remappedStreams) {
+        GlobalStreamId remapped = remappedStreams.get(new GlobalStreamId(origId, id));
+        return new OutputStream(remapped.get_streamId(), rate, areKeysSkewed);
+    }
+
+    public OutputStream scaleThroughput(double v) {
+        return new OutputStream(id, rate.scaleBy(v), areKeysSkewed);
+    }
+
+    public static class Builder {
+        private String id;
+        private NormalDistStats rate;
+        private boolean areKeysSkewed;
+
+        public String getId() {
+            return id;
+        }
+
+        public Builder withId(String id) {
+            this.id = id;
+            return this;
+        }
+
+        public NormalDistStats getRate() {
+            return rate;
+        }
+
+        public Builder withRate(NormalDistStats rate) {
+            this.rate = rate;
+            return this;
+        }
+
+        public boolean isAreKeysSkewed() {
+            return areKeysSkewed;
+        }
+
+        public Builder withAreKeysSkewed(boolean areKeysSkewed) {
+            this.areKeysSkewed = areKeysSkewed;
+            return this;
+        }
+
+        public OutputStream build() {
+            return new OutputStream(id, rate, areKeysSkewed);
+        }
+    }
+
+    /**
+     * Create a new stream with stats
+     * @param id the id of the stream
+     * @param rate the rate of tuples being emitted on this stream
+     * @param areKeysSkewed true if keys are skewed else false.  For skewed keys
+     *     we only simulate it by using a gaussian distribution to the keys instead
+     *     of an even distribution.  Tere is no effort made right not to measure the
+     *     skewness and reproduce it.
+     */
+    public OutputStream(String id, NormalDistStats rate, boolean areKeysSkewed) {
+        this.id = id;
+        this.rate = rate;
+        this.areKeysSkewed = areKeysSkewed;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/OutputStreamEngine.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/OutputStreamEngine.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/OutputStreamEngine.java
new file mode 100644
index 0000000..80111c8
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/OutputStreamEngine.java
@@ -0,0 +1,122 @@
+/**
+ * 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.loadgen;
+
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Provides an API to simulate the output of a stream.
+ * <p>
+ * Right now it is just rate, but in the future we expect to do data skew as well...
+ * </p>
+ */
+public class OutputStreamEngine {
+    private static final double NANO_PER_SEC = 1_000_000_000.0;
+    private static final long UPDATE_RATE_PERIOD_NS = ((long)NANO_PER_SEC * 2);
+    private static final String[] KEYS = new String[2048];
+
+    static {
+        //We get a new random number and seed it to make sure that runs are consistent where possible.
+        Random r = new Random(KEYS.length);
+        for (int i = 0; i < KEYS.length; i++) {
+            KEYS[i] = String.valueOf(r.nextDouble());
+        }
+    }
+
+    private long periodNano;
+    private long emitAmount;
+    private final Random rand;
+    private long nextEmitTime;
+    private long nextRateRandomizeTime;
+    private long emitsLeft;
+    private final OutputStream stats;
+    public final String streamName;
+
+    /**
+     * Create an engine that can simulate the given stats.
+     * @param stats the stats to follow
+     */
+    public OutputStreamEngine(OutputStream stats) {
+        this.stats = stats;
+        rand = ThreadLocalRandom.current();
+        selectNewRate();
+        //Start emitting right now
+        nextEmitTime = System.nanoTime();
+        nextRateRandomizeTime = nextEmitTime + UPDATE_RATE_PERIOD_NS;
+        emitsLeft = emitAmount;
+        streamName = stats.id;
+    }
+
+    private void selectNewRate() {
+        double ratePerSecond = stats.rate.nextRandom(rand);
+        if (ratePerSecond > 0) {
+            periodNano = Math.max(1, (long)(NANO_PER_SEC / ratePerSecond));
+            emitAmount = Math.max(1, (long)((ratePerSecond / NANO_PER_SEC) * periodNano));
+        } else {
+            //if it is is 0 or less it really is 1 per 10 seconds.
+            periodNano = (long)NANO_PER_SEC * 10;
+            emitAmount = 1;
+        }
+    }
+
+    /**
+     * Should we emit or not.
+     * @return the start time of the message, or null of nothing should be emitted.
+     */
+    public Long shouldEmit() {
+        long time = System.nanoTime();
+        if (emitsLeft <= 0 && nextEmitTime <= time) {
+            emitsLeft = emitAmount;
+            nextEmitTime = nextEmitTime + periodNano;
+        }
+
+        if (nextRateRandomizeTime <= time) {
+            //Once every UPDATE_RATE_PERIOD_NS
+            selectNewRate();
+            nextRateRandomizeTime = nextEmitTime + UPDATE_RATE_PERIOD_NS;
+        }
+
+        if (emitsLeft > 0) {
+            emitsLeft--;
+            return nextEmitTime - periodNano;
+        }
+        return null;
+    }
+
+    /**
+     * Get the next key to emit.
+     * @return the key that should be emitted.
+     */
+    public String nextKey() {
+        int keyIndex;
+        if (stats.areKeysSkewed) {
+            //We set the stddev of the skewed keys to be 1/5 of the length, but then we use the absolute value
+            // of that so everything is skewed towards 0
+            keyIndex = Math.min(KEYS.length - 1 , Math.abs((int)(rand.nextGaussian() * KEYS.length / 5)));
+        } else {
+            keyIndex = rand.nextInt(KEYS.length);
+        }
+        return KEYS[keyIndex];
+    }
+
+    public int nextInt(int bound) {
+        return rand.nextInt(bound);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ScopedTopologySet.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ScopedTopologySet.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ScopedTopologySet.java
new file mode 100644
index 0000000..1b6ed74
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ScopedTopologySet.java
@@ -0,0 +1,91 @@
+/**
+ * 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.loadgen;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.storm.generated.KillOptions;
+import org.apache.storm.generated.Nimbus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A set of topology names that will be killed when this is closed.
+ */
+public class ScopedTopologySet extends HashSet<String> implements AutoCloseable {
+    private static final Logger LOG = LoggerFactory.getLogger(ScopedTopologySet.class);
+    private static final KillOptions NO_WAIT_KILL = new KillOptions();
+
+    static {
+        NO_WAIT_KILL.set_wait_secs(0);
+    }
+
+    private final Nimbus.Iface client;
+    private final Set<String> unmodWrapper;
+
+    public ScopedTopologySet(Nimbus.Iface client) {
+        this.client = client;
+        unmodWrapper = Collections.unmodifiableSet(this);
+    }
+
+    @Override
+    public boolean remove(Object o) {
+        throw new RuntimeException("Unmodifiable Set");
+    }
+
+    @Override
+    public void clear() {
+        throw new RuntimeException("Unmodifiable Set");
+    }
+
+    @Override
+    public boolean removeAll(Collection<?> c) {
+        throw new RuntimeException("Unmodifiable Set");
+    }
+
+    @Override
+    public boolean retainAll(Collection<?> c) {
+        throw new RuntimeException("Unmodifiable Set");
+    }
+
+    @Override
+    public void close() {
+        RuntimeException saved = null;
+        for (Iterator<String> it = super.iterator(); it.hasNext();) {
+            String name = it.next();
+            try {
+                client.killTopologyWithOpts(name, NO_WAIT_KILL);
+            } catch (Exception e) {
+                RuntimeException wrapped = new RuntimeException("Error trying to kill " + name, e);
+                if (saved != null) {
+                    saved.addSuppressed(wrapped);
+                } else {
+                    saved = wrapped;
+                }
+            }
+        }
+        super.clear();
+        if (saved != null) {
+            throw saved;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
index 96c13c5..2c22b42 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/ThroughputVsLatency.java
@@ -15,114 +15,67 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.storm.starter;
 
-import java.util.Collection;
+package org.apache.storm.loadgen;
+
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicLong;
 
-import org.HdrHistogram.Histogram;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
 import org.apache.storm.Config;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.ClusterSummary;
-import org.apache.storm.generated.ExecutorSummary;
-import org.apache.storm.generated.KillOptions;
-import org.apache.storm.generated.Nimbus;
-import org.apache.storm.generated.SpoutStats;
-import org.apache.storm.generated.TopologyInfo;
-import org.apache.storm.generated.TopologySummary;
-import org.apache.storm.metric.api.IMetricsConsumer.DataPoint;
-import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
-import org.apache.storm.metrics.hdrhistogram.HistogramMetric;
-import org.apache.storm.misc.metric.HttpForwardingMetricsServer;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.TopologyContext;
+import org.apache.storm.metric.LoggingMetricsConsumer;
 import org.apache.storm.topology.BasicOutputCollector;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseBasicBolt;
-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.NimbusClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * WordCount but the spout goes at a predefined rate and we collect
  * proper latency statistics.
  */
 public class ThroughputVsLatency {
-    private static class SentWithTime {
-        public final String sentence;
-        public final long time;
-
-        SentWithTime(String sentence, long time) {
-            this.sentence = sentence;
-            this.time = time;
-        }
-    }
-
-    public static class FastRandomSentenceSpout extends BaseRichSpout {
-        static final String[] SENTENCES = new String[]{ "the cow jumped over the moon", "an apple a day keeps the doctor away",
-                "four score and seven years ago", "snow white and the seven dwarfs", "i am at two with nature" };
-
-        SpoutOutputCollector _collector;
-        long _periodNano;
-        long _emitAmount;
-        Random _rand;
-        long _nextEmitTime;
-        long _emitsLeft;
-        HistogramMetric _histo;
+    private static final Logger LOG = LoggerFactory.getLogger(GenLoad.class);
+    private static final int TEST_EXECUTE_TIME_DEFAULT = 5;
+    private static final long DEFAULT_RATE_PER_SECOND = 500;
+    private static final String DEFAULT_TOPO_NAME = "wc-test";
+    private static final int DEFAULT_NUM_SPOUTS = 1;
+    private static final int DEFAULT_NUM_SPLITS = 1;
+    private static final int DEFAULT_NUM_COUNTS = 1;
+
+    public static class FastRandomSentenceSpout extends LoadSpout {
+        static final String[] SENTENCES = new String[] {
+            "the cow jumped over the moon",
+            "an apple a day keeps the doctor away",
+            "four score and seven years ago",
+            "snow white and the seven dwarfs",
+            "i am at two with nature"
+        };
 
+        /**
+         * Constructor.
+         * @param ratePerSecond the rate to emite tuples at.
+         */
         public FastRandomSentenceSpout(long ratePerSecond) {
-            if (ratePerSecond > 0) {
-                _periodNano = Math.max(1, 1000000000/ratePerSecond);
-                _emitAmount = Math.max(1, (long)((ratePerSecond / 1000000000.0) * _periodNano));
-            } else {
-                _periodNano = Long.MAX_VALUE - 1;
-                _emitAmount = 1;
-            }
+            super(ratePerSecond);
         }
 
         @Override
-        public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
-            _collector = collector;
-            _rand = ThreadLocalRandom.current();
-            _nextEmitTime = System.nanoTime();
-            _emitsLeft = _emitAmount;
-            _histo = new HistogramMetric(3600000000000L, 3);
-            context.registerMetric("comp-lat-histo", _histo, 10); //Update every 10 seconds, so we are not too far behind
-        }
-
-        @Override
-        public void nextTuple() {
-            if (_emitsLeft <= 0 && _nextEmitTime <= System.nanoTime()) {
-                _emitsLeft = _emitAmount;
-                _nextEmitTime = _nextEmitTime + _periodNano;
-            }
-
-            if (_emitsLeft > 0) {
-                String sentence = SENTENCES[_rand.nextInt(SENTENCES.length)];
-                _collector.emit(new Values(sentence), new SentWithTime(sentence, _nextEmitTime - _periodNano));
-                _emitsLeft--;
-            }
-        }
-
-        @Override
-        public void ack(Object id) {
-            long end = System.nanoTime();
-            SentWithTime st = (SentWithTime)id;
-            _histo.recordValue(end-st.time);
-        }
-
-        @Override
-        public void fail(Object id) {
-            SentWithTime st = (SentWithTime)id;
-            _collector.emit(new Values(st.sentence), id);
+        protected Values getNextValues(OutputStreamEngine se) {
+            String sentence = SENTENCES[se.nextInt(SENTENCES.length)];
+            return new Values(sentence);
         }
 
         @Override
@@ -147,14 +100,15 @@ public class ThroughputVsLatency {
     }
 
     public static class WordCount extends BaseBasicBolt {
-        Map<String, Integer> counts = new HashMap<String, Integer>();
+        Map<String, Integer> counts = new HashMap<>();
 
         @Override
         public void execute(Tuple tuple, BasicOutputCollector collector) {
             String word = tuple.getString(0);
             Integer count = counts.get(word);
-            if (count == null)
+            if (count == null) {
                 count = 0;
+            }
             count++;
             counts.put(word, count);
             collector.emit(new Values(word, count));
@@ -166,184 +120,103 @@ public class ThroughputVsLatency {
         }
     }
 
-    private static class MemMeasure {
-        private long _mem = 0;
-        private long _time = 0;
-
-        public synchronized void update(long mem) {
-            _mem = mem;
-            _time = System.currentTimeMillis();
-        }
-
-        public synchronized long get() {
-            return isExpired() ? 0l : _mem;
-        }
-
-        public synchronized boolean isExpired() {
-            return (System.currentTimeMillis() - _time) >= 20000;
-        }
-    }
-
-    private static final Histogram _histo = new Histogram(3600000000000L, 3);
-    private static final AtomicLong _systemCPU = new AtomicLong(0);
-    private static final AtomicLong _userCPU = new AtomicLong(0);
-    private static final AtomicLong _gcCount = new AtomicLong(0);
-    private static final AtomicLong _gcMs = new AtomicLong(0);
-    private static final ConcurrentHashMap<String, MemMeasure> _memoryBytes = new ConcurrentHashMap<String, MemMeasure>();
-
-    private static long readMemory() {
-        long total = 0;
-        for (MemMeasure mem: _memoryBytes.values()) {
-            total += mem.get();
-        }
-        return total;
-    }
-
-    private static long _prev_acked = 0;
-    private static long _prev_uptime = 0;
-
-    public static void printMetrics(Nimbus.Iface client, String name) throws Exception {
-        ClusterSummary summary = client.getClusterInfo();
-        String id = null;
-        for (TopologySummary ts: summary.get_topologies()) {
-            if (name.equals(ts.get_name())) {
-                id = ts.get_id();
+    /**
+     * The main entry point for ThroughputVsLatency.
+     * @param args the command line args
+     * @throws Exception on any error.
+     */
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+        options.addOption(Option.builder("h")
+            .longOpt("help")
+            .desc("Print a help message")
+            .build());
+        options.addOption(Option.builder("t")
+            .longOpt("test-time")
+            .argName("MINS")
+            .hasArg()
+            .desc("How long to run the tests for in mins (defaults to " + TEST_EXECUTE_TIME_DEFAULT + ")")
+            .build());
+        options.addOption(Option.builder()
+            .longOpt("rate")
+            .argName("SENTENCES/SEC")
+            .hasArg()
+            .desc("How many sentences per second to run. (defaults to " + DEFAULT_RATE_PER_SECOND + ")")
+            .build());
+        options.addOption(Option.builder()
+            .longOpt("name")
+            .argName("TOPO_NAME")
+            .hasArg()
+            .desc("Name of the topology to run (defaults to " + DEFAULT_TOPO_NAME + ")")
+            .build());
+        options.addOption(Option.builder()
+            .longOpt("spouts")
+            .argName("NUM")
+            .hasArg()
+            .desc("Number of spouts to use (defaults to " + DEFAULT_NUM_SPOUTS + ")")
+            .build());
+        options.addOption(Option.builder()
+            .longOpt("splitters")
+            .argName("NUM")
+            .hasArg()
+            .desc("Number of splitter bolts to use (defaults to " + DEFAULT_NUM_SPLITS + ")")
+            .build());
+        options.addOption(Option.builder()
+            .longOpt("counters")
+            .argName("NUM")
+            .hasArg()
+            .desc("Number of counter bolts to use (defaults to " + DEFAULT_NUM_COUNTS + ")")
+            .build());
+        LoadMetricsServer.addCommandLineOptions(options);
+        CommandLineParser parser = new DefaultParser();
+        CommandLine cmd = null;
+        Exception commandLineException = null;
+        double numMins = TEST_EXECUTE_TIME_DEFAULT;
+        double ratePerSecond = DEFAULT_RATE_PER_SECOND;
+        String name = DEFAULT_TOPO_NAME;
+        int numSpouts = DEFAULT_NUM_SPOUTS;
+        int numSplits = DEFAULT_NUM_SPLITS;
+        int numCounts = DEFAULT_NUM_COUNTS;
+        try {
+            cmd = parser.parse(options, args);
+            if (cmd.hasOption("t")) {
+                numMins = Double.valueOf(cmd.getOptionValue("t"));
             }
-        }
-        if (id == null) {
-            throw new Exception("Could not find a topology named "+name);
-        }
-        TopologyInfo info = client.getTopologyInfo(id);
-        int uptime = info.get_uptime_secs();
-        long acked = 0;
-        long failed = 0;
-        for (ExecutorSummary exec: info.get_executors()) {
-            if ("spout".equals(exec.get_component_id()) && exec.get_stats() != null && exec.get_stats().get_specific() != null) {
-                SpoutStats stats = exec.get_stats().get_specific().get_spout();
-                Map<String, Long> failedMap = stats.get_failed().get(":all-time");
-                Map<String, Long> ackedMap = stats.get_acked().get(":all-time");
-                if (ackedMap != null) {
-                    for (String key: ackedMap.keySet()) {
-                        if (failedMap != null) {
-                            Long tmp = failedMap.get(key);
-                            if (tmp != null) {
-                                failed += tmp;
-                            }
-                        }
-                        long ackVal = ackedMap.get(key);
-                        acked += ackVal;
-                    }
-                }
+            if (cmd.hasOption("rate")) {
+                ratePerSecond = Double.parseDouble(cmd.getOptionValue("rate"));
             }
+            if (cmd.hasOption("name")) {
+                name = cmd.getOptionValue("name");
+            }
+            if (cmd.hasOption("spouts")) {
+                numSpouts = Integer.parseInt(cmd.getOptionValue("spouts"));
+            }
+            if (cmd.hasOption("splitters")) {
+                numSplits = Integer.parseInt(cmd.getOptionValue("splitters"));
+            }
+            if (cmd.hasOption("counters")) {
+                numCounts = Integer.parseInt(cmd.getOptionValue("counters"));
+            }
+        } catch (ParseException | NumberFormatException e) {
+            commandLineException = e;
         }
-        long ackedThisTime = acked - _prev_acked;
-        long thisTime = uptime - _prev_uptime;
-        long nnpct, nnnpct, min, max;
-        double mean, stddev;
-        synchronized(_histo) {
-            nnpct = _histo.getValueAtPercentile(99.0);
-            nnnpct = _histo.getValueAtPercentile(99.9);
-            min = _histo.getMinValue();
-            max = _histo.getMaxValue();
-            mean = _histo.getMean();
-            stddev = _histo.getStdDeviation();
-            _histo.reset();
-        }
-        long user = _userCPU.getAndSet(0);
-        long sys = _systemCPU.getAndSet(0);
-        long gc = _gcMs.getAndSet(0);
-        double memMB = readMemory() / (1024.0 * 1024.0);
-        System.out.printf("uptime: %,4d acked: %,9d acked/sec: %,10.2f failed: %,8d " +
-                "99%%: %,15d 99.9%%: %,15d min: %,15d max: %,15d mean: %,15.2f " +
-                "stddev: %,15.2f user: %,10d sys: %,10d gc: %,10d mem: %,10.2f\n",
-                uptime, ackedThisTime, (((double)ackedThisTime)/thisTime), failed, nnpct, nnnpct,
-                min, max, mean, stddev, user, sys, gc, memMB);
-        _prev_uptime = uptime;
-        _prev_acked = acked;
-    }
-
-    public static void kill(Nimbus.Iface client, String name) throws Exception {
-        KillOptions opts = new KillOptions();
-        opts.set_wait_secs(0);
-        client.killTopologyWithOpts(name, opts);
-    }
-
-    public static void main(String[] args) throws Exception {
-        long ratePerSecond = 500;
-        if (args != null && args.length > 0) {
-            ratePerSecond = Long.valueOf(args[0]);
-        }
-
-        int parallelism = 4;
-        if (args != null && args.length > 1) {
-            parallelism = Integer.valueOf(args[1]);
-        }
-
-        int numMins = 5;
-        if (args != null && args.length > 2) {
-            numMins = Integer.valueOf(args[2]);
-        }
-
-        String name = "wc-test";
-        if (args != null && args.length > 3) {
-            name = args[3];
+        if (commandLineException != null || cmd.hasOption('h')) {
+            if (commandLineException != null) {
+                System.err.println("ERROR " + commandLineException.getMessage());
+            }
+            new HelpFormatter().printHelp("ThroughputVsLatency [options]", options);
+            return;
         }
 
         Config conf = new Config();
-        HttpForwardingMetricsServer metricServer = new HttpForwardingMetricsServer(conf) {
-            @Override
-            public void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
-                String worker = taskInfo.srcWorkerHost + ":" + taskInfo.srcWorkerPort;
-                for (DataPoint dp: dataPoints) {
-                    if ("comp-lat-histo".equals(dp.name) && dp.value instanceof Histogram) {
-                        synchronized(_histo) {
-                            _histo.add((Histogram)dp.value);
-                        }
-                    } else if ("CPU".equals(dp.name) && dp.value instanceof Map) {
-                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
-                        Object sys = m.get("sys-ms");
-                        if (sys instanceof Number) {
-                            _systemCPU.getAndAdd(((Number)sys).longValue());
-                        }
-                        Object user = m.get("user-ms");
-                        if (user instanceof Number) {
-                            _userCPU.getAndAdd(((Number)user).longValue());
-                        }
-                    } else if (dp.name.startsWith("GC/") && dp.value instanceof Map) {
-                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
-                        Object count = m.get("count");
-                        if (count instanceof Number) {
-                            _gcCount.getAndAdd(((Number)count).longValue());
-                        }
-                        Object time = m.get("timeMs");
-                        if (time instanceof Number) {
-                            _gcMs.getAndAdd(((Number)time).longValue());
-                        }
-                    } else if (dp.name.startsWith("memory/") && dp.value instanceof Map) {
-                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
-                        Object val = m.get("usedBytes");
-                        if (val instanceof Number) {
-                            MemMeasure mm = _memoryBytes.get(worker);
-                            if (mm == null) {
-                                mm = new MemMeasure();
-                                MemMeasure tmp = _memoryBytes.putIfAbsent(worker, mm);
-                                mm = tmp == null ? mm : tmp; 
-                            }
-                            mm.update(((Number)val).longValue());
-                        }
-                    }
-                }
-            }
-        };
-
+        LoadMetricsServer metricServer = new LoadMetricsServer(conf, cmd);
         metricServer.serve();
         String url = metricServer.getUrl();
 
         NimbusClient client = NimbusClient.getConfiguredClient(conf);
-        conf.setNumWorkers(parallelism);
-        conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class);
-        conf.registerMetricsConsumer(org.apache.storm.misc.metric.HttpForwardingMetricsConsumer.class, url, 1);
-        Map<String, String> workerMetrics = new HashMap<String, String>();
+        conf.registerMetricsConsumer(LoggingMetricsConsumer.class);
+        conf.registerMetricsConsumer(HttpForwardingMetricsConsumer.class, url, 1);
+        Map<String, String> workerMetrics = new HashMap<>();
         if (!NimbusClient.isLocalOverride()) {
             //sigar uses JNI and does not work in local mode
             workerMetrics.put("CPU", "org.apache.storm.metrics.sigar.CPUMetric");
@@ -351,27 +224,27 @@ public class ThroughputVsLatency {
         conf.put(Config.TOPOLOGY_WORKER_METRICS, workerMetrics);
         conf.put(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS, 10);
         conf.put(Config.TOPOLOGY_WORKER_GC_CHILDOPTS,
-                "-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:NewSize=128m -XX:CMSInitiatingOccupancyFraction=70 -XX:-CMSConcurrentMTEnabled");
+            "-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+UseConcMarkSweepGC "
+                + "-XX:NewSize=128m -XX:CMSInitiatingOccupancyFraction=70 -XX:-CMSConcurrentMTEnabled");
         conf.put(Config.TOPOLOGY_WORKER_CHILDOPTS, "-Xmx2g");
 
         TopologyBuilder builder = new TopologyBuilder();
 
-        int numEach = 4 * parallelism;
-        builder.setSpout("spout", new FastRandomSentenceSpout(ratePerSecond/numEach), numEach);
+        builder.setSpout("spout", new FastRandomSentenceSpout((long) ratePerSecond / numSpouts), numSpouts);
+        builder.setBolt("split", new SplitSentence(), numSplits).shuffleGrouping("spout");
+        builder.setBolt("count", new WordCount(), numCounts).fieldsGrouping("split", new Fields("word"));
 
-        builder.setBolt("split", new SplitSentence(), numEach).shuffleGrouping("spout");
-        builder.setBolt("count", new WordCount(), numEach).fieldsGrouping("split", new Fields("word"));
-
-        try {
+        int exitStatus = -1;
+        try (ScopedTopologySet topologyNames = new ScopedTopologySet(client.getClient())) {
             StormSubmitter.submitTopology(name, conf, builder.createTopology());
+            topologyNames.add(name);
 
-            for (int i = 0; i < numMins * 2; i++) {
-                Thread.sleep(30 * 1000);
-                printMetrics(client.getClient(), name);
-            }
+            metricServer.monitorFor(numMins, client.getClient(), topologyNames);
+            exitStatus = 0;
+        } catch (Exception e) {
+            LOG.error("Error while running test", e);
         } finally {
-            kill(client.getClient(), name);
-            System.exit(0);
+            System.exit(exitStatus);
         }
     }
 }


[14/18] storm git commit: Added in a skipped max spout penting metric

Posted by bo...@apache.org.
Added in a skipped max spout penting metric


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/96e68c02
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/96e68c02
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/96e68c02

Branch: refs/heads/master
Commit: 96e68c02453eb1884785db1b32e387782f576a29
Parents: 2f5cde8
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Sep 1 09:47:40 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Sep 1 09:47:40 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                    |  3 ++-
 .../org/apache/storm/loadgen/LoadMetricsServer.java | 16 ++++++++++++++--
 2 files changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/96e68c02/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index 52f5358..6c476bd 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -118,7 +118,7 @@ There are a lot of different metrics supported
 |end_time| The ending time of the metrics window from the the first topology was launched. | all
 |time_window| the length in seconds for the time window. | all
 |ids| The topology ids that are being tracked | all
-|congested| Componets that appear to be congested | all
+|congested| Components that appear to be congested | all
 |storm_version| The version of storm as reported by the client | all
 |java_version| The version of java as reported by the client | all
 |os_arch| The OS architecture as reported by the client | all
@@ -128,6 +128,7 @@ There are a lot of different metrics supported
 |hosts| The number of hosts the monitored topologies are running on| all
 |executors| The number of running executors in the monitored topologies | all
 |workers| The number of workers the monitored topologies are running on | all
+|skipped\_max\_spout| The number of ms in total that the spout reported it skipped trying to emit because of `topology.max.spout.pending`. This is the sum for all spouts and can be used to decide if setting the value higher will likely improve throughput. `congested` reports individual spouts that appear to be slowed down by this to a large degree. | all
 |target_rate| The target rate in sentences per second for the ThroughputVsLatency topology | ThroughputVsLatency
 |spout_parallel| The parallelism of the spout for the `ThroughputVsLatency` topology. | ThroughputVsLatency
 |split_parallel| The parallelism of the split bolt for the `ThroughputVsLatency` topology. | ThroughputVsLatency

http://git-wip-us.apache.org/repos/asf/storm/blob/96e68c02/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
index 31bea1f..e6c1616 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -91,6 +91,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
 
     public static class Measurements {
         private final Histogram histo;
+        private long skippedMaxSpoutMs;
         private double userMs;
         private double sysMs;
         private double gcMs;
@@ -114,7 +115,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
          */
         public Measurements(long uptimeSecs, long acked, long timeWindow, long failed, Histogram histo,
                             double userMs, double sysMs, double gcMs, long memBytes, Set<String> topologyIds,
-                            long workers, long executors, long hosts, Map<String, String> congested) {
+                            long workers, long executors, long hosts, Map<String, String> congested, long skippedMaxSpoutMs) {
             this.uptimeSecs = uptimeSecs;
             this.acked = acked;
             this.timeWindow = timeWindow;
@@ -129,6 +130,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             this.executors = executors;
             this.hosts = hosts;
             this.congested = congested;
+            this.skippedMaxSpoutMs = skippedMaxSpoutMs;
         }
 
         /**
@@ -149,6 +151,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             executors = 0;
             hosts = 0;
             congested = new HashMap<>();
+            skippedMaxSpoutMs = 0;
         }
 
         /**
@@ -170,6 +173,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             executors = Math.max(executors, other.executors);
             hosts = Math.max(hosts, other.hosts);
             congested.putAll(other.congested);
+            skippedMaxSpoutMs += other.skippedMaxSpoutMs;
         }
 
         public double getLatencyAtPercentile(double percential, TimeUnit unit) {
@@ -204,6 +208,10 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             return convert(gcMs, TimeUnit.MILLISECONDS, unit);
         }
 
+        public double getSkippedMaxSpout(TimeUnit unit) {
+            return convert(skippedMaxSpoutMs, TimeUnit.MILLISECONDS, unit);
+        }
+
         public double getMemMb() {
             return memBytes / (1024.0 * 1024.0);
         }
@@ -397,6 +405,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         tmp.put("user_cpu", new MetricExtractor((m, unit) -> m.getUserTime(unit)));
         tmp.put("sys_cpu", new MetricExtractor((m, unit) -> m.getSysTime(unit)));
         tmp.put("gc_cpu", new MetricExtractor((m, unit) -> m.getGc(unit)));
+        tmp.put("skipped_max_spout", new MetricExtractor((m, unit) -> m.getSkippedMaxSpout(unit)));
         tmp.put("acked",  new MetricExtractor((m, unit) -> m.getAcked(), ""));
         tmp.put("acked_rate",  new MetricExtractor((m, unit) -> m.getAckedPerSec(), "tuple/s"));
         tmp.put("completed",  new MetricExtractor((m, unit) -> m.getCompleted(), ""));
@@ -730,6 +739,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
     private final AtomicLong userCpu = new AtomicLong(0);
     private final AtomicLong gcCount = new AtomicLong(0);
     private final AtomicLong gcMs = new AtomicLong(0);
+    private final AtomicLong skippedMaxSpoutMs = new AtomicLong(0);
     private final ConcurrentHashMap<String, MemMeasure> memoryBytes = new ConcurrentHashMap<>();
     private final AtomicReference<ConcurrentHashMap<String, String>> congested = new AtomicReference<>(new ConcurrentHashMap<>());
     private final List<MetricResultsReporter> reporters;
@@ -909,10 +919,11 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         long user = userCpu.getAndSet(0);
         long sys = systemCpu.getAndSet(0);
         long gc = gcMs.getAndSet(0);
+        long skippedMaxSpout = skippedMaxSpoutMs.getAndSet(0);
         long memBytes = readMemory();
 
         allCombined.add(new Measurements(uptime, ackedThisTime, thisTime, failedThisTime, copy, user, sys, gc, memBytes,
-            ids, workers.size(), executors, hosts.size(), congested.getAndSet(new ConcurrentHashMap<>())));
+            ids, workers.size(), executors, hosts.size(), congested.getAndSet(new ConcurrentHashMap<>()), skippedMaxSpout));
         Measurements inWindow = Measurements.combine(allCombined, null, windowLength);
         for (MetricResultsReporter reporter: reporters) {
             reporter.reportWindow(inWindow, allCombined);
@@ -974,6 +985,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                 }
             } else if (dp.name.equals("__skipped-max-spout-ms")) {
                 if (dp.value instanceof Number) {
+                    skippedMaxSpoutMs.getAndAdd(((Number) dp.value).longValue());
                     double full = ((Number) dp.value).doubleValue() / 10_000.0; //The frequency of reporting
                     if (full >= 0.8) {
                         congested.get().put(


[16/18] storm git commit: Added in precision for csv, and tsv too

Posted by bo...@apache.org.
Added in precision for csv, and tsv too


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/113413cd
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/113413cd
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/113413cd

Branch: refs/heads/master
Commit: 113413cddefce23f920ec12733321284d31ce9d7
Parents: 19508b9
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Sep 1 10:44:34 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Sep 1 10:44:34 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                |  2 +-
 .../apache/storm/loadgen/LoadMetricsServer.java | 33 ++++++++++++++++----
 2 files changed, 28 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/113413cd/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index 1858ea4..b24b8ed 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -89,8 +89,8 @@ Not all options are supported by all reporters.
 |columns | A comma separated list of columns to output (see below for the metrics supported).  A `*` is replaced by all metrics. Defaults to "start_time", "end_time", "rate", "mean", "99%ile", "99.9%ile", "cores", "mem", "failed", "ids", "congested" | csv, tsv, fixed |
 |extraColumns | Like columns but ones that should be added to the defaults instead of replacing them. A `*` is replaced by all metrics. | csv, tsv, fixed |
 |meta | An arbitrary string that will appear as a "meta" column at the end.  This helps when appending to files to keep different runs separated | csv, tsv, fixed|
+|precision | The number of places after the decimal point to print out.  The default for fixed is 3, all others it is unlimited. | csv, tsv, fixed|
 |columnWidth | The width of each field | fixed|
-|precision | The number of places after the decimal point to print out | fixed|
 
 There are a lot of different metrics supported
 

http://git-wip-us.apache.org/repos/asf/storm/blob/113413cd/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
index a44ab45..546b3ca 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -476,9 +476,16 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         protected final TimeUnit targetUnit;
         protected final List<String> extractors;
         protected final String meta;
+        protected final int precision;
+        protected String doubleFormat;
 
         public ColumnsFileReporter(String path, Map<String, String> query, Map<String, MetricExtractor> extractorsMap)
             throws FileNotFoundException {
+            this(path, query, extractorsMap, null);
+        }
+
+        public ColumnsFileReporter(String path, Map<String, String> query, Map<String, MetricExtractor> extractorsMap,
+                                   String defaultPreceision) throws FileNotFoundException {
             super(path, query, extractorsMap);
             targetUnit = UNIT_MAP.get(query.getOrDefault("time", "MILLISECONDS").toUpperCase());
             if (targetUnit == null) {
@@ -511,7 +518,14 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                     }
                 }
             }
-
+            String strPrecision = query.getOrDefault("precision", defaultPreceision);
+            if (strPrecision == null) {
+                precision = -1;
+                doubleFormat = "%f";
+            } else {
+                precision = Integer.parseInt(strPrecision);
+                doubleFormat = "%." + precision + "f";
+            }
             meta = query.get("meta");
         }
 
@@ -536,19 +550,25 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             }
             return ret;
         }
+
+        protected String format(Object o) {
+            if (o instanceof Double || o instanceof Float) {
+                return String.format(doubleFormat, o);
+            } else {
+                return o == null ? "" : o.toString();
+            }
+        }
     }
 
 
     static class FixedWidthReporter extends  ColumnsFileReporter {
-        public final String doubleFormat;
         public final String longFormat;
         public final String stringFormat;
 
         public FixedWidthReporter(String path, Map<String, String> query, Map<String, MetricExtractor> extractorsMap)
             throws FileNotFoundException {
-            super(path, query, extractorsMap);
+            super(path, query, extractorsMap, "3");
             int columnWidth = Integer.parseInt(query.getOrDefault("columnWidth", "15")) - 1;//Always have a space in between
-            int precision = Integer.parseInt(query.getOrDefault("precision", "3"));
             doubleFormat = "%," + columnWidth + "." + precision + "f";
             longFormat = "%," + columnWidth + "d";
             stringFormat = "%" + columnWidth + "s";
@@ -558,7 +578,8 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             this(null, Collections.emptyMap(), allExtractors);
         }
 
-        private String format(Object o) {
+        @Override
+        protected String format(Object o) {
             if (o instanceof Double || o instanceof Float) {
                 return String.format(doubleFormat, o);
             } else if (o instanceof Integer || o instanceof Long) {
@@ -638,7 +659,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                 }
                 first = false;
                 Object value = allExtractors.get(name).get(m, targetUnit);
-                String svalue = value == null ? "" : value.toString();
+                String svalue = format(value);
                 out.print(escape(svalue));
             }
             if (meta != null) {


[09/18] storm git commit: STORM-2702: Addressed the rest of the review comments

Posted by bo...@apache.org.
STORM-2702: Addressed the rest of the review comments


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/0e3fc5a0
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/0e3fc5a0
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/0e3fc5a0

Branch: refs/heads/master
Commit: 0e3fc5a097d156bad4cf17c74c147255575aa646
Parents: 6bc3213
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Thu Aug 24 11:13:08 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Thu Aug 24 11:13:08 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                |  21 ++--
 .../apache/storm/loadgen/LoadMetricsServer.java | 116 ++++++++++++++++---
 2 files changed, 116 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0e3fc5a0/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index 50a618f..1e5c69d 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -72,9 +72,12 @@ storm jar storm-loadgen.jar org.apache.storm.loadgen.ThroughputVsLatency [option
 # Reporters
 Reporters provide a way to store various statistics about a running topology. There are currently a few supported reporters
 
- * legacy - report values like ThroughputVsLatency has done in the past
- * TSV - tab separated values
- * CSV - comma separated values
+ * `legacy` - report values like ThroughputVsLatency has done in the past
+ * `tsv` - tab separated values
+ * `csv` - comma separated values
+ * `fixed` - a human readable fixed width format
+
+A `fixed` reporter to stdout will be added if no other reporters are writing to stdout or stderr.
 
 All of these types can have their data written out to a file.  To do this add a path after the type.  For example `legacy:./legacy_data` or `tsv:my_run.tsv`. By default the file will be over written unless an option is given to append instead. Options are in a URL like format, with a `?` separating the type:path from the options, and all of the options separated by a `&`.  To append to the file you can do something like `csv:./my_run.csv?append` or  `csv:./my_run.csv?append=true`
 
@@ -82,10 +85,12 @@ Not all options are supported by all reporters.
 
 |Reporter Option| Description | Supported Reporters|
 |---------------|-------------|--------------------|
-|time | Set the time unit that you want latency and CPU reported in.  This can be from nanoseconds up to seconds.  Most names are supported for the types| legacy, csv, tsv|
-|columns | A comma separated list of columns to output (see below for the metrics supported).  A `*` is replaced by all metrics. Defaults to "start_time", "end_time", "completion_rate", "mean", "99%ile", "99.9%ile", "cores", "mem", "failed" | csv, tsv |
-|extraColumns | Like columns but ones that should be added to the defaults instead of replacing them. A `*` is replaced by all metrics. | csv, tsv |
-|meta | An arbitrary string that will appear as a "meta" column at the end.  This helps when appending to files to keep different runs separated | csv, tsv|
+|time | Set the time unit that you want latency and CPU reported in.  This can be from nanoseconds up to seconds.  Most names are supported for the types| legacy, csv, tsv, fixed|
+|columns | A comma separated list of columns to output (see below for the metrics supported).  A `*` is replaced by all metrics. Defaults to "start_time", "end_time", "rate", "mean", "99%ile", "99.9%ile", "cores", "mem", "failed", "ids" | csv, tsv, fixed |
+|extraColumns | Like columns but ones that should be added to the defaults instead of replacing them. A `*` is replaced by all metrics. | csv, tsv, fixed |
+|meta | An arbitrary string that will appear as a "meta" column at the end.  This helps when appending to files to keep different runs separated | csv, tsv, fixed|
+|columnWidth | The width of each field | fixed|
+|precision | The number of places after the decimal point to print out | fixed|
 
 There are a lot of different metrics supported
 
@@ -106,7 +111,7 @@ There are a lot of different metrics supported
 |acked| The number of tuples fully acked as reported by Storm's metrics. | all
 |acked_rate| The rate of tuples fully acked as reported by Storm's metrics. | all
 |completed| The number of tuples fully acked as reported by the latency histogram metrics. | all
-|completion_rate| The rate of tuples fully acked as reported by the latency histogram metrics. | all
+|rate| The rate of tuples fully acked as reported by the latency histogram metrics. | all
 |mem| The amount of memory used by the topology in MB, as reported by the JVM. | all
 |failed| The number of failed tuples as reported by Storm's metrics. | all
 |start_time| The starting time of the metrics window from when the first topology was launched. | all

http://git-wip-us.apache.org/repos/asf/storm/blob/0e3fc5a0/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
index 69adabc..457dedf 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -19,9 +19,11 @@
 package org.apache.storm.loadgen;
 
 import com.google.common.annotations.VisibleForTesting;
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
+import java.io.PrintWriter;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -372,7 +374,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         LinkedHashMap<String, MetricExtractor> tmp = new LinkedHashMap<>();
         tmp.put("start_time",  new MetricExtractor((m, unit) -> m.startTime(),"s"));
         tmp.put("end_time",  new MetricExtractor((m, unit) -> m.endTime(), "s"));
-        tmp.put("completion_rate",  new MetricExtractor((m, unit) -> m.getCompletedPerSec(), "tuple/s"));
+        tmp.put("rate",  new MetricExtractor((m, unit) -> m.getCompletedPerSec(), "tuple/s"));
         tmp.put("mean", new MetricExtractor((m, unit) -> m.getMeanLatency(unit)));
         tmp.put("99%ile", new MetricExtractor((m, unit) -> m.getLatencyAtPercentile(99.0, unit)));
         tmp.put("99.9%ile", new MetricExtractor((m, unit) -> m.getLatencyAtPercentile(99.9, unit)));
@@ -442,16 +444,14 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         }
     }
 
-    static class SepValReporter extends  FileReporter {
-        private final TimeUnit targetUnit;
-        private final String separator;
-        private final List<String> extractors;
-        private final String meta;
+    abstract static class ColumnsFileReporter extends FileReporter {
+        protected final TimeUnit targetUnit;
+        protected final List<String> extractors;
+        protected final String meta;
 
-        public SepValReporter(String separator, String path, Map<String, String> query, Map<String, MetricExtractor> extractorsMap)
+        public ColumnsFileReporter(String path, Map<String, String> query, Map<String, MetricExtractor> extractorsMap)
             throws FileNotFoundException {
             super(path, query, extractorsMap);
-            this.separator = separator;
             targetUnit = UNIT_MAP.get(query.getOrDefault("time", "MILLISECONDS").toUpperCase());
             if (targetUnit == null) {
                 throw new IllegalArgumentException(query.get("time") + " is not a supported time unit");
@@ -467,8 +467,8 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                 this.extractors = extractors;
             } else {
                 //Wrapping it makes it mutable
-                extractors = new ArrayList<>(Arrays.asList("start_time", "end_time", "completion_rate",
-                    "mean", "99%ile", "99.9%ile", "cores", "mem", "failed"));
+                extractors = new ArrayList<>(Arrays.asList("start_time", "end_time", "rate",
+                    "mean", "99%ile", "99.9%ile", "cores", "mem", "failed", "ids"));
             }
 
             if (query.containsKey("extraColumns")) {
@@ -487,7 +487,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             meta = query.get("meta");
         }
 
-        private List<String> handleExtractorCleanup(List<String> orig) {
+        protected List<String> handleExtractorCleanup(List<String> orig) {
             Map<String, Object> stormConfig = Utils.readStormConfig();
             List<String> ret = new ArrayList<>(orig.size());
             for (String extractor: orig) {
@@ -508,6 +508,81 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             }
             return ret;
         }
+    }
+
+
+    static class FixedWidthReporter extends  ColumnsFileReporter {
+        public final String doubleFormat;
+        public final String longFormat;
+        public final String stringFormat;
+
+        public FixedWidthReporter(String path, Map<String, String> query, Map<String, MetricExtractor> extractorsMap)
+            throws FileNotFoundException {
+            super(path, query, extractorsMap);
+            int columnWidth = Integer.parseInt(query.getOrDefault("columnWidth", "15")) - 1;//Always have a space in between
+            int precision = Integer.parseInt(query.getOrDefault("precision", "3"));
+            doubleFormat = "%," + columnWidth + "." + precision + "f";
+            longFormat = "%," + columnWidth + "d";
+            stringFormat = "%" + columnWidth + "s";
+        }
+
+        public FixedWidthReporter(Map<String, MetricExtractor> allExtractors) throws FileNotFoundException {
+            this(null, Collections.emptyMap(), allExtractors);
+        }
+
+        private String format(Object o) {
+            if (o instanceof Double || o instanceof Float) {
+                return String.format(doubleFormat, o);
+            } else if (o instanceof Integer || o instanceof Long) {
+                return String.format(longFormat, o);
+            } else {
+                return String.format(stringFormat, o);
+            }
+        }
+
+        @Override
+        public void start() {
+            boolean first = true;
+            for (String name: extractors) {
+                if (!first) {
+                    out.print(" ");
+                }
+                first = false;
+                out.print(format(allExtractors.get(name).formatName(name, targetUnit)));
+            }
+            if (meta != null) {
+                out.print(" ");
+                out.print(format("meta"));
+            }
+            out.println();
+        }
+
+        @Override
+        public void reportWindow(Measurements m, List<Measurements> allTime) {
+            boolean first = true;
+            for (String name: extractors) {
+                if (!first) {
+                    out.print(" ");
+                }
+                first = false;
+                out.print(format(allExtractors.get(name).get(m, targetUnit)));
+            }
+            if (meta != null) {
+                out.print(" ");
+                out.print(format(meta));
+            }
+            out.println();
+        }
+    }
+
+    static class SepValReporter extends  ColumnsFileReporter {
+        private final String separator;
+
+        public SepValReporter(String separator, String path, Map<String, String> query, Map<String, MetricExtractor> extractorsMap)
+            throws FileNotFoundException {
+            super(path, query, extractorsMap);
+            this.separator = separator;
+        }
 
         @Override
         public void start() {
@@ -628,6 +703,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             .hasArg()
             .argName("TYPE:PATH?OPTIONS")
             .desc("Provide the config for a reporter to run.  Supported types are:\n"
+                + "FIXED - a fixed width format that should be more human readable\n"
                 + "LEGACY - (write things out in the legacy format)\n"
                 + "TSV - tab separated values\n"
                 + "CSV - comma separated values\n"
@@ -693,6 +769,9 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                 }
                 type = type.toUpperCase();
                 switch (type) {
+                    case "FIXED":
+                        reporters.add(new FixedWidthReporter(path, query, allExtractors));
+                        break;
                     case "LEGACY":
                         reporters.add(new LegacyReporter(path, query, allExtractors));
                         break;
@@ -706,8 +785,19 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                         throw new RuntimeException(type + " is not a supported reporter type");
                 }
             }
-        } else {
-            reporters.add(new LegacyReporter(allExtractors));
+        }
+        boolean foundStdOutOrErr = false;
+        for (MetricResultsReporter rep : reporters) {
+            if (rep instanceof FileReporter) {
+                PrintStream ps = ((FileReporter) rep).out;
+                if (ps == System.out || ps == System.err) {
+                    foundStdOutOrErr = true;
+                    break;
+                }
+            }
+        }
+        if (!foundStdOutOrErr) {
+            reporters.add(new FixedWidthReporter(allExtractors));
         }
     }
 


[07/18] storm git commit: Better explination of latency along with a spell check

Posted by bo...@apache.org.
Better explination of latency along with a spell check


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/5238df2e
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/5238df2e
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/5238df2e

Branch: refs/heads/master
Commit: 5238df2ebfea91926b9a85e57aad2b73ce01e43c
Parents: b4cd98f
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Thu Aug 24 09:22:47 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Thu Aug 24 09:22:47 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5238df2e/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index 827fd77..6e9c3ed 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -7,6 +7,8 @@ The idea behind all of these tools is to measure the trade-offs between latency,
 
 When processing data you typically will know a few things.  First you will know about how much data you are going to be processing.  This will typically be a range of values that change throughput the day.  You also will have an idea of how quickly you need the data processed by.  Often this is measured in terms of the latency it takes to process data at the some percentile or set of percentiles.  This is because of most use cases the value of the data declines over time, and being able to react to the data quickly is more valuable.  You probably also have a budget for how much you are willing to spend to be able to process this data.  There are always trade-offs in how quickly you can process some data and how efficiently you can processes that data both in terms of resource usage (cost) and latency.  These tools are designed to help you explore that space.
 
+A note on how latency is measured.  Storm typically measures latency from when a message is emitted by a spout until the point it is fully acked or failed (in many versions of storm it actually does this in the acker instead of the spout so it is trying to be a measure of how long it takes for the actual processing, removing as much of the acker overhead as possible).  For these tools we do it differently.  We simulate a throughput and measure the start time of the tuple from when it would have been emitted if the topology could keep up with the load.  In the normal case this should not be an issue, but if the topology cannot keep up with the throughput you will see the latency grow very high compared to the latency reported by storm.
+
 ## Tools
 ### CaptureLoad 
 
@@ -120,14 +122,14 @@ There are a lot of different metrics supported
 |hosts| The number of hosts the monitored topologies are running on| all
 |executors| The number of running executors in the monitored topologies | all
 |workers| The number of workers the monitored topologies are running on | all
-|target_rate| The target rate in sentenses per second for the ThroughputVsLatency topology | ThroughputVsLatency
+|target_rate| The target rate in sentences per second for the ThroughputVsLatency topology | ThroughputVsLatency
 |spout_parallel| The parallelism of the spout for the `ThroughputVsLatency` topology. | ThroughputVsLatency
 |split_parallel| The parallelism of the split bolt for the `ThroughputVsLatency` topology. | ThroughputVsLatency
 |count_parallel| The parallelism of the count bolt for the `ThroughputVsLatency` topology. | ThroughputVsLatency
 |parallel\_adjust| The adjustment to the parallelism in `GenLoad`. | GenLoad
-|topo_parallel| A list of topology/component specfic adjustment rules to the parallelism in `GenLoad`. | GenLoad
+|topo_parallel| A list of topology/component specific adjustment rules to the parallelism in `GenLoad`. | GenLoad
 |throughput_adjust| The adjustment to the throughput in `GenLoad`. | GenLoad
-|topo_throughput| A list of topology/component specfic adjustment rules to the throughput in `GenLoad`. | GenLoad
+|topo_throughput| A list of topology/component specific adjustment rules to the throughput in `GenLoad`. | GenLoad
 |local\_or\_shuffle| true if shuffles were replaced with local or shuffle in GenLoad. | GenLoad
 
 There are also some generic rules that you can use for some metrics.  Any metric that starts with `"conf:"` will be the config for that.  It does not include config overrides from the `GenLoad` file.


[08/18] storm git commit: STORM-2702: some code cleanup for unused stuff and added some RAS support

Posted by bo...@apache.org.
STORM-2702: some code cleanup for unused stuff and added some RAS support


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/6bc32138
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/6bc32138
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/6bc32138

Branch: refs/heads/master
Commit: 6bc3213806c67818c8e7c0c8d72c0b8a234468b7
Parents: 5238df2
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Thu Aug 24 10:02:03 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Thu Aug 24 10:07:21 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                |   2 +
 .../org/apache/storm/loadgen/CaptureLoad.java   | 131 ++++++++++++++++++-
 .../org/apache/storm/loadgen/CompStats.java     |  52 --------
 .../java/org/apache/storm/loadgen/GenLoad.java  |  20 ++-
 .../org/apache/storm/loadgen/LoadCompConf.java  |  42 +++---
 .../org/apache/storm/loadgen/LoadEngine.java    |  45 -------
 6 files changed, 173 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/6bc32138/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index 6e9c3ed..50a618f 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -159,6 +159,8 @@ Spouts and bolts have the same format.
 | id | The id of the bolt or spout.  This should be unique within the topology |
 | parallelism | How many instances of this component should be a part of the topology |
 | streams | The streams that are produced by this bolt or spout |
+| cpuLoad | The number of cores this component needs for resource aware scheduling |
+| memoryLoad | The amount of memory in MB that this component needs for resource aware scheduling |
 
 ### Output Streams
 

http://git-wip-us.apache.org/repos/asf/storm/blob/6bc32138/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
index e748efa..649a4c0 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
@@ -49,7 +49,10 @@ import org.apache.storm.generated.TopologyPageInfo;
 import org.apache.storm.generated.TopologySummary;
 import org.apache.storm.generated.WorkerSummary;
 import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.ObjectReader;
+import org.json.simple.JSONObject;
 import org.json.simple.JSONValue;
+import org.json.simple.parser.JSONParser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -150,6 +153,22 @@ public class CaptureLoad {
                     .withId(boltComp);
                 boltBuilders.put(boltComp, builder);
             }
+
+            Map<String, Map<String, Double>> boltResources = getBoltsResources(topo, topoConf);
+            for (Map.Entry<String, Map<String, Double>> entry: boltResources.entrySet()) {
+                LoadCompConf.Builder bd = boltBuilders.get(entry.getKey());
+                if (bd != null) {
+                    Map<String, Double> resources = entry.getValue();
+                    Double cpu = resources.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT);
+                    if (cpu != null) {
+                        bd.withCpuLoad(cpu);
+                    }
+                    Double mem = resources.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
+                    if (mem != null) {
+                        bd.withMemoryLoad(mem);
+                    }
+                }
+            }
         }
 
         //Spouts
@@ -175,6 +194,22 @@ public class CaptureLoad {
                     .withId(spoutComp);
                 spoutBuilders.put(spoutComp, builder);
             }
+
+            Map<String, Map<String, Double>> spoutResources = getSpoutsResources(topo, topoConf);
+            for (Map.Entry<String, Map<String, Double>> entry: spoutResources.entrySet()) {
+                LoadCompConf.Builder sd = spoutBuilders.get(entry.getKey());
+                if (sd != null) {
+                    Map<String, Double> resources = entry.getValue();
+                    Double cpu = resources.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT);
+                    if (cpu != null) {
+                        sd.withCpuLoad(cpu);
+                    }
+                    Double mem = resources.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
+                    if (mem != null) {
+                        sd.withMemoryLoad(mem);
+                    }
+                }
+            }
         }
 
         //Stats...
@@ -247,8 +282,6 @@ public class CaptureLoad {
                 }
             }
             builder.withRate(new NormalDistStats(emittedRate));
-            //TODO to know if the output keys are skewed we have to guess by looking
-            // at the down stream executed stats, but for now we are going to ignore it
 
             //The OutputStream is done
             LoadCompConf.Builder comp = boltBuilders.get(id.get_componentId());
@@ -338,4 +371,98 @@ public class CaptureLoad {
             System.exit(exitStatus);
         }
     }
+
+    //ResourceUtils.java is not a available on the classpath to let us parse out the resources we want.
+    // So we have copied and pasted some of the needed methods here. (with a few changes to logging)
+    static Map<String, Map<String, Double>> getBoltsResources(StormTopology topology,
+                                                                     Map<String, Object> topologyConf) {
+        Map<String, Map<String, Double>> boltResources = new HashMap<>();
+        if (topology.get_bolts() != null) {
+            for (Map.Entry<String, Bolt> bolt : topology.get_bolts().entrySet()) {
+                Map<String, Double> topologyResources = parseResources(bolt.getValue().get_common().get_json_conf());
+                checkIntialization(topologyResources, bolt.getValue().toString(), topologyConf);
+                boltResources.put(bolt.getKey(), topologyResources);
+            }
+        }
+        return boltResources;
+    }
+
+    static Map<String, Map<String, Double>> getSpoutsResources(StormTopology topology,
+                                                                      Map<String, Object> topologyConf) {
+        Map<String, Map<String, Double>> spoutResources = new HashMap<>();
+        if (topology.get_spouts() != null) {
+            for (Map.Entry<String, SpoutSpec> spout : topology.get_spouts().entrySet()) {
+                Map<String, Double> topologyResources = parseResources(spout.getValue().get_common().get_json_conf());
+                checkIntialization(topologyResources, spout.getValue().toString(), topologyConf);
+                spoutResources.put(spout.getKey(), topologyResources);
+            }
+        }
+        return spoutResources;
+    }
+
+    static Map<String, Double> parseResources(String input) {
+        Map<String, Double> topologyResources = new HashMap<>();
+        JSONParser parser = new JSONParser();
+        LOG.debug("Input to parseResources {}", input);
+        try {
+            if (input != null) {
+                Object obj = parser.parse(input);
+                JSONObject jsonObject = (JSONObject) obj;
+                if (jsonObject.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB)) {
+                    Double topoMemOnHeap = ObjectReader
+                        .getDouble(jsonObject.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null);
+                    topologyResources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, topoMemOnHeap);
+                }
+                if (jsonObject.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)) {
+                    Double topoMemOffHeap = ObjectReader
+                        .getDouble(jsonObject.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
+                    topologyResources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, topoMemOffHeap);
+                }
+                if (jsonObject.containsKey(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT)) {
+                    Double topoCpu = ObjectReader.getDouble(jsonObject.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT),
+                        null);
+                    topologyResources.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, topoCpu);
+                }
+                LOG.debug("Topology Resources {}", topologyResources);
+            }
+        } catch (org.json.simple.parser.ParseException e) {
+            LOG.error("Failed to parse component resources is:" + e.toString(), e);
+            return null;
+        }
+        return topologyResources;
+    }
+
+    static void checkIntialization(Map<String, Double> topologyResources, String com,
+                                          Map<String, Object> topologyConf) {
+        checkInitMem(topologyResources, com, topologyConf);
+        checkInitCpu(topologyResources, com, topologyConf);
+    }
+
+    static void checkInitMem(Map<String, Double> topologyResources, String com,
+                                     Map<String, Object> topologyConf) {
+        if (!topologyResources.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB)) {
+            Double onHeap = ObjectReader.getDouble(
+                topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null);
+            if (onHeap != null) {
+                topologyResources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, onHeap);
+            }
+        }
+        if (!topologyResources.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)) {
+            Double offHeap = ObjectReader.getDouble(
+                topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
+            if (offHeap != null) {
+                topologyResources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, offHeap);
+            }
+        }
+    }
+
+    static void checkInitCpu(Map<String, Double> topologyResources, String com,
+                                     Map<String, Object> topologyConf) {
+        if (!topologyResources.containsKey(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT)) {
+            Double cpu = ObjectReader.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null);
+            if (cpu != null) {
+                topologyResources.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, cpu);
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/6bc32138/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CompStats.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CompStats.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CompStats.java
deleted file mode 100644
index d0e0bd3..0000000
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CompStats.java
+++ /dev/null
@@ -1,52 +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.loadgen;
-
-import java.io.Serializable;
-import java.util.Map;
-import org.apache.storm.utils.ObjectReader;
-
-/**
- * A set of measurements about a component (bolt/spout) so we can statistically reproduce it.
- */
-public class CompStats implements Serializable {
-    public final double cpuPercent; // Right now we don't have a good way to measure any kind of a distribution, this is all approximate
-    public final double memoryMb; //again no good way to get a distribution...
-
-    /**
-     * Parse out a CompStats from a config map.
-     * @param conf the map holding the CompStats values
-     * @return the parsed CompStats
-     */
-    public static CompStats fromConf(Map<String, Object> conf) {
-        double cpu = ObjectReader.getDouble(conf.get("cpuPercent"), 0.0);
-        double memory = ObjectReader.getDouble(conf.get("memoryMb"), 0.0);
-        return new CompStats(cpu, memory);
-    }
-
-    public void addToConf(Map<String, Object> ret) {
-        ret.put("cpuPercent", cpuPercent);
-        ret.put("memoryMb", memoryMb);
-    }
-
-    public CompStats(double cpuPercent, double memoryMb) {
-        this.cpuPercent = cpuPercent;
-        this.memoryMb = memoryMb;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/6bc32138/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
index 95ba8dd..f535b32 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
@@ -40,6 +40,7 @@ import org.apache.storm.generated.AuthorizationException;
 import org.apache.storm.generated.InvalidTopologyException;
 import org.apache.storm.metric.LoggingMetricsConsumer;
 import org.apache.storm.topology.BoltDeclarer;
+import org.apache.storm.topology.SpoutDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.utils.NimbusClient;
 import org.slf4j.Logger;
@@ -227,7 +228,7 @@ public class GenLoad {
         return tlc;
     }
 
-    static int uniquifier = 0;
+    private static int uniquifier = 0;
 
     private static String parseAndSubmit(TopologyLoadConf tlc, String url) throws IOException, InvalidTopologyException,
         AuthorizationException, AlreadyAliveException {
@@ -260,7 +261,13 @@ public class GenLoad {
         TopologyBuilder builder = new TopologyBuilder();
         for (LoadCompConf spoutConf : tlc.spouts) {
             System.out.println("ADDING SPOUT " + spoutConf.id);
-            builder.setSpout(spoutConf.id, new LoadSpout(spoutConf), spoutConf.parallelism);
+            SpoutDeclarer sd = builder.setSpout(spoutConf.id, new LoadSpout(spoutConf), spoutConf.parallelism);
+            if (spoutConf.memoryLoad > 0) {
+                sd.setMemoryLoad(spoutConf.memoryLoad);
+            }
+            if (spoutConf.cpuLoad > 0) {
+                sd.setCPULoad(spoutConf.cpuLoad);
+            }
         }
 
         Map<String, BoltDeclarer> boltDeclarers = new HashMap<>();
@@ -270,7 +277,14 @@ public class GenLoad {
                 System.out.println("ADDING BOLT " + boltConf.id);
                 LoadBolt lb = new LoadBolt(boltConf);
                 bolts.put(boltConf.id, lb);
-                boltDeclarers.put(boltConf.id, builder.setBolt(boltConf.id, lb, boltConf.parallelism));
+                BoltDeclarer bd = builder.setBolt(boltConf.id, lb, boltConf.parallelism);
+                if (boltConf.memoryLoad > 0) {
+                    bd.setMemoryLoad(boltConf.memoryLoad);
+                }
+                if (boltConf.cpuLoad > 0) {
+                    bd.setCPULoad(boltConf.cpuLoad);
+                }
+                boltDeclarers.put(boltConf.id, bd);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/6bc32138/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadCompConf.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadCompConf.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadCompConf.java
index baead0f..6548cc8 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadCompConf.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadCompConf.java
@@ -33,7 +33,8 @@ public class LoadCompConf {
     public final String id;
     public final int parallelism;
     public final List<OutputStream> streams;
-    public final CompStats stats;
+    public final double cpuLoad;
+    public final double memoryLoad;
 
     /**
      * Parse the LoadCompConf from a config Map.
@@ -50,8 +51,10 @@ public class LoadCompConf {
                 streams.add(OutputStream.fromConf(streamInfo));
             }
         }
+        double memoryMb = ObjectReader.getDouble(conf.get("memoryLoad"), 0.0);
+        double cpuPercent = ObjectReader.getDouble(conf.get("cpuLoad"), 0.0);
 
-        return new LoadCompConf(id, parallelism, streams, CompStats.fromConf(conf));
+        return new LoadCompConf(id, parallelism, streams, memoryMb, cpuPercent);
     }
 
     /**
@@ -62,6 +65,12 @@ public class LoadCompConf {
         Map<String, Object> ret = new HashMap<>();
         ret.put("id", id);
         ret.put("parallelism", parallelism);
+        if (memoryLoad > 0) {
+            ret.put("memoryLoad", memoryLoad);
+        }
+        if (cpuLoad > 0) {
+            ret.put("cpuLoad", cpuLoad);
+        }
 
         if (streams != null) {
             List<Map<String, Object>> streamData = new ArrayList<>();
@@ -70,9 +79,6 @@ public class LoadCompConf {
             }
             ret.put("streams", streamData);
         }
-        if (stats != null) {
-            stats.addToConf(ret);
-        }
         return ret;
     }
 
@@ -89,7 +95,7 @@ public class LoadCompConf {
                 .map((orig) -> orig.remap(id, remappedStreams))
                 .collect(Collectors.toList());
 
-        return new LoadCompConf(remappedId, parallelism, remappedOutStreams, stats);
+        return new LoadCompConf(remappedId, parallelism, remappedOutStreams, cpuLoad, memoryLoad);
     }
 
     /**
@@ -110,7 +116,7 @@ public class LoadCompConf {
     public LoadCompConf setParallel(int newParallelism) {
         //We need to adjust the throughput accordingly (so that it stays the same in aggregate)
         double throughputAdjustment = ((double)parallelism) / newParallelism;
-        return new LoadCompConf(id, newParallelism, streams, stats).scaleThroughput(throughputAdjustment);
+        return new LoadCompConf(id, newParallelism, streams, cpuLoad, memoryLoad).scaleThroughput(throughputAdjustment);
     }
 
     /**
@@ -121,7 +127,7 @@ public class LoadCompConf {
     public LoadCompConf scaleThroughput(double v) {
         if (streams != null) {
             List<OutputStream> newStreams = streams.stream().map((s) -> s.scaleThroughput(v)).collect(Collectors.toList());
-            return new LoadCompConf(id, parallelism, newStreams, stats);
+            return new LoadCompConf(id, parallelism, newStreams, cpuLoad, memoryLoad);
         } else {
             return this;
         }
@@ -147,7 +153,8 @@ public class LoadCompConf {
         private String id;
         private int parallelism = 1;
         private List<OutputStream> streams;
-        private CompStats stats;
+        private double cpuLoad = 0.0;
+        private double memoryLoad = 0.0;
 
         public String getId() {
             return id;
@@ -189,17 +196,18 @@ public class LoadCompConf {
             return this;
         }
 
-        public CompStats getStats() {
-            return stats;
+        public Builder withCpuLoad(double cpuLoad) {
+            this.cpuLoad = cpuLoad;
+            return this;
         }
 
-        public Builder withStats(CompStats stats) {
-            this.stats = stats;
+        public Builder withMemoryLoad(double memoryLoad) {
+            this.memoryLoad = memoryLoad;
             return this;
         }
 
         public LoadCompConf build() {
-            return new LoadCompConf(id, parallelism, streams, stats);
+            return new LoadCompConf(id, parallelism, streams, cpuLoad, memoryLoad);
         }
     }
 
@@ -208,15 +216,15 @@ public class LoadCompConf {
      * @param id the id of the component.
      * @param parallelism tha parallelism of the component.
      * @param streams the output streams of the component.
-     * @param stats the stats of the component.
      */
-    public LoadCompConf(String id, int parallelism, List<OutputStream> streams, CompStats stats) {
+    public LoadCompConf(String id, int parallelism, List<OutputStream> streams, double cpuLoad, double memoryLoad) {
         this.id = id;
         if (id == null) {
             throw new IllegalArgumentException("A spout ID cannot be null");
         }
         this.parallelism = parallelism;
         this.streams = streams;
-        this.stats = stats;
+        this.cpuLoad = cpuLoad;
+        this.memoryLoad = memoryLoad;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/6bc32138/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadEngine.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadEngine.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadEngine.java
deleted file mode 100644
index 9030379..0000000
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadEngine.java
+++ /dev/null
@@ -1,45 +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.loadgen;
-
-import java.util.Random;
-import java.util.concurrent.ThreadLocalRandom;
-import org.apache.storm.loadgen.CompStats;
-
-/**
- * The goal of this class is to provide a set of "Tuples" to send that will match as closely as possible the characteristics
- * measured from a production topology.
- */
-public class LoadEngine {
-
-    //TODO need to do a lot...
-
-    /**
-     * Provides an API to simulate the timings and CPU utilization of a bolt or spout.
-     */
-    public static class InputTimingEngine {
-        private final Random rand;
-        private final CompStats stats;
-
-        public InputTimingEngine(CompStats stats) {
-            this.stats = stats;
-            rand = ThreadLocalRandom.current();
-        }
-    }
-}


[04/18] storm git commit: STORM-2702: storm-loadgen

Posted by bo...@apache.org.
STORM-2702: storm-loadgen


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/6c2dcbed
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/6c2dcbed
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/6c2dcbed

Branch: refs/heads/master
Commit: 6c2dcbedabb88970697f42b6f66bf64177e2ac9c
Parents: 0d10b8a
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Mon Aug 21 14:36:10 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Aug 21 14:36:10 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/pom.xml                  |  13 +-
 .../org/apache/storm/loadgen/CaptureLoad.java   | 341 ++++++++
 .../org/apache/storm/loadgen/CompStats.java     |  52 ++
 .../storm/loadgen/EstimateThroughput.java       | 108 +++
 .../java/org/apache/storm/loadgen/GenLoad.java  | 235 ++++++
 .../org/apache/storm/loadgen/GroupingType.java  |  91 +++
 .../loadgen/HttpForwardingMetricsConsumer.java  |  51 +-
 .../loadgen/HttpForwardingMetricsServer.java    |  79 +-
 .../org/apache/storm/loadgen/InputStream.java   | 263 +++++++
 .../java/org/apache/storm/loadgen/LoadBolt.java | 146 ++++
 .../org/apache/storm/loadgen/LoadCompConf.java  | 222 ++++++
 .../org/apache/storm/loadgen/LoadEngine.java    |  45 ++
 .../apache/storm/loadgen/LoadMetricsServer.java | 784 +++++++++++++++++++
 .../org/apache/storm/loadgen/LoadSpout.java     | 137 ++++
 .../apache/storm/loadgen/NormalDistStats.java   | 151 ++++
 .../org/apache/storm/loadgen/OutputStream.java  | 121 +++
 .../storm/loadgen/OutputStreamEngine.java       | 122 +++
 .../apache/storm/loadgen/ScopedTopologySet.java |  91 +++
 .../storm/loadgen/ThroughputVsLatency.java      | 405 ++++------
 .../apache/storm/loadgen/TopologyLoadConf.java  | 432 ++++++++++
 .../apache/storm/loadgen/LoadCompConfTest.java  |  57 ++
 .../storm/loadgen/LoadMetricsServerTest.java    |  36 +
 .../storm/loadgen/NormalDistStatsTest.java      |  43 +
 .../apache/storm/loadgen/OutputStreamTest.java  |  37 +
 24 files changed, 3724 insertions(+), 338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/pom.xml b/examples/storm-loadgen/pom.xml
index f75e575..1b2e4f7 100644
--- a/examples/storm-loadgen/pom.xml
+++ b/examples/storm-loadgen/pom.xml
@@ -41,14 +41,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-server</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-servlet</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.apache.storm</groupId>
       <artifactId>storm-client</artifactId>
       <version>${project.version}</version>
@@ -60,6 +52,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     </dependency>
     <dependency>
       <groupId>org.apache.storm</groupId>
+      <artifactId>storm-client-misc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.storm</groupId>
       <artifactId>storm-metrics</artifactId>
       <version>${project.version}</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
new file mode 100644
index 0000000..e748efa
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CaptureLoad.java
@@ -0,0 +1,341 @@
+/**
+ * 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.loadgen;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.storm.Config;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.BoltStats;
+import org.apache.storm.generated.ClusterSummary;
+import org.apache.storm.generated.ComponentCommon;
+import org.apache.storm.generated.ExecutorSummary;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.Nimbus;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.generated.TopologyInfo;
+import org.apache.storm.generated.TopologyPageInfo;
+import org.apache.storm.generated.TopologySummary;
+import org.apache.storm.generated.WorkerSummary;
+import org.apache.storm.utils.NimbusClient;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Capture running topologies for load gen later on.
+ */
+public class CaptureLoad {
+    private static final Logger LOG = LoggerFactory.getLogger(CaptureLoad.class);
+    public static final String DEFAULT_OUT_DIR = "./loadgen/";
+
+    private static List<Double> extractBoltValues(List<ExecutorSummary> summaries,
+                                                  GlobalStreamId id,
+                                                  Function<BoltStats, Map<String, Map<GlobalStreamId, Double>>> func) {
+
+        List<Double> ret = new ArrayList<>();
+        if (summaries != null) {
+            for (ExecutorSummary summ : summaries) {
+                if (summ != null && summ.is_set_stats()) {
+                    Map<String, Map<GlobalStreamId, Double>> data = func.apply(summ.get_stats().get_specific().get_bolt());
+                    if (data != null) {
+                        List<Double> subvalues = data.values().stream()
+                            .map((subMap) -> subMap.get(id))
+                            .filter((value) -> value != null)
+                            .mapToDouble((value) -> value.doubleValue())
+                            .boxed().collect(Collectors.toList());
+                        ret.addAll(subvalues);
+                    }
+                }
+            }
+        }
+        return ret;
+    }
+
+    static TopologyLoadConf captureTopology(Nimbus.Iface client, TopologySummary topologySummary) throws Exception {
+        String topologyName = topologySummary.get_name();
+        LOG.info("Capturing {}...", topologyName);
+        String topologyId = topologySummary.get_id();
+        TopologyInfo info = client.getTopologyInfo(topologyId);
+        TopologyPageInfo tpinfo = client.getTopologyPageInfo(topologyId, ":all-time", false);
+        @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
+        StormTopology topo = client.getUserTopology(topologyId);
+        //Done capturing topology information...
+
+        Map<String, Object> savedTopoConf = new HashMap<>();
+        Map<String, Object> topoConf = (Map<String, Object>) JSONValue.parse(client.getTopologyConf(topologyId));
+        for (String key: TopologyLoadConf.IMPORTANT_CONF_KEYS) {
+            Object o = topoConf.get(key);
+            if (o != null) {
+                savedTopoConf.put(key, o);
+                LOG.info("with config {}: {}", key, o);
+            }
+        }
+        //Lets use the number of actually scheduled workers as a way to bridge RAS and non-RAS
+        int numWorkers = tpinfo.get_num_workers();
+        if (savedTopoConf.containsKey(Config.TOPOLOGY_WORKERS)) {
+            numWorkers = Math.max(numWorkers, ((Number)savedTopoConf.get(Config.TOPOLOGY_WORKERS)).intValue());
+        }
+        savedTopoConf.put(Config.TOPOLOGY_WORKERS, numWorkers);
+
+        Map<String, LoadCompConf.Builder> boltBuilders = new HashMap<>();
+        Map<String, LoadCompConf.Builder> spoutBuilders = new HashMap<>();
+        List<InputStream.Builder> inputStreams = new ArrayList<>();
+        Map<GlobalStreamId, OutputStream.Builder> outStreams = new HashMap<>();
+
+        //Bolts
+        if (topo.get_bolts() != null) {
+            for (Map.Entry<String, Bolt> boltSpec : topo.get_bolts().entrySet()) {
+                String boltComp = boltSpec.getKey();
+                LOG.info("Found bolt {}...", boltComp);
+                Bolt bolt = boltSpec.getValue();
+                ComponentCommon common = bolt.get_common();
+                Map<GlobalStreamId, Grouping> inputs = common.get_inputs();
+                if (inputs != null) {
+                    for (Map.Entry<GlobalStreamId, Grouping> input : inputs.entrySet()) {
+                        GlobalStreamId id = input.getKey();
+                        LOG.info("with input {}...", id);
+                        Grouping grouping = input.getValue();
+                        InputStream.Builder builder = new InputStream.Builder()
+                            .withId(id.get_streamId())
+                            .withFromComponent(id.get_componentId())
+                            .withToComponent(boltComp)
+                            .withGroupingType(grouping);
+                        inputStreams.add(builder);
+                    }
+                }
+                Map<String, StreamInfo> outputs = common.get_streams();
+                if (outputs != null) {
+                    for (String name : outputs.keySet()) {
+                        GlobalStreamId id = new GlobalStreamId(boltComp, name);
+                        LOG.info("and output {}...", id);
+                        OutputStream.Builder builder = new OutputStream.Builder()
+                            .withId(name);
+                        outStreams.put(id, builder);
+                    }
+                }
+                LoadCompConf.Builder builder = new LoadCompConf.Builder()
+                    .withParallelism(common.get_parallelism_hint())
+                    .withId(boltComp);
+                boltBuilders.put(boltComp, builder);
+            }
+        }
+
+        //Spouts
+        if (topo.get_spouts() != null) {
+            for (Map.Entry<String, SpoutSpec> spoutSpec : topo.get_spouts().entrySet()) {
+                String spoutComp = spoutSpec.getKey();
+                LOG.info("Found Spout {}...", spoutComp);
+                SpoutSpec spout = spoutSpec.getValue();
+                ComponentCommon common = spout.get_common();
+
+                Map<String, StreamInfo> outputs = common.get_streams();
+                if (outputs != null) {
+                    for (String name : outputs.keySet()) {
+                        GlobalStreamId id = new GlobalStreamId(spoutComp, name);
+                        LOG.info("with output {}...", id);
+                        OutputStream.Builder builder = new OutputStream.Builder()
+                            .withId(name);
+                        outStreams.put(id, builder);
+                    }
+                }
+                LoadCompConf.Builder builder = new LoadCompConf.Builder()
+                    .withParallelism(common.get_parallelism_hint())
+                    .withId(spoutComp);
+                spoutBuilders.put(spoutComp, builder);
+            }
+        }
+
+        //Stats...
+        Map<String, List<ExecutorSummary>> byComponent = new HashMap<>();
+        for (ExecutorSummary executor: info.get_executors()) {
+            String component = executor.get_component_id();
+            List<ExecutorSummary> list = byComponent.get(component);
+            if (list == null) {
+                list = new ArrayList<>();
+                byComponent.put(component, list);
+            }
+            list.add(executor);
+        }
+
+        List<InputStream> streams = new ArrayList<>(inputStreams.size());
+        //Compute the stats for the different input streams
+        for (InputStream.Builder builder : inputStreams) {
+            GlobalStreamId streamId = new GlobalStreamId(builder.getFromComponent(), builder.getId());
+            List<ExecutorSummary> summaries = byComponent.get(builder.getToComponent());
+            //Execute and process latency...
+            builder.withProcessTime(new NormalDistStats(
+                extractBoltValues(summaries, streamId, BoltStats::get_process_ms_avg)));
+            builder.withExecTime(new NormalDistStats(
+                extractBoltValues(summaries, streamId, BoltStats::get_execute_ms_avg)));
+            //InputStream is done
+            streams.add(builder.build());
+        }
+
+        //There is a bug in some versions that returns 0 for the uptime.
+        // To work around it we should get it an alternative (working) way.
+        Map<String, Integer> workerToUptime = new HashMap<>();
+        for (WorkerSummary ws : tpinfo.get_workers()) {
+            workerToUptime.put(ws.get_supervisor_id() + ":" + ws.get_port(), ws.get_uptime_secs());
+        }
+        LOG.debug("WORKER TO UPTIME {}", workerToUptime);
+
+        for (Map.Entry<GlobalStreamId, OutputStream.Builder> entry : outStreams.entrySet()) {
+            OutputStream.Builder builder = entry.getValue();
+            GlobalStreamId id = entry.getKey();
+            List<Double> emittedRate = new ArrayList<>();
+            List<ExecutorSummary> summaries = byComponent.get(id.get_componentId());
+            if (summaries != null) {
+                for (ExecutorSummary summary: summaries) {
+                    if (summary.is_set_stats()) {
+                        int uptime = summary.get_uptime_secs();
+                        LOG.debug("UPTIME {}", uptime);
+                        if (uptime <= 0) {
+                            //Likely it is because of a bug, so try to get it another way
+                            String key = summary.get_host() + ":" + summary.get_port();
+                            uptime = workerToUptime.getOrDefault(key, 1);
+                            LOG.debug("Getting uptime for worker {}, {}", key, uptime);
+                        }
+                        for (Map.Entry<String, Map<String, Long>> statEntry : summary.get_stats().get_emitted().entrySet()) {
+                            String timeWindow = statEntry.getKey();
+                            long timeSecs = uptime;
+                            try {
+                                timeSecs = Long.valueOf(timeWindow);
+                            } catch (NumberFormatException e) {
+                                //Ignored...
+                            }
+                            timeSecs = Math.min(timeSecs, uptime);
+                            Long count = statEntry.getValue().get(id.get_streamId());
+                            if (count != null) {
+                                LOG.debug("{} emitted {} for {} secs or {} tuples/sec",
+                                    id, count, timeSecs, count.doubleValue() / timeSecs);
+                                emittedRate.add(count.doubleValue() / timeSecs);
+                            }
+                        }
+                    }
+                }
+            }
+            builder.withRate(new NormalDistStats(emittedRate));
+            //TODO to know if the output keys are skewed we have to guess by looking
+            // at the down stream executed stats, but for now we are going to ignore it
+
+            //The OutputStream is done
+            LoadCompConf.Builder comp = boltBuilders.get(id.get_componentId());
+            if (comp == null) {
+                comp = spoutBuilders.get(id.get_componentId());
+            }
+            comp.withStream(builder.build());
+        }
+
+        List<LoadCompConf> spouts = spoutBuilders.values().stream()
+            .map((b) -> b.build())
+            .collect(Collectors.toList());
+
+        List<LoadCompConf> bolts = boltBuilders.values().stream()
+            .map((b) -> b.build())
+            .collect(Collectors.toList());
+
+        return new TopologyLoadConf(topologyName, savedTopoConf, spouts, bolts, streams);
+    }
+
+    /**
+     * Main entry point for CaptureLoad command.
+     * @param args the arguments to the command
+     * @throws Exception on any error
+     */
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+        options.addOption(Option.builder("a")
+            .longOpt("anonymize")
+            .desc("Strip out any possibly identifiable information")
+            .build());
+        options.addOption(Option.builder("o")
+            .longOpt("output-dir")
+            .argName("<file>")
+            .hasArg()
+            .desc("Where to write (defaults to " + DEFAULT_OUT_DIR + ")")
+            .build());
+        options.addOption(Option.builder("h")
+            .longOpt("help")
+            .desc("Print a help message")
+            .build());
+        CommandLineParser parser = new DefaultParser();
+        CommandLine cmd = null;
+        ParseException pe = null;
+        try {
+            cmd = parser.parse(options, args);
+        } catch (ParseException e) {
+            pe = e;
+        }
+        if (pe != null || cmd.hasOption('h')) {
+            if (pe != null) {
+                System.err.println("ERROR " + pe.getMessage());
+            }
+            new HelpFormatter().printHelp("CaptureLoad [options] [topologyName]*", options);
+            return;
+        }
+
+        Config conf = new Config();
+        int exitStatus = -1;
+        String outputDir = DEFAULT_OUT_DIR;
+        if (cmd.hasOption('o')) {
+            outputDir = cmd.getOptionValue('o');
+        }
+        File baseOut = new File(outputDir);
+        LOG.info("Will save captured topologies to {}", baseOut);
+        baseOut.mkdirs();
+
+        try (NimbusClient nc = NimbusClient.getConfiguredClient(conf)) {
+            Nimbus.Iface client = nc.getClient();
+            List<String> topologyNames = cmd.getArgList();
+
+            ClusterSummary clusterSummary = client.getClusterInfo();
+            for (TopologySummary topologySummary: clusterSummary.get_topologies()) {
+                if (topologyNames.isEmpty() || topologyNames.contains(topologySummary.get_name())) {
+                    TopologyLoadConf capturedConf = captureTopology(client, topologySummary);
+                    if (cmd.hasOption('a')) {
+                        capturedConf = capturedConf.anonymize();
+                    }
+                    capturedConf.writeTo(new File(baseOut, capturedConf.name + ".yaml"));
+                }
+            }
+
+            exitStatus = 0;
+        } catch (Exception e) {
+            LOG.error("Error trying to capture topologies...", e);
+        } finally {
+            System.exit(exitStatus);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CompStats.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CompStats.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CompStats.java
new file mode 100644
index 0000000..d0e0bd3
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/CompStats.java
@@ -0,0 +1,52 @@
+/**
+ * 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.loadgen;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.storm.utils.ObjectReader;
+
+/**
+ * A set of measurements about a component (bolt/spout) so we can statistically reproduce it.
+ */
+public class CompStats implements Serializable {
+    public final double cpuPercent; // Right now we don't have a good way to measure any kind of a distribution, this is all approximate
+    public final double memoryMb; //again no good way to get a distribution...
+
+    /**
+     * Parse out a CompStats from a config map.
+     * @param conf the map holding the CompStats values
+     * @return the parsed CompStats
+     */
+    public static CompStats fromConf(Map<String, Object> conf) {
+        double cpu = ObjectReader.getDouble(conf.get("cpuPercent"), 0.0);
+        double memory = ObjectReader.getDouble(conf.get("memoryMb"), 0.0);
+        return new CompStats(cpu, memory);
+    }
+
+    public void addToConf(Map<String, Object> ret) {
+        ret.put("cpuPercent", cpuPercent);
+        ret.put("memoryMb", memoryMb);
+    }
+
+    public CompStats(double cpuPercent, double memoryMb) {
+        this.cpuPercent = cpuPercent;
+        this.memoryMb = memoryMb;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/EstimateThroughput.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/EstimateThroughput.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/EstimateThroughput.java
new file mode 100644
index 0000000..80ede37
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/EstimateThroughput.java
@@ -0,0 +1,108 @@
+/**
+ * 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.loadgen;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.storm.Config;
+import org.apache.storm.generated.ClusterSummary;
+import org.apache.storm.generated.Nimbus;
+import org.apache.storm.generated.TopologySummary;
+import org.apache.storm.loadgen.CaptureLoad;
+import org.apache.storm.utils.NimbusClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Estimate the throughput of all topologies.
+ */
+public class EstimateThroughput {
+    private static final Logger LOG = LoggerFactory.getLogger(EstimateThroughput.class);
+
+    /**
+     * Main entry point for estimate throughput command.
+     * @param args the command line arguments.
+     * @throws Exception on any error.
+     */
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+        options.addOption(Option.builder("h")
+            .longOpt("help")
+            .desc("Print a help message")
+            .build());
+        CommandLineParser parser = new DefaultParser();
+        CommandLine cmd = null;
+        ParseException pe = null;
+        try {
+            cmd = parser.parse(options, args);
+        } catch (ParseException e) {
+            pe = e;
+        }
+        if (pe != null || cmd.hasOption('h')) {
+            if (pe != null) {
+                System.err.println("ERROR " + pe.getMessage());
+            }
+            new HelpFormatter().printHelp("EstimateThroughput [options] [topologyName]*", options);
+            return;
+        }
+
+        Config conf = new Config();
+        int exitStatus = -1;
+
+        List<TopologyLoadConf> regular = new ArrayList<>();
+        List<TopologyLoadConf> trident = new ArrayList<>();
+
+        try (NimbusClient nc = NimbusClient.getConfiguredClient(conf)) {
+            Nimbus.Iface client = nc.getClient();
+            List<String> topologyNames = cmd.getArgList();
+
+            ClusterSummary clusterSummary = client.getClusterInfo();
+            for (TopologySummary topologySummary: clusterSummary.get_topologies()) {
+                if (topologyNames.isEmpty() || topologyNames.contains(topologySummary.get_name())) {
+                    TopologyLoadConf capturedConf = CaptureLoad.captureTopology(client, topologySummary);
+                    if (capturedConf.looksLikeTrident()) {
+                        trident.add(capturedConf);
+                    } else {
+                        regular.add(capturedConf);
+                    }
+                }
+            }
+
+            System.out.println("TOPOLOGY\tTOTAL MESSAGES/sec\tESTIMATED INPUT MESSAGES/sec");
+            for (TopologyLoadConf tl: regular) {
+                System.out.println(tl.name + "\t" + tl.getAllEmittedAggregate() + "\t" + tl.getSpoutEmittedAggregate());
+            }
+            for (TopologyLoadConf tl: trident) {
+                System.out.println(tl.name + "\t" + tl.getAllEmittedAggregate() + "\t" + tl.getTridentEstimatedEmittedAggregate());
+            }
+            exitStatus = 0;
+        } catch (Exception e) {
+            LOG.error("Error trying to capture topologies...", e);
+        } finally {
+            System.exit(exitStatus);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
new file mode 100644
index 0000000..7998fdc
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GenLoad.java
@@ -0,0 +1,235 @@
+/**
+ * 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.loadgen;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.storm.Config;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.generated.AlreadyAliveException;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.InvalidTopologyException;
+import org.apache.storm.metric.LoggingMetricsConsumer;
+import org.apache.storm.topology.BoltDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.utils.NimbusClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Generate a simulated load.
+ */
+public class GenLoad {
+    private static final Logger LOG = LoggerFactory.getLogger(GenLoad.class);
+    private static final int TEST_EXECUTE_TIME_DEFAULT = 5;
+
+    /**
+     * Main entry point for GenLoad application.
+     * @param args the command line args.
+     * @throws Exception on any error.
+     */
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+        options.addOption(Option.builder("h")
+            .longOpt("help")
+            .desc("Print a help message")
+            .build());
+        options.addOption(Option.builder("t")
+            .longOpt("test-time")
+            .argName("MINS")
+            .hasArg()
+            .desc("How long to run the tests for in mins (defaults to " + TEST_EXECUTE_TIME_DEFAULT + ")")
+            .build());
+        options.addOption(Option.builder()
+            .longOpt("parallel")
+            .argName("MULTIPLIER")
+            .hasArg()
+            .desc("How much to scale the topology up or down in parallelism.\n"
+                + "The new parallelism will round up to the next whole number\n"
+                + "(defaults to 1.0 no scaling)")
+            .build());
+        options.addOption(Option.builder()
+            .longOpt("throughput")
+            .argName("MULTIPLIER")
+            .hasArg()
+            .desc("How much to scale the topology up or down in throughput.\n"
+                + "Note this is applied after and build on any parallelism changes.\n"
+                + "(defaults to 1.0 no scaling)")
+            .build());
+        options.addOption(Option.builder()
+            .longOpt("local-or-shuffle")
+            .desc("replace shuffle grouping with local or shuffle grouping")
+            .build());
+        options.addOption(Option.builder()
+            .longOpt("debug")
+            .desc("Print debug information about the adjusted topology before submitting it.")
+            .build());
+        LoadMetricsServer.addCommandLineOptions(options);
+        CommandLineParser parser = new DefaultParser();
+        CommandLine cmd = null;
+        Exception commandLineException = null;
+        double executeTime = TEST_EXECUTE_TIME_DEFAULT;
+        double parallel = 1.0;
+        double throughput = 1.0;
+        try {
+            cmd = parser.parse(options, args);
+            if (cmd.hasOption("t")) {
+                executeTime = Double.valueOf(cmd.getOptionValue("t"));
+            }
+            if (cmd.hasOption("parallel")) {
+                parallel = Double.parseDouble(cmd.getOptionValue("parallel"));
+            }
+            if (cmd.hasOption("throughput")) {
+                throughput = Double.parseDouble(cmd.getOptionValue("throughput"));
+            }
+        } catch (ParseException | NumberFormatException e) {
+            commandLineException = e;
+        }
+        if (commandLineException != null || cmd.hasOption('h')) {
+            if (commandLineException != null) {
+                System.err.println("ERROR " + commandLineException.getMessage());
+            }
+            new HelpFormatter().printHelp("GenLoad [options] [captured_file]*", options);
+            return;
+        }
+        Config conf = new Config();
+        LoadMetricsServer metricServer = new LoadMetricsServer(conf, cmd);
+
+        metricServer.serve();
+        String url = metricServer.getUrl();
+        int exitStatus = -1;
+        try (NimbusClient client = NimbusClient.getConfiguredClient(conf);
+             ScopedTopologySet topoNames = new ScopedTopologySet(client.getClient())) {
+            for (String topoFile : cmd.getArgList()) {
+                try {
+                    TopologyLoadConf tlc = readTopology(topoFile);
+                    if (parallel != 1.0) {
+                        tlc = tlc.scaleParallel(parallel);
+                    }
+                    if (throughput != 1.0) {
+                        tlc = tlc.scaleThroughput(throughput);
+                    }
+                    if (cmd.hasOption("local-or-shuffle")) {
+                        tlc = tlc.replaceShuffleWithLocalOrShuffle();
+                    }
+                    if (cmd.hasOption("debug")) {
+                        LOG.info("DEBUGGING: {}", tlc.toYamlString());
+                    }
+                    topoNames.add(parseAndSubmit(tlc, url));
+                } catch (Exception e) {
+                    System.err.println("Could Not Submit Topology From " + topoFile);
+                    e.printStackTrace(System.err);
+                }
+            }
+
+            metricServer.monitorFor(executeTime, client.getClient(), topoNames);
+            exitStatus = 0;
+        } catch (Exception e) {
+            LOG.error("Error trying to run topologies...", e);
+        } finally {
+            System.exit(exitStatus);
+        }
+    }
+
+    private static TopologyLoadConf readTopology(String topoFile) throws IOException {
+        File f = new File(topoFile);
+
+        TopologyLoadConf tlc = TopologyLoadConf.fromConf(f);
+        if (tlc.name == null) {
+            String fileName = f.getName();
+            int dot = fileName.lastIndexOf('.');
+            final String baseName = fileName.substring(0, dot);
+            tlc = tlc.withName(baseName);
+        }
+        return tlc;
+    }
+
+    static int uniquifier = 0;
+
+    private static String parseAndSubmit(TopologyLoadConf tlc, String url) throws IOException, InvalidTopologyException,
+        AuthorizationException, AlreadyAliveException {
+
+        //First we need some configs
+        Config conf = new Config();
+        if (tlc.topoConf != null) {
+            conf.putAll(tlc.topoConf);
+        }
+        //For some reason on the new code if ackers is null we get 0???
+        Object ackers = conf.get(Config.TOPOLOGY_ACKER_EXECUTORS);
+        Object workers = conf.get(Config.TOPOLOGY_WORKERS);
+        if (ackers == null || ((Number)ackers).intValue() <= 0) {
+            if (workers == null) {
+                workers = 1;
+            }
+            conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, workers);
+        }
+        conf.registerMetricsConsumer(LoggingMetricsConsumer.class);
+        conf.registerMetricsConsumer(HttpForwardingMetricsConsumer.class, url, 1);
+        Map<String, String> workerMetrics = new HashMap<>();
+        if (!NimbusClient.isLocalOverride()) {
+            //sigar uses JNI and does not work in local mode
+            workerMetrics.put("CPU", "org.apache.storm.metrics.sigar.CPUMetric");
+        }
+        conf.put(Config.TOPOLOGY_WORKER_METRICS, workerMetrics);
+        conf.put(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS, 10);
+
+        //Lets build a topology.
+        TopologyBuilder builder = new TopologyBuilder();
+        for (LoadCompConf spoutConf : tlc.spouts) {
+            System.out.println("ADDING SPOUT " + spoutConf.id);
+            builder.setSpout(spoutConf.id, new LoadSpout(spoutConf), spoutConf.parallelism);
+        }
+
+        Map<String, BoltDeclarer> boltDeclarers = new HashMap<>();
+        Map<String, LoadBolt> bolts = new HashMap<>();
+        if (tlc.bolts != null) {
+            for (LoadCompConf boltConf : tlc.bolts) {
+                System.out.println("ADDING BOLT " + boltConf.id);
+                LoadBolt lb = new LoadBolt(boltConf);
+                bolts.put(boltConf.id, lb);
+                boltDeclarers.put(boltConf.id, builder.setBolt(boltConf.id, lb, boltConf.parallelism));
+            }
+        }
+
+        if (tlc.streams != null) {
+            for (InputStream in : tlc.streams) {
+                BoltDeclarer declarer = boltDeclarers.get(in.toComponent);
+                if (declarer == null) {
+                    throw new IllegalArgumentException("to bolt " + in.toComponent + " does not exist");
+                }
+                LoadBolt lb = bolts.get(in.toComponent);
+                lb.add(in);
+                in.groupingType.assign(declarer, in);
+            }
+        }
+
+        String topoName = tlc.name + "-" + uniquifier++;
+        StormSubmitter.submitTopology(topoName, conf, builder.createTopology());
+        return topoName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GroupingType.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GroupingType.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GroupingType.java
new file mode 100644
index 0000000..a4e0c1a
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/GroupingType.java
@@ -0,0 +1,91 @@
+/**
+ * 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.loadgen;
+
+import java.util.Locale;
+import org.apache.storm.topology.BoltDeclarer;
+import org.apache.storm.tuple.Fields;
+
+/**
+ * The different types of groupings that are supported.
+ */
+public enum GroupingType {
+    SHUFFLE {
+        @Override
+        public void assign(BoltDeclarer declarer, InputStream stream) {
+            declarer.shuffleGrouping(stream.fromComponent, stream.id);
+        }
+    },
+    FIELDS {
+        @Override
+        public void assign(BoltDeclarer declarer, InputStream stream) {
+            declarer.fieldsGrouping(stream.fromComponent, stream.id, new Fields("key"));
+        }
+    },
+    ALL {
+        @Override
+        public void assign(BoltDeclarer declarer, InputStream stream) {
+            declarer.allGrouping(stream.fromComponent, stream.id);
+        }
+    },
+    GLOBAL {
+        @Override
+        public void assign(BoltDeclarer declarer, InputStream stream) {
+            declarer.globalGrouping(stream.fromComponent, stream.id);
+        }
+    },
+    LOCAL_OR_SHUFFLE {
+        @Override
+        public void assign(BoltDeclarer declarer, InputStream stream) {
+            declarer.localOrShuffleGrouping(stream.fromComponent, stream.id);
+        }
+    },
+    NONE {
+        @Override
+        public void assign(BoltDeclarer declarer, InputStream stream) {
+            declarer.noneGrouping(stream.fromComponent, stream.id);
+        }
+    },
+    PARTIAL_KEY {
+        @Override
+        public void assign(BoltDeclarer declarer, InputStream stream) {
+            declarer.partialKeyGrouping(stream.fromComponent, stream.id, new Fields("key"));
+        }
+    };
+
+    /**
+     * Parse a String config value and covert it into the enum.
+     * @param conf the string config.
+     * @return the parsed grouping type or SHUFFLE if conf is null.
+     * @throws IllegalArgumentException if parsing does not work.
+     */
+    public static GroupingType fromConf(String conf) {
+        String gt = "SHUFFLE";
+        if (conf != null) {
+            gt = conf.toUpperCase(Locale.ENGLISH);
+        }
+        return GroupingType.valueOf(gt);
+    }
+
+    public String toConf() {
+        return toString();
+    }
+
+    public abstract void assign(BoltDeclarer declarer, InputStream stream);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
index aa4579c..5829e9d 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java
@@ -15,49 +15,46 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.storm.misc.metric;
 
+package org.apache.storm.loadgen;
+
+import com.esotericsoftware.kryo.io.Output;
+import java.net.HttpURLConnection;
+import java.net.URL;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Map;
-import java.net.URL;
-import java.net.HttpURLConnection;
-
-import com.esotericsoftware.kryo.io.Output;
-import org.apache.storm.serialization.KryoValuesSerializer;
-
 import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.serialization.KryoValuesSerializer;
 import org.apache.storm.task.IErrorReporter;
 import org.apache.storm.task.TopologyContext;
 
 /**
- * Listens for all metrics and POSTs them serialized to a configured URL
- *
- * To use, add this to your topology's configuration:
+ * Listens for all metrics and POSTs them serialized to a configured URL.
  *
+ * <p>To use, add this to your topology's configuration:
  * ```java
- *   conf.registerMetricsConsumer(org.apache.storm.metrics.HttpForwardingMetricsConsumer.class, "http://example.com:8080/metrics/my-topology/", 1);
+ *   conf.registerMetricsConsumer(HttpForwardingMetricsConsumer.class, "http://example.com:8080/metrics/my-topology/", 1);
  * ```
  *
- * The body of the post is data serialized using {@link org.apache.storm.serialization.KryoValuesSerializer}, with the data passed in
- * as a list of `[TaskInfo, Collection<DataPoint>]`.  More things may be appended to the end of the list in the future.
+ * <p>The body of the post is data serialized using {@link org.apache.storm.serialization.KryoValuesSerializer}, with the data passed in
+ * as a list of `[TaskInfo, Collection&lt;DataPoint&gt;]`.  More things may be appended to the end of the list in the future.
  *
- * The values can be deserialized using the org.apache.storm.serialization.KryoValuesDeserializer, and a 
- * correct config + classpath.
+ * <p>The values can be deserialized using the org.apache.storm.serialization.KryoValuesDeserializer, and a correct config + classpath.
  *
- * @see org.apache.storm.serialization.KryoValuesSerializer
+ * <p>@see org.apache.storm.serialization.KryoValuesSerializer
  */
 public class HttpForwardingMetricsConsumer implements IMetricsConsumer {
-    private transient URL _url; 
-    private transient IErrorReporter _errorReporter;
-    private transient KryoValuesSerializer _serializer;
+    private transient URL url;
+    private transient IErrorReporter errorReporter;
+    private transient KryoValuesSerializer serializer;
 
     @Override
     public void prepare(Map<String, Object> topoConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) { 
         try {
-            _url = new URL((String)registrationArgument);
-            _errorReporter = errorReporter;
-            _serializer = new KryoValuesSerializer(topoConf);
+            url = new URL((String)registrationArgument);
+            this.errorReporter = errorReporter;
+            serializer = new KryoValuesSerializer(topoConf);
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
@@ -66,13 +63,13 @@ public class HttpForwardingMetricsConsumer implements IMetricsConsumer {
     @Override
     public void handleDataPoints(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
         try {
-            HttpURLConnection con = (HttpURLConnection)_url.openConnection();
+            HttpURLConnection con = (HttpURLConnection) url.openConnection();
             con.setRequestMethod("POST");
             con.setDoOutput(true);
-            Output out = new Output(con.getOutputStream());
-            _serializer.serializeInto(Arrays.asList(taskInfo, dataPoints), out);
-            out.flush();
-            out.close();
+            try (Output out = new Output(con.getOutputStream())) {
+                serializer.serializeInto(Arrays.asList(taskInfo, dataPoints), out);
+                out.flush();
+            }
             //The connection is not sent unless a response is requested
             int response = con.getResponseCode();
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
index ef2769a..99a980b 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsServer.java
@@ -15,27 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.storm.misc.metric;
 
+package org.apache.storm.loadgen;
+
+import com.esotericsoftware.kryo.io.Input;
 import java.io.IOException;
+import java.net.InetAddress;
+import java.net.ServerSocket;
 import java.util.Collection;
-import java.util.Map;
 import java.util.List;
-import java.net.ServerSocket;
-import java.net.InetAddress;
-
+import java.util.Map;
+import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
-import javax.servlet.ServletException;
-
-import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
 import org.apache.storm.metric.api.IMetricsConsumer.DataPoint;
-
-import com.esotericsoftware.kryo.io.Input;
+import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
 import org.apache.storm.serialization.KryoValuesDeserializer;
 import org.apache.storm.utils.Utils;
-
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
@@ -44,64 +41,72 @@ import org.eclipse.jetty.servlet.ServletHolder;
  * A server that can listen for metrics from the HttpForwardingMetricsConsumer.
  */
 public abstract class HttpForwardingMetricsServer {
-    private Map _conf;
-    private Server _server = null;
-    private int _port = -1;
-    private String _url = null;
+    private Map conf;
+    private Server server = null;
+    private int port = -1;
+    private String url = null;
 
-    ThreadLocal<KryoValuesDeserializer> _des = new ThreadLocal<KryoValuesDeserializer>() {
+    ThreadLocal<KryoValuesDeserializer> des = new ThreadLocal<KryoValuesDeserializer>() {
         @Override
         protected KryoValuesDeserializer initialValue() {
-            return new KryoValuesDeserializer(_conf);
+            return new KryoValuesDeserializer(conf);
         }
     };
 
-    private class MetricsCollectionServlet extends HttpServlet
-    {
-        protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException
-        {
+    private class MetricsCollectionServlet extends HttpServlet {
+        protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
             Input in = new Input(request.getInputStream());
-            List<Object> metrics = _des.get().deserializeFrom(in);
+            List<Object> metrics = des.get().deserializeFrom(in);
             handle((TaskInfo)metrics.get(0), (Collection<DataPoint>)metrics.get(1));
             response.setStatus(HttpServletResponse.SC_OK);
         }
     }
 
+    /**
+     * Constructor.
+     * @param conf the configuration for storm.
+     */
     public HttpForwardingMetricsServer(Map<String, Object> conf) {
-        _conf = Utils.readStormConfig();
+        this.conf = Utils.readStormConfig();
         if (conf != null) {
-            _conf.putAll(conf);
+            this.conf.putAll(conf);
         }
     }
 
     //This needs to be thread safe
     public abstract void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints);
 
+    /**
+     * Start the server.
+     * @param port the port it shuld listen on, or null/<= 0 to pick a free ephemeral port.
+     */
     public void serve(Integer port) {
         try {
-            if (_server != null) throw new RuntimeException("The server is already running");
+            if (server != null) {
+                throw new RuntimeException("The server is already running");
+            }
     
             if (port == null || port <= 0) {
                 ServerSocket s = new ServerSocket(0);
                 port = s.getLocalPort();
                 s.close();
             }
-            _server = new Server(port);
-            _port = port;
-            _url = "http://"+InetAddress.getLocalHost().getHostName()+":"+_port+"/";
+            server = new Server(port);
+            this.port = port;
+            url = "http://" + InetAddress.getLocalHost().getHostName() + ":" + this.port + "/";
  
             ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
             context.setContextPath("/");
-            _server.setHandler(context);
+            server.setHandler(context);
  
             context.addServlet(new ServletHolder(new MetricsCollectionServlet()),"/*");
 
-            _server.start();
-         } catch (RuntimeException e) {
-             throw e;
-         } catch (Exception e) {
-             throw new RuntimeException(e);
-         }
+            server.start();
+        } catch (RuntimeException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
     }
 
     public void serve() {
@@ -109,10 +114,10 @@ public abstract class HttpForwardingMetricsServer {
     }
 
     public int getPort() {
-        return _port;
+        return port;
     }
 
     public String getUrl() {
-        return _url;
+        return url;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/InputStream.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/InputStream.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/InputStream.java
new file mode 100644
index 0000000..19802d9
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/InputStream.java
@@ -0,0 +1,263 @@
+/**
+ * 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.loadgen;
+
+import java.io.ByteArrayInputStream;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.grouping.PartialKeyGrouping;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A set of measurements about a stream so we can statistically reproduce it.
+ */
+public class InputStream implements Serializable {
+    private static final Logger LOG = LoggerFactory.getLogger(InputStream.class);
+    public final String fromComponent;
+    public final String toComponent;
+    public final String id;
+    public final NormalDistStats execTime;
+    public final NormalDistStats processTime;
+    public final GroupingType groupingType;
+    //Cached GlobalStreamId
+    private GlobalStreamId gsid = null;
+
+    /**
+     * Create an output stream from a config.
+     * @param conf the config to read from.
+     * @return the read OutputStream.
+     */
+    public static InputStream fromConf(Map<String, Object> conf) {
+        String component = (String) conf.get("from");
+        String toComp = (String) conf.get("to");
+        NormalDistStats execTime = NormalDistStats.fromConf((Map<String, Object>) conf.get("execTime"));
+        NormalDistStats processTime = NormalDistStats.fromConf((Map<String, Object>) conf.get("processTime"));
+        Map<String, Object> grouping = (Map<String, Object>) conf.get("grouping");
+        GroupingType groupingType = GroupingType.fromConf((String) grouping.get("type"));
+        String streamId = (String) grouping.getOrDefault("streamId", "default");
+        return new InputStream(component, toComp, streamId, execTime, processTime, groupingType);
+    }
+
+    /**
+     * Convert this to a conf.
+     * @return the conf.
+     */
+    public Map<String, Object> toConf() {
+        Map<String, Object> ret = new HashMap<>();
+        ret.put("from", fromComponent);
+        ret.put("to", toComponent);
+        ret.put("execTime", execTime.toConf());
+        ret.put("processTime", processTime.toConf());
+
+        Map<String, Object> grouping = new HashMap<>();
+        grouping.put("streamId", id);
+        grouping.put("type", groupingType.toConf());
+        ret.put("grouping", grouping);
+
+        return ret;
+    }
+
+    public static class Builder {
+        private String fromComponent;
+        private String toComponent;
+        private String id;
+        private NormalDistStats execTime;
+        private NormalDistStats processTime;
+        private GroupingType groupingType = GroupingType.SHUFFLE;
+
+        public String getFromComponent() {
+            return fromComponent;
+        }
+
+        public Builder withFromComponent(String fromComponent) {
+            this.fromComponent = fromComponent;
+            return this;
+        }
+
+        public String getToComponent() {
+            return toComponent;
+        }
+
+        public Builder withToComponent(String toComponent) {
+            this.toComponent = toComponent;
+            return this;
+        }
+
+        public String getId() {
+            return id;
+        }
+
+        public Builder withId(String id) {
+            this.id = id;
+            return this;
+        }
+
+        public NormalDistStats getExecTime() {
+            return execTime;
+        }
+
+        public Builder withExecTime(NormalDistStats execTime) {
+            this.execTime = execTime;
+            return this;
+        }
+
+        public NormalDistStats getProcessTime() {
+            return processTime;
+        }
+
+        public Builder withProcessTime(NormalDistStats processTime) {
+            this.processTime = processTime;
+            return this;
+        }
+
+        public GroupingType getGroupingType() {
+            return groupingType;
+        }
+
+        public Builder withGroupingType(GroupingType groupingType) {
+            this.groupingType = groupingType;
+            return this;
+        }
+
+        /**
+         * Add the grouping type based off of the thrift Grouping class.
+         * @param grouping the Grouping to extract the grouping type from
+         * @return this
+         */
+        @SuppressWarnings("checkstyle:FallThrough")
+        public Builder withGroupingType(Grouping grouping) {
+            GroupingType group = GroupingType.SHUFFLE;
+            Grouping._Fields thriftType = grouping.getSetField();
+
+            switch (thriftType) {
+                case FIELDS:
+                    //Global Grouping is fields with an empty list
+                    if (grouping.get_fields().isEmpty()) {
+                        group = GroupingType.GLOBAL;
+                    } else {
+                        group = GroupingType.FIELDS;
+                    }
+                    break;
+                case ALL:
+                    group = GroupingType.ALL;
+                    break;
+                case NONE:
+                    group = GroupingType.NONE;
+                    break;
+                case SHUFFLE:
+                    group = GroupingType.SHUFFLE;
+                    break;
+                case LOCAL_OR_SHUFFLE:
+                    group = GroupingType.LOCAL_OR_SHUFFLE;
+                    break;
+                case CUSTOM_SERIALIZED:
+                    //This might be a partial key grouping..
+                    byte[] data = grouping.get_custom_serialized();
+                    try (ByteArrayInputStream bis = new ByteArrayInputStream(data);
+                         ObjectInputStream ois = new ObjectInputStream(bis);) {
+                        Object cg = ois.readObject();
+                        if (cg instanceof PartialKeyGrouping) {
+                            group = GroupingType.PARTIAL_KEY;
+                            break;
+                        }
+                    } catch (Exception e) {
+                        //ignored
+                    }
+                    //Fall through if not supported
+                default:
+                    LOG.warn("{} is not supported for replay of a topology.  Using SHUFFLE", thriftType);
+                    break;
+            }
+            return withGroupingType(group);
+        }
+
+        public InputStream build() {
+            return new InputStream(fromComponent, toComponent, id, execTime, processTime, groupingType);
+        }
+    }
+
+    /**
+     * Create a new input stream to a bolt.
+     * @param fromComponent the source component of the stream.
+     * @param id the id of the stream
+     * @param execTime exec time stats
+     * @param processTime process time stats
+     */
+    public InputStream(String fromComponent, String toComponent, String id, NormalDistStats execTime,
+                       NormalDistStats processTime, GroupingType groupingType) {
+        this.fromComponent = fromComponent;
+        this.toComponent = toComponent;
+        if (fromComponent == null) {
+            throw new IllegalArgumentException("from cannot be null");
+        }
+        if (toComponent == null) {
+            throw new IllegalArgumentException("to cannot be null");
+        }
+        this.id = id;
+        if (id == null) {
+            throw new IllegalArgumentException("id cannot be null");
+        }
+        this.execTime = execTime;
+        this.processTime = processTime;
+        this.groupingType = groupingType;
+        if (groupingType == null) {
+            throw new IllegalArgumentException("grouping type cannot be null");
+        }
+    }
+
+    /**
+     * Get the global stream id for this input stream.
+     * @return the GlobalStreamId for this input stream.
+     */
+    public synchronized GlobalStreamId gsid() {
+        if (gsid == null) {
+            gsid = new GlobalStreamId(fromComponent, id);
+        }
+        return gsid;
+    }
+
+    /**
+     * Remap the names of components.
+     * @param remappedComponents old name to new name of components.
+     * @param remappedStreams old ID to new ID of streams.
+     * @return a modified version of this with names remapped.
+     */
+    public InputStream remap(Map<String, String> remappedComponents, Map<GlobalStreamId, GlobalStreamId> remappedStreams) {
+        String remapTo = remappedComponents.get(toComponent);
+        String remapFrom = remappedComponents.get(fromComponent);
+        GlobalStreamId remapStreamId = remappedStreams.get(gsid());
+        return new InputStream(remapFrom, remapTo, remapStreamId.get_streamId(), execTime, processTime, groupingType);
+    }
+
+    /**
+     * Replace all SHUFFLE groupings with LOCAL_OR_SHUFFLE.
+     * @return a modified copy of this
+     */
+    public InputStream replaceShuffleWithLocalOrShuffle() {
+        if (groupingType != GroupingType.SHUFFLE) {
+            return this;
+        }
+        return new InputStream(fromComponent, toComponent, id, execTime, processTime, GroupingType.LOCAL_OR_SHUFFLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadBolt.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadBolt.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadBolt.java
new file mode 100644
index 0000000..e02e8f8
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadBolt.java
@@ -0,0 +1,146 @@
+/**
+ * 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.loadgen;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.LockSupport;
+import java.util.stream.Collectors;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A bolt that simulates a real world bolt based off of statistics about it.
+ */
+public class LoadBolt extends BaseRichBolt {
+    private static final Logger LOG = LoggerFactory.getLogger(LoadBolt.class);
+    private static final long NANO_IN_MS = TimeUnit.NANOSECONDS.convert(1, TimeUnit.MILLISECONDS);
+    private final List<OutputStream> outputStreamStats;
+    private List<OutputStreamEngine> outputStreams;
+    private final Map<GlobalStreamId, InputStream> inputStreams = new HashMap<>();
+    private OutputCollector collector;
+    private Random rand;
+    private ScheduledExecutorService timer;
+
+    private static long toNano(double ms) {
+        return (long)(ms * NANO_IN_MS);
+    }
+
+    public LoadBolt(LoadCompConf conf) {
+        this.outputStreamStats = Collections.unmodifiableList(new ArrayList<>(conf.streams));
+    }
+
+    public void add(InputStream inputStream) {
+        GlobalStreamId id = inputStream.gsid();
+        inputStreams.put(id, inputStream);
+    }
+
+    @Override
+    public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
+        outputStreams = Collections.unmodifiableList(outputStreamStats.stream()
+            .map((ss) -> new OutputStreamEngine(ss)).collect(Collectors.toList()));
+        this.collector = collector;
+        this.rand = ThreadLocalRandom.current();
+        this.timer = Executors.newSingleThreadScheduledExecutor();
+    }
+
+    private final AtomicLong parkOffset = new AtomicLong(0);
+
+    private void mySleep(long endTime) {
+        //There are some different levels of accuracy here, and we want to deal with all of them
+        long start = System.nanoTime();
+        long newEnd = endTime - parkOffset.get();
+        long diff = newEnd - start;
+        if (diff <= 1_000) {
+            //We are done, nothing that short is going to work here
+        } else if (diff < NANO_IN_MS) {
+            //Busy wait...
+            long sum = 0;
+            while (System.nanoTime() < newEnd) {
+                for (long i = 0; i < 1_000_000; i++) {
+                    sum += i;
+                }
+            }
+        } else {
+            //More accurate that thread.sleep, but still not great
+            LockSupport.parkNanos(newEnd - System.nanoTime() - parkOffset.get());
+            // A small control algorithm to adjust the amount of time that we sleep to make it more accurate
+        }
+        parkOffset.addAndGet((System.nanoTime() - endTime) / 2);
+    }
+
+    private void emitTuples(Tuple input) {
+        for (OutputStreamEngine se: outputStreams) {
+            // we may output many tuples for a given input tuple
+            while (se.shouldEmit() != null) {
+                collector.emit(se.streamName, input, new Values(se.nextKey(), "SOME-BOLT-VALUE"));
+            }
+        }
+    }
+
+    @Override
+    public void execute(final Tuple input) {
+        long startTimeNs = System.nanoTime();
+        InputStream in = inputStreams.get(input.getSourceGlobalStreamId());
+        if (in == null) {
+            emitTuples(input);
+            collector.ack(input);
+        } else {
+            long endExecNs = startTimeNs + toNano(in.execTime.nextRandom(rand));
+            long endProcNs = startTimeNs + toNano(in.processTime.nextRandom(rand));
+
+            if ((endProcNs - 1_000_000) < endExecNs) {
+                mySleep(endProcNs);
+                emitTuples(input);
+                collector.ack(input);
+            } else {
+                timer.schedule(() -> {
+                    emitTuples(input);
+                    collector.ack(input);
+                }, Math.max(0, endProcNs - System.nanoTime()), TimeUnit.NANOSECONDS);
+            }
+
+            mySleep(endExecNs);
+        }
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        for (OutputStream s: outputStreamStats) {
+            declarer.declareStream(s.id, new Fields("key", "value"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadCompConf.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadCompConf.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadCompConf.java
new file mode 100644
index 0000000..baead0f
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadCompConf.java
@@ -0,0 +1,222 @@
+/**
+ * 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.loadgen;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.utils.ObjectReader;
+
+/**
+ * Configuration for a simulated spout.
+ */
+public class LoadCompConf {
+    public final String id;
+    public final int parallelism;
+    public final List<OutputStream> streams;
+    public final CompStats stats;
+
+    /**
+     * Parse the LoadCompConf from a config Map.
+     * @param conf the map holding the config for a LoadCompConf.
+     * @return the parsed object.
+     */
+    public static LoadCompConf fromConf(Map<String, Object> conf) {
+        String id = (String) conf.get("id");
+        int parallelism = ObjectReader.getInt(conf.get("parallelism"), 1);
+        List<OutputStream> streams = new ArrayList<>();
+        List<Map<String, Object>> streamData = (List<Map<String, Object>>) conf.get("streams");
+        if (streamData != null) {
+            for (Map<String, Object> streamInfo: streamData) {
+                streams.add(OutputStream.fromConf(streamInfo));
+            }
+        }
+
+        return new LoadCompConf(id, parallelism, streams, CompStats.fromConf(conf));
+    }
+
+    /**
+     * Build a config map for this object.
+     * @return the config map.
+     */
+    public Map<String, Object> toConf() {
+        Map<String, Object> ret = new HashMap<>();
+        ret.put("id", id);
+        ret.put("parallelism", parallelism);
+
+        if (streams != null) {
+            List<Map<String, Object>> streamData = new ArrayList<>();
+            for (OutputStream out : streams) {
+                streamData.add(out.toConf());
+            }
+            ret.put("streams", streamData);
+        }
+        if (stats != null) {
+            stats.addToConf(ret);
+        }
+        return ret;
+    }
+
+    /**
+     * Chenge the name of components and streams according to the parameters passed in.
+     * @param remappedComponents original component name to new component name.
+     * @param remappedStreams original stream id to new stream id.
+     * @return a copy of this with the values remapped.
+     */
+    public LoadCompConf remap(Map<String, String> remappedComponents, Map<GlobalStreamId, GlobalStreamId> remappedStreams) {
+        String remappedId = remappedComponents.get(id);
+        List<OutputStream> remappedOutStreams = (streams == null) ? null :
+            streams.stream()
+                .map((orig) -> orig.remap(id, remappedStreams))
+                .collect(Collectors.toList());
+
+        return new LoadCompConf(remappedId, parallelism, remappedOutStreams, stats);
+    }
+
+    /**
+     * Scale the parallelism of this component by v.  The aggregate throughput will be the same.
+     * The parallelism will be rounded up to the next largest whole number.  Parallelism will always be at least 1.
+     * @param v 1.0 is not change 0.5 is drop the parallelism by half.
+     * @return a copy of this with the parallelism adjusted.
+     */
+    public LoadCompConf scaleParallel(double v) {
+        return setParallel(Math.max(1, (int)Math.ceil(parallelism * v)));
+    }
+
+    /**
+     * Set the parallelism of this component, and adjust the throughput so in aggregate it stays the same.
+     * @param newParallelism the new parallelism to set.
+     * @return a copy of this with the adjustments made.
+     */
+    public LoadCompConf setParallel(int newParallelism) {
+        //We need to adjust the throughput accordingly (so that it stays the same in aggregate)
+        double throughputAdjustment = ((double)parallelism) / newParallelism;
+        return new LoadCompConf(id, newParallelism, streams, stats).scaleThroughput(throughputAdjustment);
+    }
+
+    /**
+     * Scale the throughput of this component.
+     * @param v 1.0 is unchanged 0.5 will cut the throughput in half.
+     * @return a copu of this with the adjustments made.
+     */
+    public LoadCompConf scaleThroughput(double v) {
+        if (streams != null) {
+            List<OutputStream> newStreams = streams.stream().map((s) -> s.scaleThroughput(v)).collect(Collectors.toList());
+            return new LoadCompConf(id, parallelism, newStreams, stats);
+        } else {
+            return this;
+        }
+    }
+
+    /**
+     * Compute the total amount of all messages emitted in all streams per second.
+     * @return the sum of all messages emitted per second.
+     */
+    public double getAllEmittedAggregate() {
+        double ret = 0;
+        if (streams != null) {
+            for (OutputStream out: streams) {
+                if (out.rate != null) {
+                    ret += out.rate.mean * parallelism;
+                }
+            }
+        }
+        return ret;
+    }
+
+    public static class Builder {
+        private String id;
+        private int parallelism = 1;
+        private List<OutputStream> streams;
+        private CompStats stats;
+
+        public String getId() {
+            return id;
+        }
+
+        public Builder withId(String id) {
+            this.id = id;
+            return this;
+        }
+
+        public int getParallelism() {
+            return parallelism;
+        }
+
+        public Builder withParallelism(int parallelism) {
+            this.parallelism = parallelism;
+            return this;
+        }
+
+        public List<OutputStream> getStreams() {
+            return streams;
+        }
+
+        /**
+         * Add in a single OutputStream to this component.
+         * @param stream the stream to add
+         * @return this
+         */
+        public Builder withStream(OutputStream stream) {
+            if (streams == null) {
+                streams = new ArrayList<>();
+            }
+            streams.add(stream);
+            return this;
+        }
+
+        public Builder withStreams(List<OutputStream> streams) {
+            this.streams = streams;
+            return this;
+        }
+
+        public CompStats getStats() {
+            return stats;
+        }
+
+        public Builder withStats(CompStats stats) {
+            this.stats = stats;
+            return this;
+        }
+
+        public LoadCompConf build() {
+            return new LoadCompConf(id, parallelism, streams, stats);
+        }
+    }
+
+    /**
+     * Create a new LoadCompConf with the given values.
+     * @param id the id of the component.
+     * @param parallelism tha parallelism of the component.
+     * @param streams the output streams of the component.
+     * @param stats the stats of the component.
+     */
+    public LoadCompConf(String id, int parallelism, List<OutputStream> streams, CompStats stats) {
+        this.id = id;
+        if (id == null) {
+            throw new IllegalArgumentException("A spout ID cannot be null");
+        }
+        this.parallelism = parallelism;
+        this.streams = streams;
+        this.stats = stats;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/6c2dcbed/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadEngine.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadEngine.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadEngine.java
new file mode 100644
index 0000000..9030379
--- /dev/null
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadEngine.java
@@ -0,0 +1,45 @@
+/**
+ * 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.loadgen;
+
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.storm.loadgen.CompStats;
+
+/**
+ * The goal of this class is to provide a set of "Tuples" to send that will match as closely as possible the characteristics
+ * measured from a production topology.
+ */
+public class LoadEngine {
+
+    //TODO need to do a lot...
+
+    /**
+     * Provides an API to simulate the timings and CPU utilization of a bolt or spout.
+     */
+    public static class InputTimingEngine {
+        private final Random rand;
+        private final CompStats stats;
+
+        public InputTimingEngine(CompStats stats) {
+            this.stats = stats;
+            rand = ThreadLocalRandom.current();
+        }
+    }
+}


[15/18] storm git commit: Added in latency as reported by the ui

Posted by bo...@apache.org.
Added in latency as reported by the ui


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/19508b94
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/19508b94
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/19508b94

Branch: refs/heads/master
Commit: 19508b94253a813aca4adca8ef373cef87a69e6d
Parents: 96e68c0
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Sep 1 10:28:22 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Sep 1 10:28:22 2017 -0500

----------------------------------------------------------------------
 examples/storm-loadgen/README.md                |  1 +
 .../apache/storm/loadgen/LoadMetricsServer.java | 26 ++++++++++++++++++--
 2 files changed, 25 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/19508b94/examples/storm-loadgen/README.md
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/README.md b/examples/storm-loadgen/README.md
index 6c476bd..1858ea4 100644
--- a/examples/storm-loadgen/README.md
+++ b/examples/storm-loadgen/README.md
@@ -129,6 +129,7 @@ There are a lot of different metrics supported
 |executors| The number of running executors in the monitored topologies | all
 |workers| The number of workers the monitored topologies are running on | all
 |skipped\_max\_spout| The number of ms in total that the spout reported it skipped trying to emit because of `topology.max.spout.pending`. This is the sum for all spouts and can be used to decide if setting the value higher will likely improve throughput. `congested` reports individual spouts that appear to be slowed down by this to a large degree. | all
+|ui\_complete\_latency| This is a special metric, as it is the average completion latency as reported on the ui for `:all-time`. Because it is comes from the UI it does not follow the normal windows.  Within a window the maximum value reported is used.  | all
 |target_rate| The target rate in sentences per second for the ThroughputVsLatency topology | ThroughputVsLatency
 |spout_parallel| The parallelism of the spout for the `ThroughputVsLatency` topology. | ThroughputVsLatency
 |split_parallel| The parallelism of the split bolt for the `ThroughputVsLatency` topology. | ThroughputVsLatency

http://git-wip-us.apache.org/repos/asf/storm/blob/19508b94/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
index e6c1616..a44ab45 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -50,6 +50,7 @@ import org.apache.storm.generated.ExecutorSummary;
 import org.apache.storm.generated.Nimbus;
 import org.apache.storm.generated.SpoutStats;
 import org.apache.storm.generated.TopologyInfo;
+import org.apache.storm.generated.TopologyPageInfo;
 import org.apache.storm.generated.TopologySummary;
 import org.apache.storm.metric.api.IMetricsConsumer;
 import org.apache.storm.utils.Utils;
@@ -91,6 +92,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
 
     public static class Measurements {
         private final Histogram histo;
+        private double uiCompleteLatency;
         private long skippedMaxSpoutMs;
         private double userMs;
         private double sysMs;
@@ -115,7 +117,8 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
          */
         public Measurements(long uptimeSecs, long acked, long timeWindow, long failed, Histogram histo,
                             double userMs, double sysMs, double gcMs, long memBytes, Set<String> topologyIds,
-                            long workers, long executors, long hosts, Map<String, String> congested, long skippedMaxSpoutMs) {
+                            long workers, long executors, long hosts, Map<String, String> congested, long skippedMaxSpoutMs,
+                            double uiCompleteLatency) {
             this.uptimeSecs = uptimeSecs;
             this.acked = acked;
             this.timeWindow = timeWindow;
@@ -131,6 +134,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             this.hosts = hosts;
             this.congested = congested;
             this.skippedMaxSpoutMs = skippedMaxSpoutMs;
+            this.uiCompleteLatency = uiCompleteLatency;
         }
 
         /**
@@ -152,6 +156,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             hosts = 0;
             congested = new HashMap<>();
             skippedMaxSpoutMs = 0;
+            uiCompleteLatency = 0.0;
         }
 
         /**
@@ -174,6 +179,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             hosts = Math.max(hosts, other.hosts);
             congested.putAll(other.congested);
             skippedMaxSpoutMs += other.skippedMaxSpoutMs;
+            uiCompleteLatency = Math.max(uiCompleteLatency, other.uiCompleteLatency);
         }
 
         public double getLatencyAtPercentile(double percential, TimeUnit unit) {
@@ -196,6 +202,10 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
             return convert(histo.getStdDeviation(), TimeUnit.NANOSECONDS, unit);
         }
 
+        public double getUiCompleteLatency(TimeUnit unit) {
+            return convert(uiCompleteLatency, TimeUnit.MILLISECONDS, unit);
+        }
+
         public double getUserTime(TimeUnit unit) {
             return convert(userMs, TimeUnit.MILLISECONDS, unit);
         }
@@ -424,6 +434,7 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         tmp.put("os_name", new MetricExtractor((m, unit) -> System.getProperty("os.name"), ""));
         tmp.put("os_version", new MetricExtractor((m, unit) -> System.getProperty("os.version"), ""));
         tmp.put("config_override", new MetricExtractor((m, unit) -> Utils.readCommandLineOpts(), ""));
+        tmp.put("ui_complete_latency", new MetricExtractor((m, unit) -> m.getUiCompleteLatency(unit)));
         NAMED_EXTRACTORS = Collections.unmodifiableMap(tmp);
     }
 
@@ -874,8 +885,12 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         int uptime = 0;
         long acked = 0;
         long failed = 0;
+        double totalLatMs = 0;
+        long totalLatCount = 0;
         for (String id: ids) {
             TopologyInfo info = client.getTopologyInfo(id);
+            @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
+            TopologyPageInfo tpi = client.getTopologyPageInfo(id, ":all-time", false);
             uptime = Math.max(uptime, info.get_uptime_secs());
             for (ExecutorSummary exec : info.get_executors()) {
                 hosts.add(exec.get_host());
@@ -900,6 +915,12 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                     }
                 }
             }
+            Double latency = tpi.get_topology_stats().get_window_to_complete_latencies_ms().get(":all-time");
+            Long latAcked = tpi.get_topology_stats().get_window_to_acked().get(":all-time");
+            if (latency != null && latAcked != null) {
+                totalLatCount += latAcked;
+                totalLatMs += (latAcked * latency);
+            }
         }
         @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
         long failedThisTime = failed - prevFailed;
@@ -923,7 +944,8 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
         long memBytes = readMemory();
 
         allCombined.add(new Measurements(uptime, ackedThisTime, thisTime, failedThisTime, copy, user, sys, gc, memBytes,
-            ids, workers.size(), executors, hosts.size(), congested.getAndSet(new ConcurrentHashMap<>()), skippedMaxSpout));
+            ids, workers.size(), executors, hosts.size(), congested.getAndSet(new ConcurrentHashMap<>()), skippedMaxSpout,
+            totalLatMs / totalLatCount));
         Measurements inWindow = Measurements.combine(allCombined, null, windowLength);
         for (MetricResultsReporter reporter: reporters) {
             reporter.reportWindow(inWindow, allCombined);


[11/18] storm git commit: Added in max spout pending detection

Posted by bo...@apache.org.
Added in max spout pending detection


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/0ef492e7
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/0ef492e7
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/0ef492e7

Branch: refs/heads/master
Commit: 0ef492e7e7189fe13bef067b3892bbd1be7457a0
Parents: a4c372c
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Thu Aug 24 17:36:22 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Thu Aug 24 17:36:22 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/storm/loadgen/LoadMetricsServer.java    | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0ef492e7/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
----------------------------------------------------------------------
diff --git a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
index 36050ae..31bea1f 100644
--- a/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
+++ b/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/LoadMetricsServer.java
@@ -972,6 +972,15 @@ public class LoadMetricsServer extends HttpForwardingMetricsServer {
                             "receive " + pop + "/" + cap);
                     }
                 }
+            } else if (dp.name.equals("__skipped-max-spout-ms")) {
+                if (dp.value instanceof Number) {
+                    double full = ((Number) dp.value).doubleValue() / 10_000.0; //The frequency of reporting
+                    if (full >= 0.8) {
+                        congested.get().put(
+                            topologyId + ":" + taskInfo.srcComponentId + ":" + taskInfo.srcTaskId,
+                            "max.spout.pending " + (int)(full * 100) + "%");
+                    }
+                }
             }
         }
     }