You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by av...@apache.org on 2018/05/02 23:02:54 UTC

[ambari] 01/04: AMBARI-23744 : Fix UUID computation issues in AMS.

This is an automated email from the ASF dual-hosted git repository.

avijayan pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/ambari.git

commit 525923fc7dfd09fae0864a4876b8ca21e2dcaafe
Author: Aravindan Vijayan <av...@hortonworks.com>
AuthorDate: Wed May 2 11:31:18 2018 -0700

    AMBARI-23744 : Fix UUID computation issues in AMS.
---
 .../sink/timeline/TimelineMetricUtils.java         |     65 +
 .../host/aggregator/AggregatorApplication.java     |    357 +-
 .../host/aggregator/TimelineMetricsHolder.java     |    137 +-
 .../host/aggregator/TimelineMetricsHolderTest.java |      5 +-
 .../timeline/AggregatedMetricsPublisherTest.java   |      3 +-
 .../sink/timeline/RawMetricsPublisherTest.java     |      4 +-
 .../conf/unix/sqlline/phoenix_utils.py             |      7 +-
 .../conf/unix/sqlline/sqlline.py                   |      4 +-
 .../metrics/core/timeline/FunctionUtils.java       |     47 +
 .../core/timeline/HBaseTimelineMetricsService.java |    115 +-
 .../core/timeline/PhoenixHBaseAccessor.java        |    195 +-
 .../core/timeline/TimelineMetricConfiguration.java |     16 +-
 .../timeline/TimelineMetricServiceSummary.java     |     74 +
 .../metrics/core/timeline/TimelineMetricStore.java |      5 +
 .../core/timeline/TimelineMetricsIgniteCache.java  |     19 +-
 .../core/timeline/TransientMetricReadHelper.java   |    139 +
 .../core/timeline/aggregators/AggregatorUtils.java |     13 -
 .../TimelineMetricAggregatorFactory.java           |      1 +
 .../aggregators/TimelineMetricAppAggregator.java   |      2 +
 .../TimelineMetricClusterAggregatorSecond.java     |     35 +-
 .../TimelineMetricFilteringHostAggregator.java     |      3 +-
 .../v2/TimelineMetricFilteringHostAggregator.java  |      2 +-
 .../availability/MetricCollectorHAController.java  |     23 +
 .../discovery/TimelineMetricMetadataManager.java   |    341 +-
 .../metrics/core/timeline/query/Condition.java     |      1 +
 .../core/timeline/query/ConditionBuilder.java      |     23 +-
 .../core/timeline/query/DefaultCondition.java      |    147 +-
 .../core/timeline/query/EmptyCondition.java        |      5 +
 .../core/timeline/query/PhoenixTransactSQL.java    |    387 +-
 .../query/SplitByMetricNamesCondition.java         |      5 +
 .../timeline/query/TransientMetricCondition.java   |    206 +
 .../timeline/uuid/HashBasedUuidGenStrategy.java    |     10 +-
 ...uidGenStrategy.java => MD5UuidGenStrategy.java} |     45 +-
 .../core/timeline/uuid/MetricUuidGenStrategy.java  |      8 -
 .../ambari/metrics/webapp/TimelineWebServices.java |     20 +
 .../main/resources/metrics_def/KAFKA_BROKER.dat    |  18271 ++-
 .../src/main/resources/metrics_def/NIMBUS.dat      | 137283 +++++++++++++++++-
 .../timeline/AbstractMiniHBaseClusterTest.java     |      3 +-
 .../core/timeline/ITPhoenixHBaseAccessor.java      |     12 +-
 .../core/timeline/PhoenixHBaseAccessorTest.java    |     17 +-
 .../core/timeline/TestPhoenixTransactSQL.java      |     83 +-
 .../core/timeline/TestTimelineMetricStore.java     |      5 +
 .../timeline/TimelineMetricsIgniteCacheTest.java   |      5 +-
 .../timeline/aggregators/ITClusterAggregator.java  |      2 +-
 .../TimelineMetricClusterAggregatorSecondTest.java |      3 +-
 .../timeline/discovery/TestMetadataManager.java    |     65 +-
 .../core/timeline/query/DefaultConditionTest.java  |      2 +
 .../uuid/TimelineMetricUuidManagerTest.java        |     54 +-
 .../ambari/server/upgrade/UpgradeCatalog270.java   |     17 +
 .../0.1.0/configuration/ams-site.xml               |     30 +-
 .../server/upgrade/UpgradeCatalog270Test.java      |     56 +
 51 files changed, 157547 insertions(+), 830 deletions(-)

diff --git a/ambari-metrics/ambari-metrics-common/src/main/java/org/apache/hadoop/metrics2/sink/timeline/TimelineMetricUtils.java b/ambari-metrics/ambari-metrics-common/src/main/java/org/apache/hadoop/metrics2/sink/timeline/TimelineMetricUtils.java
new file mode 100644
index 0000000..f140a8e
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-common/src/main/java/org/apache/hadoop/metrics2/sink/timeline/TimelineMetricUtils.java
@@ -0,0 +1,65 @@
+/**
+ * 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.hadoop.metrics2.sink.timeline;
+
+import org.apache.commons.lang.StringUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class TimelineMetricUtils {
+
+  /**
+   * Given a SQL regex, convert it to JAVA regex.
+   * @param sqlRegex
+   * @return
+   */
+  public static String getJavaRegexFromSqlRegex(String sqlRegex) {
+    String javaRegEx;
+    if (sqlRegex.contains("*") || sqlRegex.contains("__%")) {
+      //Special case handling for metric name with * and __%.
+      //For example, dfs.NNTopUserOpCounts.windowMs=300000.op=*.user=%.count
+      // or dfs.NNTopUserOpCounts.windowMs=300000.op=__%.user=%.count
+      String metricNameWithEscSeq = sqlRegex.replace("*", "\\*").replace("__%", "..%");
+      javaRegEx = metricNameWithEscSeq.replace("%", ".*");
+    } else {
+      javaRegEx = sqlRegex.replace("%", ".*");
+    }
+    return javaRegEx;
+  }
+
+  /**
+   * Wrapper method to split comma separated strings and then invoke getJavaRegexFromSqlRegex.
+   * @param commaSeparatedMetricPatternsString
+   * @return
+   */
+  public static List<String> getJavaMetricPatterns(String commaSeparatedMetricPatternsString) {
+
+    List<String> javaPatterns = new ArrayList<>();
+    if (StringUtils.isEmpty(commaSeparatedMetricPatternsString)) {
+      return javaPatterns;
+    }
+
+    for (String patternString : commaSeparatedMetricPatternsString.split(",")) {
+      String javaPatternString = getJavaRegexFromSqlRegex(patternString);
+      javaPatterns.add(javaPatternString);
+    }
+    return javaPatterns;
+  }
+}
diff --git a/ambari-metrics/ambari-metrics-host-aggregator/src/main/java/org/apache/hadoop/metrics2/host/aggregator/AggregatorApplication.java b/ambari-metrics/ambari-metrics-host-aggregator/src/main/java/org/apache/hadoop/metrics2/host/aggregator/AggregatorApplication.java
index f8ed95f..b5a4026 100644
--- a/ambari-metrics/ambari-metrics-host-aggregator/src/main/java/org/apache/hadoop/metrics2/host/aggregator/AggregatorApplication.java
+++ b/ambari-metrics/ambari-metrics-host-aggregator/src/main/java/org/apache/hadoop/metrics2/host/aggregator/AggregatorApplication.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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.
@@ -28,10 +28,14 @@ import java.net.InetAddress;
 import java.net.URI;
 import java.net.URL;
 import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 
 import com.sun.net.httpserver.HttpsConfigurator;
 import com.sun.net.httpserver.HttpsServer;
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -40,204 +44,213 @@ import org.apache.hadoop.metrics2.sink.timeline.AggregatedMetricsPublisher;
 import org.apache.hadoop.metrics2.sink.timeline.RawMetricsPublisher;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 
+import static org.apache.hadoop.metrics2.sink.timeline.TimelineMetricUtils.getJavaMetricPatterns;
+
 /**
  * WEB application with 2 publisher threads that processes received metrics and submits results to the collector
  */
-public class AggregatorApplication
-{
-    private static final int STOP_SECONDS_DELAY = 0;
-    private static final int JOIN_SECONDS_TIMEOUT = 5;
-    private static final String METRICS_SITE_CONFIGURATION_FILE = "ams-site.xml";
-    private static final String METRICS_SSL_SERVER_CONFIGURATION_FILE = "ssl-server.xml";
-    private Log LOG;
-    private final int webApplicationPort;
-    private final int rawPublishingInterval;
-    private final int aggregationInterval;
-    private final String webServerProtocol;
-    private Configuration configuration;
-    private Thread aggregatePublisherThread;
-    private Thread rawPublisherThread;
-    private TimelineMetricsHolder timelineMetricsHolder;
-    private HttpServer httpServer;
-
-    public AggregatorApplication(String hostname, String collectorHosts) {
-        LOG = LogFactory.getLog(this.getClass());
-        configuration = new Configuration(true);
-        initConfiguration();
-        configuration.set("timeline.metrics.collector.hosts", collectorHosts);
-        configuration.set("timeline.metrics.hostname", hostname);
-        configuration.set("timeline.metrics.zk.quorum", getZkQuorumFromConfiguration());
-        this.aggregationInterval = configuration.getInt("timeline.metrics.host.aggregator.minute.interval", 300);
-        this.rawPublishingInterval = configuration.getInt("timeline.metrics.sink.report.interval", 60);
-        this.webApplicationPort = configuration.getInt("timeline.metrics.host.inmemory.aggregation.port", 61888);
-        this.webServerProtocol = configuration.get("timeline.metrics.host.inmemory.aggregation.http.policy", "HTTP_ONLY").equalsIgnoreCase("HTTP_ONLY") ? "http" : "https";
-        this.timelineMetricsHolder = TimelineMetricsHolder.getInstance(rawPublishingInterval, aggregationInterval);
-        try {
-            this.httpServer = createHttpServer();
-        } catch (Exception e) {
-            LOG.error("Exception while starting HTTP server. Exiting", e);
-            System.exit(1);
-        }
-    }
+public class AggregatorApplication {
+  private static final int STOP_SECONDS_DELAY = 0;
+  private static final int JOIN_SECONDS_TIMEOUT = 5;
+  private static final String METRICS_SITE_CONFIGURATION_FILE = "ams-site.xml";
+  private static final String METRICS_SSL_SERVER_CONFIGURATION_FILE = "ssl-server.xml";
+  private Log LOG;
+  private final int webApplicationPort;
+  private final int rawPublishingInterval;
+  private final int aggregationInterval;
+  private final String webServerProtocol;
+  private Configuration configuration;
+  private Thread aggregatePublisherThread;
+  private Thread rawPublisherThread;
+  private TimelineMetricsHolder timelineMetricsHolder;
+  private HttpServer httpServer;
 
-    private String getZkQuorumFromConfiguration() {
-        String zkClientPort = configuration.getTrimmed("cluster.zookeeper.property.clientPort", "2181");
-        String zkServerHosts = configuration.getTrimmed("cluster.zookeeper.quorum", "");
-        return getZkConnectionUrl(zkClientPort, zkServerHosts);
-    }
+  public AggregatorApplication(String hostname, String collectorHosts) {
+    LOG = LogFactory.getLog(this.getClass());
+    configuration = new Configuration(true);
+    initConfiguration();
+    configuration.set("timeline.metrics.collector.hosts", collectorHosts);
+    configuration.set("timeline.metrics.hostname", hostname);
+    configuration.set("timeline.metrics.zk.quorum", getZkQuorumFromConfiguration());
+    this.aggregationInterval = configuration.getInt("timeline.metrics.host.aggregator.minute.interval", 300);
+    this.rawPublishingInterval = configuration.getInt("timeline.metrics.sink.report.interval", 60);
+    this.webApplicationPort = configuration.getInt("timeline.metrics.host.inmemory.aggregation.port", 61888);
+    this.webServerProtocol = configuration.get("timeline.metrics.host.inmemory.aggregation.http.policy",
+      "HTTP_ONLY").equalsIgnoreCase("HTTP_ONLY") ? "http" : "https";
 
-    protected void initConfiguration() {
-        ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
-        if (classLoader == null) {
-            classLoader = getClass().getClassLoader();
-        }
-
-        URL amsResUrl = classLoader.getResource(METRICS_SITE_CONFIGURATION_FILE);
-        LOG.info("Found metric service configuration: " + amsResUrl);
-        URL sslConfUrl = classLoader.getResource(METRICS_SSL_SERVER_CONFIGURATION_FILE);
-        LOG.info("Found metric service configuration: " + sslConfUrl);
-        if (amsResUrl == null) {
-            throw new IllegalStateException(String.format("Unable to initialize the metrics " +
-                    "subsystem. No %s present in the classpath.", METRICS_SITE_CONFIGURATION_FILE));
-        }
-        if (sslConfUrl == null) {
-            throw new IllegalStateException(String.format("Unable to initialize the metrics " +
-                    "subsystem. No %s present in the classpath.", METRICS_SSL_SERVER_CONFIGURATION_FILE));
-        }
-
-        try {
-            configuration.addResource(amsResUrl.toURI().toURL());
-            configuration.addResource(sslConfUrl.toURI().toURL());
-        } catch (Exception e) {
-            LOG.error("Couldn't init configuration. ", e);
-            System.exit(1);
-        }
+    // Skip aggregating transient metrics.
+    String skipAggregationMetricPatternsString = configuration.get("timeline.metrics.transient.metric.patterns", StringUtils.EMPTY);
+    List<String> skipAggregationPatterns = new ArrayList<>();
+    if (StringUtils.isNotEmpty(skipAggregationMetricPatternsString)) {
+      skipAggregationPatterns.addAll(getJavaMetricPatterns(skipAggregationMetricPatternsString));
     }
 
-    protected String getHostName() {
-        String hostName = "localhost";
-        try {
-            hostName = InetAddress.getLocalHost().getCanonicalHostName();
-        } catch (UnknownHostException e) {
-            LOG.error(e);
-        }
-        return hostName;
+    this.timelineMetricsHolder = TimelineMetricsHolder.getInstance(rawPublishingInterval, aggregationInterval, skipAggregationPatterns);
+    try {
+      this.httpServer = createHttpServer();
+    } catch (Exception e) {
+      LOG.error("Exception while starting HTTP server. Exiting", e);
+      System.exit(1);
     }
+  }
 
-    protected URI getURI() {
-        URI uri = UriBuilder.fromUri("/").scheme(this.webServerProtocol).host(getHostName()).port(this.webApplicationPort).build();
-        LOG.info(String.format("Web server at %s", uri));
-        return uri;
-    }
+  private String getZkQuorumFromConfiguration() {
+    String zkClientPort = configuration.getTrimmed("cluster.zookeeper.property.clientPort", "2181");
+    String zkServerHosts = configuration.getTrimmed("cluster.zookeeper.quorum", "");
+    return getZkConnectionUrl(zkClientPort, zkServerHosts);
+  }
 
-    protected HttpServer createHttpServer() throws Exception {
-        ResourceConfig resourceConfig = new PackagesResourceConfig("org.apache.hadoop.metrics2.host.aggregator");
-        HashMap<String, Object> params = new HashMap();
-        params.put("com.sun.jersey.api.json.POJOMappingFeature", "true");
-        resourceConfig.setPropertiesAndFeatures(params);
-        HttpServer server = HttpServerFactory.create(getURI(), resourceConfig);
-
-        if (webServerProtocol.equalsIgnoreCase("https")) {
-            HttpsServer httpsServer = (HttpsServer) server;
-            SslContextFactory sslContextFactory = new SslContextFactory();
-            String keyStorePath = configuration.get("ssl.server.keystore.location");
-            String keyStorePassword = configuration.get("ssl.server.keystore.password");
-            String keyManagerPassword = configuration.get("ssl.server.keystore.keypassword");
-            String trustStorePath = configuration.get("ssl.server.truststore.location");
-            String trustStorePassword = configuration.get("ssl.server.truststore.password");
-
-            sslContextFactory.setKeyStorePath(keyStorePath);
-            sslContextFactory.setKeyStorePassword(keyStorePassword);
-            sslContextFactory.setKeyManagerPassword(keyManagerPassword);
-            sslContextFactory.setTrustStorePath(trustStorePath);
-            sslContextFactory.setTrustStorePassword(trustStorePassword);
-
-            sslContextFactory.start();
-            SSLContext sslContext = sslContextFactory.getSslContext();
-            sslContextFactory.stop();
-            HttpsConfigurator httpsConfigurator = new HttpsConfigurator(sslContext);
-            httpsServer.setHttpsConfigurator(httpsConfigurator);
-            server = httpsServer;
-        }
-        return server;
+  protected void initConfiguration() {
+    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+    if (classLoader == null) {
+      classLoader = getClass().getClassLoader();
     }
 
-    private void startWebServer() {
-        LOG.info("Starting web server.");
-        this.httpServer.start();
+    URL amsResUrl = classLoader.getResource(METRICS_SITE_CONFIGURATION_FILE);
+    LOG.info("Found metric service configuration: " + amsResUrl);
+    URL sslConfUrl = classLoader.getResource(METRICS_SSL_SERVER_CONFIGURATION_FILE);
+    LOG.info("Found metric service configuration: " + sslConfUrl);
+    if (amsResUrl == null) {
+      throw new IllegalStateException(String.format("Unable to initialize the metrics " +
+        "subsystem. No %s present in the classpath.", METRICS_SITE_CONFIGURATION_FILE));
+    }
+    if (sslConfUrl == null) {
+      throw new IllegalStateException(String.format("Unable to initialize the metrics " +
+        "subsystem. No %s present in the classpath.", METRICS_SSL_SERVER_CONFIGURATION_FILE));
     }
 
-    private void startAggregatePublisherThread() {
-        LOG.info("Starting aggregated metrics publisher.");
-        AbstractMetricPublisher metricPublisher = new AggregatedMetricsPublisher(timelineMetricsHolder, configuration, aggregationInterval);
-        aggregatePublisherThread = new Thread(metricPublisher);
-        aggregatePublisherThread.start();
+    try {
+      configuration.addResource(amsResUrl.toURI().toURL());
+      configuration.addResource(sslConfUrl.toURI().toURL());
+    } catch (Exception e) {
+      LOG.error("Couldn't init configuration. ", e);
+      System.exit(1);
     }
+  }
 
-    private void startRawPublisherThread() {
-        LOG.info("Starting raw metrics publisher.");
-        AbstractMetricPublisher metricPublisher = new RawMetricsPublisher(timelineMetricsHolder, configuration, rawPublishingInterval);
-        rawPublisherThread = aggregatePublisherThread = new Thread(metricPublisher);
-        aggregatePublisherThread.start();
+  protected String getHostName() {
+    String hostName = "localhost";
+    try {
+      hostName = InetAddress.getLocalHost().getCanonicalHostName();
+    } catch (UnknownHostException e) {
+      LOG.error(e);
     }
+    return hostName;
+  }
 
+  protected URI getURI() {
+    URI uri = UriBuilder.fromUri("/").scheme(this.webServerProtocol).host(getHostName()).port(this.webApplicationPort).build();
+    LOG.info(String.format("Web server at %s", uri));
+    return uri;
+  }
 
+  protected HttpServer createHttpServer() throws Exception {
+    ResourceConfig resourceConfig = new PackagesResourceConfig("org.apache.hadoop.metrics2.host.aggregator");
+    HashMap<String, Object> params = new HashMap();
+    params.put("com.sun.jersey.api.json.POJOMappingFeature", "true");
+    resourceConfig.setPropertiesAndFeatures(params);
+    HttpServer server = HttpServerFactory.create(getURI(), resourceConfig);
 
-    private void stop() {
-        LOG.info("Stopping aggregator application");
-        aggregatePublisherThread.interrupt();
-        rawPublisherThread.interrupt();
-        httpServer.stop(STOP_SECONDS_DELAY);
-        LOG.info("Stopped web server.");
-        try {
-            LOG.info("Waiting for threads to join.");
-            aggregatePublisherThread.join(JOIN_SECONDS_TIMEOUT * 1000);
-            rawPublisherThread.join(JOIN_SECONDS_TIMEOUT * 1000);
-            LOG.info("Gracefully stopped Aggregator Application.");
-        } catch (InterruptedException e) {
-            LOG.error("Received exception during stop : ", e);
+    if (webServerProtocol.equalsIgnoreCase("https")) {
+      HttpsServer httpsServer = (HttpsServer) server;
+      SslContextFactory sslContextFactory = new SslContextFactory();
+      String keyStorePath = configuration.get("ssl.server.keystore.location");
+      String keyStorePassword = configuration.get("ssl.server.keystore.password");
+      String keyManagerPassword = configuration.get("ssl.server.keystore.keypassword");
+      String trustStorePath = configuration.get("ssl.server.truststore.location");
+      String trustStorePassword = configuration.get("ssl.server.truststore.password");
 
-        }
+      sslContextFactory.setKeyStorePath(keyStorePath);
+      sslContextFactory.setKeyStorePassword(keyStorePassword);
+      sslContextFactory.setKeyManagerPassword(keyManagerPassword);
+      sslContextFactory.setTrustStorePath(trustStorePath);
+      sslContextFactory.setTrustStorePassword(trustStorePassword);
 
+      sslContextFactory.start();
+      SSLContext sslContext = sslContextFactory.getSslContext();
+      sslContextFactory.stop();
+      HttpsConfigurator httpsConfigurator = new HttpsConfigurator(sslContext);
+      httpsServer.setHttpsConfigurator(httpsConfigurator);
+      server = httpsServer;
     }
+    return server;
+  }
 
-    private String getZkConnectionUrl(String zkClientPort, String zkQuorum) {
-        StringBuilder sb = new StringBuilder();
-        String[] quorumParts = zkQuorum.split(",");
-        String prefix = "";
-        for (String part : quorumParts) {
-            sb.append(prefix);
-            sb.append(part.trim());
-            if (!part.contains(":")) {
-                sb.append(":");
-                sb.append(zkClientPort);
-            }
-            prefix = ",";
-        }
-        return sb.toString();
-    }
+  private void startWebServer() {
+    LOG.info("Starting web server.");
+    this.httpServer.start();
+  }
+
+  private void startAggregatePublisherThread() {
+    LOG.info("Starting aggregated metrics publisher.");
+    AbstractMetricPublisher metricPublisher = new AggregatedMetricsPublisher(timelineMetricsHolder, configuration, aggregationInterval);
+    aggregatePublisherThread = new Thread(metricPublisher);
+    aggregatePublisherThread.start();
+  }
 
-    public static void main( String[] args ) throws Exception {
-        if (args.length != 2) {
-            throw new Exception("This jar should be executed with 2 arguments : 1st - current host name, " +
-                    "2nd - collector hosts separated with coma");
-        }
+  private void startRawPublisherThread() {
+    LOG.info("Starting raw metrics publisher.");
+    AbstractMetricPublisher metricPublisher = new RawMetricsPublisher(timelineMetricsHolder, configuration, rawPublishingInterval);
+    rawPublisherThread = aggregatePublisherThread = new Thread(metricPublisher);
+    aggregatePublisherThread.start();
+  }
 
-        final AggregatorApplication app = new AggregatorApplication(args[0], args[1]);
 
-        app.startWebServerAndPublishersThreads();
+  private void stop() {
+    LOG.info("Stopping aggregator application");
+    aggregatePublisherThread.interrupt();
+    rawPublisherThread.interrupt();
+    httpServer.stop(STOP_SECONDS_DELAY);
+    LOG.info("Stopped web server.");
+    try {
+      LOG.info("Waiting for threads to join.");
+      aggregatePublisherThread.join(JOIN_SECONDS_TIMEOUT * 1000);
+      rawPublisherThread.join(JOIN_SECONDS_TIMEOUT * 1000);
+      LOG.info("Gracefully stopped Aggregator Application.");
+    } catch (InterruptedException e) {
+      LOG.error("Received exception during stop : ", e);
 
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            public void run() {
-                app.stop();
-            }
-        });
     }
 
-    private void startWebServerAndPublishersThreads() {
-        LOG.info("Starting aggregator application");
-        startAggregatePublisherThread();
-        startRawPublisherThread();
-        startWebServer();
+  }
+
+  private String getZkConnectionUrl(String zkClientPort, String zkQuorum) {
+    StringBuilder sb = new StringBuilder();
+    String[] quorumParts = zkQuorum.split(",");
+    String prefix = "";
+    for (String part : quorumParts) {
+      sb.append(prefix);
+      sb.append(part.trim());
+      if (!part.contains(":")) {
+        sb.append(":");
+        sb.append(zkClientPort);
+      }
+      prefix = ",";
+    }
+    return sb.toString();
+  }
+
+  public static void main(String[] args) throws Exception {
+    if (args.length != 2) {
+      throw new Exception("This jar should be executed with 2 arguments : 1st - current host name, " +
+        "2nd - collector hosts separated with coma");
     }
+
+    final AggregatorApplication app = new AggregatorApplication(args[0], args[1]);
+
+    app.startWebServerAndPublishersThreads();
+
+    Runtime.getRuntime().addShutdownHook(new Thread() {
+      public void run() {
+        app.stop();
+      }
+    });
+  }
+
+  private void startWebServerAndPublishersThreads() {
+    LOG.info("Starting aggregator application");
+    startAggregatePublisherThread();
+    startRawPublisherThread();
+    startWebServer();
+  }
 }
diff --git a/ambari-metrics/ambari-metrics-host-aggregator/src/main/java/org/apache/hadoop/metrics2/host/aggregator/TimelineMetricsHolder.java b/ambari-metrics/ambari-metrics-host-aggregator/src/main/java/org/apache/hadoop/metrics2/host/aggregator/TimelineMetricsHolder.java
index 03b6542..a747f93 100644
--- a/ambari-metrics/ambari-metrics-host-aggregator/src/main/java/org/apache/hadoop/metrics2/host/aggregator/TimelineMetricsHolder.java
+++ b/ambari-metrics/ambari-metrics-host-aggregator/src/main/java/org/apache/hadoop/metrics2/host/aggregator/TimelineMetricsHolder.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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.
@@ -22,6 +22,9 @@ import com.google.common.cache.CacheBuilder;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -33,76 +36,88 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
  * Singleton class with 2 guava caches for raw and aggregated metrics storing
  */
 public class TimelineMetricsHolder {
-    private static final int DEFAULT_RAW_CACHE_EXPIRE_TIME = 60;
-    private static final int DEFAULT_AGGREGATION_CACHE_EXPIRE_TIME = 300;
-    private Cache<String, TimelineMetrics> aggregationMetricsCache;
-    private Cache<String, TimelineMetrics> rawMetricsCache;
-    private static TimelineMetricsHolder instance = null;
-    //to ensure no metric values are expired
-    private static int EXPIRE_DELAY = 30;
-    ReadWriteLock aggregationCacheLock = new ReentrantReadWriteLock();
-    ReadWriteLock rawCacheLock = new ReentrantReadWriteLock();
+  private static final int DEFAULT_RAW_CACHE_EXPIRE_TIME = 60;
+  private static final int DEFAULT_AGGREGATION_CACHE_EXPIRE_TIME = 300;
+  private Cache<String, TimelineMetrics> aggregationMetricsCache;
+  private Cache<String, TimelineMetrics> rawMetricsCache;
+  private static TimelineMetricsHolder instance = null;
+  //to ensure no metric values are expired
+  private static int EXPIRE_DELAY = 30;
+  ReadWriteLock aggregationCacheLock = new ReentrantReadWriteLock();
+  ReadWriteLock rawCacheLock = new ReentrantReadWriteLock();
+  private List<String> skipAggregationPatternStrings = new ArrayList<>();
 
-    private TimelineMetricsHolder(int rawCacheExpireTime, int aggregationCacheExpireTime) {
-        this.rawMetricsCache = CacheBuilder.newBuilder().expireAfterWrite(rawCacheExpireTime + EXPIRE_DELAY, TimeUnit.SECONDS).build();
-        this.aggregationMetricsCache = CacheBuilder.newBuilder().expireAfterWrite(aggregationCacheExpireTime + EXPIRE_DELAY, TimeUnit.SECONDS).build();
-    }
+  private TimelineMetricsHolder(int rawCacheExpireTime, int aggregationCacheExpireTime, List<String> skipAggregationPatternStrings) {
+    this.rawMetricsCache = CacheBuilder.newBuilder().expireAfterWrite(rawCacheExpireTime + EXPIRE_DELAY, TimeUnit.SECONDS).build();
+    this.aggregationMetricsCache = CacheBuilder.newBuilder().expireAfterWrite(aggregationCacheExpireTime + EXPIRE_DELAY, TimeUnit.SECONDS).build();
+    this.skipAggregationPatternStrings = skipAggregationPatternStrings;
+  }
 
-    public static TimelineMetricsHolder getInstance(int rawCacheExpireTime, int aggregationCacheExpireTime) {
-        if (instance == null) {
-            instance = new TimelineMetricsHolder(rawCacheExpireTime, aggregationCacheExpireTime);
-        }
-        return instance;
+  public static TimelineMetricsHolder getInstance(int rawCacheExpireTime, int aggregationCacheExpireTime, List<String> skipAggregationPatternStrings) {
+    if (instance == null) {
+      instance = new TimelineMetricsHolder(rawCacheExpireTime, aggregationCacheExpireTime, skipAggregationPatternStrings);
     }
+    return instance;
+  }
 
-    /**
-     * Uses default expiration time for caches initialization if they are not initialized yet.
-     * @return
-     */
-    public static TimelineMetricsHolder getInstance() {
-        return getInstance(DEFAULT_RAW_CACHE_EXPIRE_TIME, DEFAULT_AGGREGATION_CACHE_EXPIRE_TIME);
-    }
+  /**
+   * Uses default expiration time for caches initialization if they are not initialized yet.
+   * @return
+   */
+  public static TimelineMetricsHolder getInstance() {
+    return getInstance(DEFAULT_RAW_CACHE_EXPIRE_TIME, DEFAULT_AGGREGATION_CACHE_EXPIRE_TIME, Collections.emptyList());
+  }
 
-    public void putMetricsForAggregationPublishing(TimelineMetrics timelineMetrics) {
-        aggregationCacheLock.writeLock().lock();
-        aggregationMetricsCache.put(calculateCacheKey(timelineMetrics), timelineMetrics);
-        aggregationCacheLock.writeLock().unlock();
-    }
+  public void putMetricsForAggregationPublishing(TimelineMetrics timelineMetrics) {
 
-    private String calculateCacheKey(TimelineMetrics timelineMetrics) {
-        List<TimelineMetric>  metrics =  timelineMetrics.getMetrics();
-        if (metrics.size() > 0) {
-            return  metrics.get(0).getAppId() + System.currentTimeMillis();
+    //Remove metrics that need to be skipped during caching stage itself.
+    for (Iterator<TimelineMetric> iterator = timelineMetrics.getMetrics().iterator(); iterator.hasNext();) {
+      TimelineMetric timelineMetric = iterator.next();
+      for (String pattern : skipAggregationPatternStrings) {
+        if (timelineMetric.getMetricName().matches(pattern)) {
+          iterator.remove();
         }
-        return String.valueOf(System.currentTimeMillis());
+      }
     }
+    aggregationCacheLock.writeLock().lock();
+    aggregationMetricsCache.put(calculateCacheKey(timelineMetrics), timelineMetrics);
+    aggregationCacheLock.writeLock().unlock();
+  }
 
-    public Map<String, TimelineMetrics> extractMetricsForAggregationPublishing() {
-        return extractMetricsFromCacheWithLock(aggregationMetricsCache, aggregationCacheLock);
+  private String calculateCacheKey(TimelineMetrics timelineMetrics) {
+    List<TimelineMetric> metrics = timelineMetrics.getMetrics();
+    if (metrics.size() > 0) {
+      return metrics.get(0).getAppId() + System.currentTimeMillis();
     }
+    return String.valueOf(System.currentTimeMillis());
+  }
 
-    public void putMetricsForRawPublishing(TimelineMetrics metrics) {
-        rawCacheLock.writeLock().lock();
-        rawMetricsCache.put(calculateCacheKey(metrics), metrics);
-        rawCacheLock.writeLock().unlock();
-    }
+  public Map<String, TimelineMetrics> extractMetricsForAggregationPublishing() {
+    return extractMetricsFromCacheWithLock(aggregationMetricsCache, aggregationCacheLock);
+  }
 
-    public Map<String, TimelineMetrics> extractMetricsForRawPublishing() {
-        return extractMetricsFromCacheWithLock(rawMetricsCache, rawCacheLock);
-    }
+  public void putMetricsForRawPublishing(TimelineMetrics metrics) {
+    rawCacheLock.writeLock().lock();
+    rawMetricsCache.put(calculateCacheKey(metrics), metrics);
+    rawCacheLock.writeLock().unlock();
+  }
 
-    /**
-     * Returns values from cache and clears the cache
-     * @param cache
-     * @param lock
-     * @return
-     */
-    private Map<String, TimelineMetrics> extractMetricsFromCacheWithLock(Cache<String, TimelineMetrics> cache, ReadWriteLock lock) {
-        lock.writeLock().lock();
-        Map<String, TimelineMetrics> metricsMap = new TreeMap<>(cache.asMap());
-        cache.invalidateAll();
-        lock.writeLock().unlock();
-        return metricsMap;
-    }
+  public Map<String, TimelineMetrics> extractMetricsForRawPublishing() {
+    return extractMetricsFromCacheWithLock(rawMetricsCache, rawCacheLock);
+  }
+
+  /**
+   * Returns values from cache and clears the cache
+   * @param cache
+   * @param lock
+   * @return
+   */
+  private Map<String, TimelineMetrics> extractMetricsFromCacheWithLock(Cache<String, TimelineMetrics> cache, ReadWriteLock lock) {
+    lock.writeLock().lock();
+    Map<String, TimelineMetrics> metricsMap = new TreeMap<>(cache.asMap());
+    cache.invalidateAll();
+    lock.writeLock().unlock();
+    return metricsMap;
+  }
 
 }
diff --git a/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/host/aggregator/TimelineMetricsHolderTest.java b/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/host/aggregator/TimelineMetricsHolderTest.java
index 7d8ebf4..548d20f 100644
--- a/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/host/aggregator/TimelineMetricsHolderTest.java
+++ b/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/host/aggregator/TimelineMetricsHolderTest.java
@@ -24,6 +24,7 @@ import org.junit.Test;
 
 import java.lang.reflect.Field;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Map;
 
 
@@ -46,13 +47,13 @@ public class TimelineMetricsHolderTest {
     @Test
     public void testGetInstanceWithParameters() throws Exception {
         clearHolderSingleton();
-        Assert.assertNotNull(TimelineMetricsHolder.getInstance(1,2));
+        Assert.assertNotNull(TimelineMetricsHolder.getInstance(1,2, Collections.EMPTY_LIST));
     }
 
     @Test
     public void testCache() throws Exception {
         clearHolderSingleton();
-        timelineMetricsHolderInstance = TimelineMetricsHolder.getInstance(4,4);
+        timelineMetricsHolderInstance = TimelineMetricsHolder.getInstance(4,4, Collections.EMPTY_LIST);
         timelineMetricsHolderInstance.putMetricsForAggregationPublishing(getTimelineMetricsWithAppID("aggr"));
         timelineMetricsHolderInstance.putMetricsForRawPublishing(getTimelineMetricsWithAppID("raw"));
 
diff --git a/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/sink/timeline/AggregatedMetricsPublisherTest.java b/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/sink/timeline/AggregatedMetricsPublisherTest.java
index 3413052..f09de64 100644
--- a/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/sink/timeline/AggregatedMetricsPublisherTest.java
+++ b/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/sink/timeline/AggregatedMetricsPublisherTest.java
@@ -25,6 +25,7 @@ import org.junit.Test;
 import org.apache.hadoop.conf.Configuration;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -116,7 +117,7 @@ public class AggregatedMetricsPublisherTest {
 
     @Test
     public void testGetMetricsFromCache() throws InterruptedException {
-        TimelineMetricsHolder timelineMetricsHolder = TimelineMetricsHolder.getInstance(4,4);
+        TimelineMetricsHolder timelineMetricsHolder = TimelineMetricsHolder.getInstance(4,4, Collections.EMPTY_LIST);
         timelineMetricsHolder.extractMetricsForAggregationPublishing();
         timelineMetricsHolder.extractMetricsForRawPublishing();
 
diff --git a/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/sink/timeline/RawMetricsPublisherTest.java b/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/sink/timeline/RawMetricsPublisherTest.java
index 60510d2..a30b554 100644
--- a/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/sink/timeline/RawMetricsPublisherTest.java
+++ b/ambari-metrics/ambari-metrics-host-aggregator/src/test/java/org/apache/hadoop/metrics2/sink/timeline/RawMetricsPublisherTest.java
@@ -21,9 +21,11 @@ import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.host.aggregator.TimelineMetricsHolder;
 import org.apache.hadoop.metrics2.host.aggregator.TimelineMetricsHolderTest;
+import org.easymock.cglib.core.CollectionUtils;
 import org.junit.Test;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -113,7 +115,7 @@ public class RawMetricsPublisherTest {
     @Test
     public void testGetMetricsFromCache() throws InterruptedException {
 
-        TimelineMetricsHolder timelineMetricsHolder = TimelineMetricsHolder.getInstance(4,4);
+        TimelineMetricsHolder timelineMetricsHolder = TimelineMetricsHolder.getInstance(4,4, Collections.EMPTY_LIST);
         timelineMetricsHolder.extractMetricsForAggregationPublishing();
         timelineMetricsHolder.extractMetricsForRawPublishing();
 
diff --git a/ambari-metrics/ambari-metrics-timelineservice/conf/unix/sqlline/phoenix_utils.py b/ambari-metrics/ambari-metrics-timelineservice/conf/unix/sqlline/phoenix_utils.py
index 14b53db..492764b 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/conf/unix/sqlline/phoenix_utils.py
+++ b/ambari-metrics/ambari-metrics-timelineservice/conf/unix/sqlline/phoenix_utils.py
@@ -21,7 +21,7 @@
 
 import os
 import fnmatch
-from ambari_commons import subprocess32
+import subprocess
 
 def find(pattern, classPaths):
     paths = classPaths.split(os.pathsep)
@@ -62,7 +62,7 @@ def which(file):
 def findClasspath(file):
     aPath = which(file)
     command = "%s%s" %(aPath, ' classpath')
-    return subprocess32.Popen(command, shell=True, stdout=subprocess32.PIPE).stdout.read()
+    return subprocess.Popen(command, shell=True, stdout=subprocess.PIPE).stdout.read()
 
 def setPath():
     PHOENIX_CLIENT_JAR_PATTERN = "phoenix-*-client.jar"
@@ -167,8 +167,7 @@ def shell_quote(args):
     :return: shell quoted string
     """
     if os.name == 'nt':
-        from ambari_commons import subprocess32
-        return subprocess32.list2cmdline(args)
+        return subprocess.list2cmdline(args)
     else:
         # pipes module isn't available on Windows
         import pipes
diff --git a/ambari-metrics/ambari-metrics-timelineservice/conf/unix/sqlline/sqlline.py b/ambari-metrics/ambari-metrics-timelineservice/conf/unix/sqlline/sqlline.py
index 8027443..852f264 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/conf/unix/sqlline/sqlline.py
+++ b/ambari-metrics/ambari-metrics-timelineservice/conf/unix/sqlline/sqlline.py
@@ -20,7 +20,7 @@
 ############################################################################
 
 import os
-from ambari_commons import subprocess32
+import subprocess
 import sys
 import phoenix_utils
 import atexit
@@ -95,7 +95,7 @@ java_cmd = find_java() + ' -cp "' + phoenix_client_path + \
 
 print 'java command: %s' % str(java_cmd)
 
-childProc = subprocess32.Popen(java_cmd, shell=True)
+childProc = subprocess.Popen(java_cmd, shell=True)
 #Wait for child process exit
 (output, error) = childProc.communicate()
 returncode = childProc.returncode
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/FunctionUtils.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/FunctionUtils.java
new file mode 100644
index 0000000..1ee57ec
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/FunctionUtils.java
@@ -0,0 +1,47 @@
+/**
+ * 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.ambari.metrics.core.timeline;
+
+import com.google.common.collect.Multimap;
+import org.apache.ambari.metrics.core.timeline.aggregators.Function;
+
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.hadoop.metrics2.sink.timeline.TimelineMetricUtils.getJavaRegexFromSqlRegex;
+
+class FunctionUtils {
+
+  static Collection<List<Function>> findMetricFunctions(Multimap<String, List<Function>> metricFunctions,
+                                                         String metricName) {
+    if (metricFunctions.containsKey(metricName)) {
+      return metricFunctions.get(metricName);
+    }
+
+    for (String metricNameEntry : metricFunctions.keySet()) {
+      String metricRegEx = getJavaRegexFromSqlRegex(metricNameEntry);
+      if (metricName.matches(metricRegEx)) {
+        return metricFunctions.get(metricNameEntry);
+      }
+    }
+
+    return null;
+  }
+
+}
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/HBaseTimelineMetricsService.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/HBaseTimelineMetricsService.java
index ae394e3..56a28dc 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/HBaseTimelineMetricsService.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/HBaseTimelineMetricsService.java
@@ -40,15 +40,12 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.ambari.metrics.core.timeline.aggregators.Function;
 import org.apache.ambari.metrics.core.timeline.aggregators.TimelineMetricAggregator;
 import org.apache.ambari.metrics.core.timeline.aggregators.TimelineMetricAggregatorFactory;
 import org.apache.ambari.metrics.core.timeline.availability.MetricCollectorHAController;
 import org.apache.ambari.metrics.core.timeline.discovery.TimelineMetricHostMetadata;
-import org.apache.ambari.metrics.core.timeline.discovery.TimelineMetricMetadataKey;
 import org.apache.ambari.metrics.core.timeline.discovery.TimelineMetricMetadataManager;
 import org.apache.ambari.metrics.core.timeline.function.SeriesAggregateFunction;
 import org.apache.ambari.metrics.core.timeline.function.TimelineMetricsSeriesAggregateFunction;
@@ -87,9 +84,7 @@ public class HBaseTimelineMetricsService extends AbstractService implements Time
   private static volatile boolean isInitialized = false;
   private final ScheduledExecutorService watchdogExecutorService = Executors.newSingleThreadScheduledExecutor();
   private final Map<AGGREGATOR_NAME, ScheduledExecutorService> scheduledExecutors = new HashMap<>();
-  private final ConcurrentHashMap<String, Long> postedAggregatedMap = new ConcurrentHashMap<>();
   private TimelineMetricMetadataManager metricMetadataManager;
-  private Integer defaultTopNHostsLimit;
   private MetricCollectorHAController haController;
   private boolean containerMetricsDisabled = false;
 
@@ -161,7 +156,6 @@ public class HBaseTimelineMetricsService extends AbstractService implements Time
         }
       }
 
-      defaultTopNHostsLimit = Integer.parseInt(metricsConf.get(DEFAULT_TOPN_HOSTS_LIMIT, "20"));
       if (Boolean.parseBoolean(metricsConf.get(USE_GROUPBY_AGGREGATOR_QUERIES, "true"))) {
         LOG.info("Using group by aggregators for aggregating host and cluster metrics.");
       }
@@ -191,13 +185,7 @@ public class HBaseTimelineMetricsService extends AbstractService implements Time
       scheduleAggregatorThread(dailyClusterAggregator);
 
       // Start the minute host aggregator
-      if (configuration.isHostInMemoryAggregationEnabled()) {
-        LOG.info("timeline.metrics.host.inmemory.aggregation is set to True, switching to filtering host minute aggregation on collector");
-        TimelineMetricAggregator minuteHostAggregator =
-          TimelineMetricAggregatorFactory.createFilteringTimelineMetricAggregatorMinute(
-            hBaseAccessor, metricsConf, metricMetadataManager, haController, postedAggregatedMap);
-        scheduleAggregatorThread(minuteHostAggregator);
-      } else {
+      if (!configuration.isHostInMemoryAggregationEnabled()) {
         TimelineMetricAggregator minuteHostAggregator =
           TimelineMetricAggregatorFactory.createTimelineMetricAggregatorMinute(
             hBaseAccessor, metricsConf, metricMetadataManager, haController);
@@ -264,11 +252,16 @@ public class HBaseTimelineMetricsService extends AbstractService implements Time
       parseMetricNamesToAggregationFunctions(metricNames);
 
     TimelineMetrics metrics = new TimelineMetrics();
+    List<String> transientMetricNames = new ArrayList<>();
 
-    List<byte[]> uuids = metricMetadataManager.getUuids(metricFunctions.keySet(), hostnames, applicationId, instanceId);
+    List<byte[]> uuids = metricMetadataManager.getUuidsForGetMetricQuery(metricFunctions.keySet(),
+      hostnames,
+      applicationId,
+      instanceId,
+      transientMetricNames);
 
-    if (uuids.isEmpty()) {
-      LOG.warn("No metric UUIDs generated for query : " + Arrays.asList(metricNames).toString());
+    if (uuids.isEmpty() && transientMetricNames.isEmpty()) {
+      LOG.trace("No metrics satisfy the query: " + Arrays.asList(metricNames).toString());
       return metrics;
     }
 
@@ -281,26 +274,10 @@ public class HBaseTimelineMetricsService extends AbstractService implements Time
       .precision(precision)
       .limit(limit)
       .grouped(groupedByHosts)
-      .uuid(uuids);
+      .uuid(uuids)
+      .transientMetricNames(transientMetricNames);
 
-    if (topNConfig != null) {
-      if (TopNCondition.isTopNHostCondition(metricNames, hostnames) ^ //Only 1 condition should be true.
-        TopNCondition.isTopNMetricCondition(metricNames, hostnames)) {
-        conditionBuilder.topN(topNConfig.getTopN());
-        conditionBuilder.isBottomN(topNConfig.getIsBottomN());
-        Function.ReadFunction readFunction = Function.ReadFunction.getFunction(topNConfig.getTopNFunction());
-        Function function = new Function(readFunction, null);
-        conditionBuilder.topNFunction(function);
-      } else {
-        LOG.info("Invalid Input for TopN query. Ignoring TopN Request.");
-      }
-    } else if (startTime != null && hostnames != null && hostnames.size() > defaultTopNHostsLimit) {
-      // if (timeseries query AND hostnames passed AND size(hostnames) > limit)
-      LOG.info("Requesting data for more than " + defaultTopNHostsLimit + " Hosts. " +
-        "Defaulting to Top " + defaultTopNHostsLimit);
-      conditionBuilder.topN(defaultTopNHostsLimit);
-      conditionBuilder.isBottomN(false);
-    }
+    applyTopNCondition(conditionBuilder, topNConfig, metricNames, hostnames);
 
     Condition condition = conditionBuilder.build();
 
@@ -319,6 +296,24 @@ public class HBaseTimelineMetricsService extends AbstractService implements Time
     return seriesAggregateMetrics(seriesAggrFunctionInstance, metrics);
   }
 
+  private void applyTopNCondition(ConditionBuilder conditionBuilder,
+                                  TopNConfig topNConfig,
+                                  List<String> metricNames,
+                                  List<String> hostnames) {
+    if (topNConfig != null) {
+      if (TopNCondition.isTopNHostCondition(metricNames, hostnames) ^ //Only 1 condition should be true.
+        TopNCondition.isTopNMetricCondition(metricNames, hostnames)) {
+        conditionBuilder.topN(topNConfig.getTopN());
+        conditionBuilder.isBottomN(topNConfig.getIsBottomN());
+        Function.ReadFunction readFunction = Function.ReadFunction.getFunction(topNConfig.getTopNFunction());
+        Function function = new Function(readFunction, null);
+        conditionBuilder.topNFunction(function);
+      } else {
+        LOG.info("Invalid Input for TopN query. Ignoring TopN Request.");
+      }
+    }
+  }
+
   private TimelineMetrics postProcessMetrics(TimelineMetrics metrics) {
     List<TimelineMetric> metricsList = metrics.getMetrics();
 
@@ -430,51 +425,12 @@ public class HBaseTimelineMetricsService extends AbstractService implements Time
   @Override
   public Map<String, List<TimelineMetricMetadata>> getTimelineMetricMetadata(String appId, String metricPattern,
                                                                              boolean includeBlacklistedMetrics) throws SQLException, IOException {
-    Map<TimelineMetricMetadataKey, TimelineMetricMetadata> metadata =
-      metricMetadataManager.getMetadataCache();
-
-    boolean filterByAppId = StringUtils.isNotEmpty(appId);
-    boolean filterByMetricName = StringUtils.isNotEmpty(metricPattern);
-    Pattern metricFilterPattern = null;
-    if (filterByMetricName) {
-      metricFilterPattern = Pattern.compile(metricPattern);
-    }
-
-    // Group Metadata by AppId
-    Map<String, List<TimelineMetricMetadata>> metadataByAppId = new HashMap<>();
-    for (TimelineMetricMetadata metricMetadata : metadata.values()) {
-
-      if (!includeBlacklistedMetrics && !metricMetadata.isWhitelisted()) {
-        continue;
-      }
-
-      String currentAppId = metricMetadata.getAppId();
-      if (filterByAppId && !currentAppId.equals(appId)) {
-        continue;
-      }
-
-      if (filterByMetricName) {
-        Matcher m = metricFilterPattern.matcher(metricMetadata.getMetricName());
-        if (!m.find()) {
-          continue;
-        }
-      }
-
-      List<TimelineMetricMetadata> metadataList = metadataByAppId.get(currentAppId);
-      if (metadataList == null) {
-        metadataList = new ArrayList<>();
-        metadataByAppId.put(currentAppId, metadataList);
-      }
-
-      metadataList.add(metricMetadata);
-    }
-
-    return metadataByAppId;
+    return metricMetadataManager.getTimelineMetricMetadataByAppId(appId, metricPattern, includeBlacklistedMetrics);
   }
 
   @Override
   public byte[] getUuid(String metricName, String appId, String instanceId, String hostname) throws SQLException, IOException {
-    return metricMetadataManager.getUuid(metricName, appId, instanceId, hostname);
+    return metricMetadataManager.getUuid(metricName, appId, instanceId, hostname, false);
   }
 
   @Override
@@ -497,13 +453,11 @@ public class HBaseTimelineMetricsService extends AbstractService implements Time
       break;
     }
     long timestamp = aggregationResult.getTimeInMilis();
-    postedAggregatedMap.put(hostname, timestamp);
     if (LOG.isDebugEnabled()) {
       LOG.debug(String.format("Adding host %s to aggregated by in-memory aggregator. Timestamp : %s", hostname, timestamp));
     }
     hBaseAccessor.saveHostAggregateRecords(aggregateMap, PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME);
 
-
     return new TimelinePutResponse();
   }
 
@@ -578,6 +532,11 @@ public class HBaseTimelineMetricsService extends AbstractService implements Time
     return instances;
   }
 
+  @Override
+  public TimelineMetricServiceSummary getTimelineMetricServiceSummary() {
+    return new TimelineMetricServiceSummary(metricMetadataManager, haController);
+  }
+
   private void scheduleAggregatorThread(final TimelineMetricAggregator aggregator) {
     if (!aggregator.isDisabled()) {
       ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor(
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/PhoenixHBaseAccessor.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/PhoenixHBaseAccessor.java
index 967cede..040df1b 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/PhoenixHBaseAccessor.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/PhoenixHBaseAccessor.java
@@ -18,7 +18,10 @@
 package org.apache.ambari.metrics.core.timeline;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.ambari.metrics.core.timeline.FunctionUtils.findMetricFunctions;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.AGGREGATORS_SKIP_BLOCK_CACHE;
+import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL;
+import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.CLUSTER_DAILY_TABLE_TTL;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.CLUSTER_HOUR_TABLE_TTL;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.CLUSTER_MINUTE_TABLE_TTL;
@@ -30,10 +33,12 @@ import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguratio
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.HBASE_BLOCKING_STORE_FILES;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.HBASE_COMPRESSION_SCHEME;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.HBASE_ENCODING_SCHEME;
+import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_MINUTE_SLEEP_INTERVAL;
+import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TRANSIENT_METRIC_PATTERNS;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.HOST_DAILY_TABLE_TTL;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.HOST_HOUR_TABLE_TTL;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.HOST_MINUTE_TABLE_TTL;
-import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.OUT_OFF_BAND_DATA_TIME_ALLOWANCE;
+import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.METRICS_TRANSIENT_TABLE_TTL;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.PRECISION_TABLE_TTL;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_AGGREGATE_TABLES_DURABILITY;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_AGGREGATE_TABLE_HBASE_BLOCKING_STORE_FILES;
@@ -47,10 +52,10 @@ import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguratio
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_PRECISION_TABLE_DURABILITY;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_PRECISION_TABLE_HBASE_BLOCKING_STORE_FILES;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRIC_AGGREGATOR_SINK_CLASS;
-import static org.apache.ambari.metrics.core.timeline.aggregators.AggregatorUtils.getJavaRegexFromSqlRegex;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.ALTER_METRICS_METADATA_TABLE;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.CONTAINER_METRICS_TABLE_NAME;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.CREATE_CONTAINER_METRICS_TABLE_SQL;
+import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.CREATE_TRANSIENT_METRICS_TABLE_SQL;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.CREATE_HOSTED_APPS_METADATA_TABLE_SQL;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.CREATE_INSTANCE_HOST_TABLE_SQL;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_TABLE_SQL;
@@ -71,6 +76,7 @@ import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.M
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_MINUTE_TABLE_NAME;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
+import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.METRIC_TRANSIENT_TABLE_NAME;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.PHOENIX_TABLES;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.PHOENIX_TABLES_REGEX_PATTERN;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.UPSERT_AGGREGATE_RECORD_SQL;
@@ -81,6 +87,7 @@ import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.U
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.UPSERT_INSTANCE_HOST_METADATA_SQL;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.UPSERT_METADATA_SQL;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.UPSERT_METRICS_SQL;
+import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.UPSERT_TRANSIENT_METRICS_SQL;
 import static org.apache.ambari.metrics.core.timeline.source.InternalSourceProvider.SOURCE_NAME.RAW_METRICS;
 
 import java.io.IOException;
@@ -124,6 +131,7 @@ import org.apache.ambari.metrics.core.timeline.sink.ExternalMetricsSink;
 import org.apache.ambari.metrics.core.timeline.sink.ExternalSinkProvider;
 import org.apache.ambari.metrics.core.timeline.source.InternalMetricsSource;
 import org.apache.ambari.metrics.core.timeline.source.InternalSourceProvider;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -163,17 +171,19 @@ public class PhoenixHBaseAccessor {
   private static final Log LOG = LogFactory.getLog(PhoenixHBaseAccessor.class);
 
   static final int PHOENIX_MAX_MUTATION_STATE_SIZE = 50000;
-  // Default stale data allowance set to 3 minutes, 2 minutes more than time
-  // it was collected. Also 2 minutes is the default aggregation interval at
-  // cluster and host levels.
-  static final long DEFAULT_OUT_OF_BAND_TIME_ALLOWANCE = 300000;
+
   /**
-   * 22 metrics for 2hours in SECONDS (10 second data)
+   * 22 metrics for 2hours in SECONDS, 2 hosts (1 minute data records)
+   * 22 * (2 * 60) * 2 = 5280
+   * 22 cluster aggregate metrics for 2 hours (30 second data records)
+   * 22 * (2 * 60 * 2) = 5280
    * => Reasonable upper bound on the limit such that our Precision calculation for a given time range makes sense.
    */
-  private static final int METRICS_PER_MINUTE = 22;
-  private static final int POINTS_PER_MINUTE = 6;
-  public static int RESULTSET_LIMIT = (int)TimeUnit.HOURS.toMinutes(2) * METRICS_PER_MINUTE * POINTS_PER_MINUTE ;
+  public static int RESULTSET_LIMIT = 5760;
+
+  public static int hostMinuteAggregatorDataInterval = 300;
+  public static int clusterMinuteAggregatorDataInterval = 300;
+  public static int clusterSecondAggregatorDataInterval = 30;
 
   static TimelineMetricReadHelper TIMELINE_METRIC_READ_HELPER = new TimelineMetricReadHelper();
   static ObjectMapper mapper = new ObjectMapper();
@@ -183,7 +193,6 @@ public class PhoenixHBaseAccessor {
   private final Configuration metricsConf;
   private final RetryCounterFactory retryCounterFactory;
   private final PhoenixConnectionProvider dataSource;
-  private final long outOfBandTimeAllowance;
   private final int cacheSize;
   private final boolean cacheEnabled;
   private final BlockingQueue<TimelineMetrics> insertCache;
@@ -218,8 +227,7 @@ public class PhoenixHBaseAccessor {
 
   // Test friendly construction since mock instrumentation is difficult to get
   // working with hadoop mini cluster
-  PhoenixHBaseAccessor(TimelineMetricConfiguration configuration,
-                       PhoenixConnectionProvider dataSource) {
+  PhoenixHBaseAccessor(TimelineMetricConfiguration configuration, PhoenixConnectionProvider dataSource) {
     this.configuration = TimelineMetricConfiguration.getInstance();
     try {
       this.hbaseConf = configuration.getHbaseConf();
@@ -233,6 +241,10 @@ public class PhoenixHBaseAccessor {
     this.dataSource = dataSource;
 
     RESULTSET_LIMIT = metricsConf.getInt(GLOBAL_RESULT_LIMIT, RESULTSET_LIMIT);
+    clusterSecondAggregatorDataInterval = metricsConf.getInt(CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL, 30);
+    hostMinuteAggregatorDataInterval = metricsConf.getInt(HOST_AGGREGATOR_MINUTE_SLEEP_INTERVAL, 300);
+    clusterMinuteAggregatorDataInterval = metricsConf.getInt(CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL, 300);
+
     try {
       Class.forName("org.apache.phoenix.jdbc.PhoenixDriver");
     } catch (ClassNotFoundException e) {
@@ -242,8 +254,6 @@ public class PhoenixHBaseAccessor {
 
     this.retryCounterFactory = new RetryCounterFactory(metricsConf.getInt(GLOBAL_MAX_RETRIES, 10),
       (int) SECONDS.toMillis(metricsConf.getInt(GLOBAL_RETRY_INTERVAL, 3)));
-    this.outOfBandTimeAllowance = metricsConf.getLong(OUT_OFF_BAND_DATA_TIME_ALLOWANCE,
-      DEFAULT_OUT_OF_BAND_TIME_ALLOWANCE);
     this.cacheEnabled = Boolean.valueOf(metricsConf.get(TIMELINE_METRICS_CACHE_ENABLED, "true"));
     this.cacheSize = Integer.valueOf(metricsConf.get(TIMELINE_METRICS_CACHE_SIZE, "150"));
     this.cacheCommitInterval = Integer.valueOf(metricsConf.get(TIMELINE_METRICS_CACHE_COMMIT_INTERVAL, "3"));
@@ -261,6 +271,7 @@ public class PhoenixHBaseAccessor {
     tableTTL.put(METRICS_CLUSTER_AGGREGATE_MINUTE_TABLE_NAME, metricsConf.getInt(CLUSTER_MINUTE_TABLE_TTL, 30 * 86400)); //30 days
     tableTTL.put(METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME, metricsConf.getInt(CLUSTER_HOUR_TABLE_TTL, 365 * 86400)); //1 year
     tableTTL.put(METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME, metricsConf.getInt(CLUSTER_DAILY_TABLE_TTL, 730 * 86400)); //2 years
+    tableTTL.put(METRIC_TRANSIENT_TABLE_NAME, metricsConf.getInt(METRICS_TRANSIENT_TABLE_TTL, 7 * 86400)); //7 days
 
     if (cacheEnabled) {
       LOG.debug("Initialising and starting metrics cache committer thread...");
@@ -320,6 +331,7 @@ public class PhoenixHBaseAccessor {
     LOG.debug("Committing metrics to store");
     Connection conn = null;
     PreparedStatement metricRecordStmt = null;
+    List<TimelineMetric> transientMetrics = new ArrayList<>();
 
     try {
       conn = getConnection();
@@ -328,6 +340,10 @@ public class PhoenixHBaseAccessor {
       for (TimelineMetrics timelineMetrics : timelineMetricsCollection) {
         for (TimelineMetric metric : timelineMetrics.getMetrics()) {
 
+          if (metadataManagerInstance.isTransientMetric(metric.getMetricName(), metric.getAppId())) {
+            transientMetrics.add(metric);
+            continue;
+          }
           metricRecordStmt.clearParameters();
 
           if (LOG.isTraceEnabled()) {
@@ -339,7 +355,7 @@ public class PhoenixHBaseAccessor {
                   metric.getMetricValues());
 
           if (aggregates[3] != 0.0) {
-            byte[] uuid = metadataManagerInstance.getUuid(metric);
+            byte[] uuid = metadataManagerInstance.getUuid(metric, true);
             if (uuid == null) {
               LOG.error("Error computing UUID for metric. Cannot write metrics : " + metric.toString());
               continue;
@@ -367,6 +383,9 @@ public class PhoenixHBaseAccessor {
 
         }
       }
+      if (CollectionUtils.isNotEmpty(transientMetrics)) {
+        commitTransientMetrics(conn, transientMetrics);
+      }
 
       // commit() blocked if HBase unavailable
       conn.commit();
@@ -391,15 +410,53 @@ public class PhoenixHBaseAccessor {
     }
   }
 
-  private static TimelineMetric getLastTimelineMetricFromResultSet(ResultSet rs)
-      throws SQLException, IOException {
+  private void commitTransientMetrics(Connection conn, Collection<TimelineMetric> transientMetrics) throws SQLException, IOException {
+    LOG.debug("Committing transient metrics to store");
+    PreparedStatement metricTransientRecordStmt = null;
+
+    metricTransientRecordStmt = conn.prepareStatement(String.format(
+      UPSERT_TRANSIENT_METRICS_SQL, METRIC_TRANSIENT_TABLE_NAME));
+    for (TimelineMetric metric : transientMetrics) {
+
+      metricTransientRecordStmt.clearParameters();
+
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("host: " + metric.getHostName() + ", " +
+          "metricName = " + metric.getMetricName() + ", " +
+          "values: " + metric.getMetricValues());
+      }
+      double[] aggregates = AggregatorUtils.calculateAggregates(
+        metric.getMetricValues());
+
+      metricTransientRecordStmt.setString(1, metric.getMetricName());
+      metricTransientRecordStmt.setString(2, metric.getHostName());
+      metricTransientRecordStmt.setString(3, metric.getAppId());
+      metricTransientRecordStmt.setString(4, metric.getInstanceId());
+      metricTransientRecordStmt.setLong(5, metric.getStartTime());
+      metricTransientRecordStmt.setString(6, metric.getUnits());
+      metricTransientRecordStmt.setDouble(7, aggregates[0]);
+      metricTransientRecordStmt.setDouble(8, aggregates[1]);
+      metricTransientRecordStmt.setDouble(9, aggregates[2]);
+      metricTransientRecordStmt.setLong(10, (long) aggregates[3]);
+      String json = TimelineUtils.dumpTimelineRecordtoJSON(metric.getMetricValues());
+      metricTransientRecordStmt.setString(11, json);
+
+      try {
+        metricTransientRecordStmt.executeUpdate();
+      } catch (SQLException sql) {
+        LOG.error("Failed on inserting transient metric records to store.", sql);
+      }
+    }
+  }
+
+
+  private static TimelineMetric getLastTimelineMetricFromResultSet(ResultSet rs) throws SQLException, IOException {
     TimelineMetric metric = TIMELINE_METRIC_READ_HELPER.getTimelineMetricCommonsFromResultSet(rs);
     metric.setMetricValues(readLastMetricValueFromJSON(rs.getString("METRICS")));
     return metric;
   }
 
-  private static TreeMap<Long, Double> readLastMetricValueFromJSON(String json)
-      throws IOException {
+  private static TreeMap<Long, Double> readLastMetricValueFromJSON(String json) throws IOException {
     TreeMap<Long, Double> values = readMetricFromJSON(json);
     Long lastTimeStamp = values.lastKey();
 
@@ -413,8 +470,7 @@ public class PhoenixHBaseAccessor {
     return mapper.readValue(json, metricValuesTypeRef);
   }
 
-  private Connection getConnectionRetryingOnException()
-    throws SQLException, InterruptedException {
+  private Connection getConnectionRetryingOnException() throws SQLException, InterruptedException {
     RetryCounter retryCounter = retryCounterFactory.create();
     while (true) {
       try{
@@ -524,6 +580,13 @@ public class PhoenixHBaseAccessor {
         tableTTL.get(METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME),
         compression));
 
+      // Metrics Transient Table
+      String transientMetricPatterns = metricsConf.get(TRANSIENT_METRIC_PATTERNS, StringUtils.EMPTY);
+      if (StringUtils.isNotEmpty(transientMetricPatterns)) {
+        String transientMetricsTableSql = String.format(CREATE_TRANSIENT_METRICS_TABLE_SQL,
+          encoding, tableTTL.get(METRIC_TRANSIENT_TABLE_NAME), compression);
+        int row = stmt.executeUpdate(transientMetricsTableSql);
+      }
 
       conn.commit();
 
@@ -647,8 +710,11 @@ public class PhoenixHBaseAccessor {
   private boolean setDurabilityForTable(String tableName, TableDescriptorBuilder tableDescriptor) {
 
     boolean modifyTable = false;
-    // Set WAL preferences
-    if (METRICS_RECORD_TABLE_NAME.equals(tableName)) {
+
+    if (METRIC_TRANSIENT_TABLE_NAME.equalsIgnoreCase(tableName)) {
+      tableDescriptor.setDurability(Durability.SKIP_WAL);
+      modifyTable = true;
+    } else if (METRICS_RECORD_TABLE_NAME.equals(tableName)) {
       if (!timelineMetricsPrecisionTableDurability.isEmpty()) {
         LOG.info("Setting WAL option " + timelineMetricsPrecisionTableDurability + " for table : " + tableName);
         boolean validDurability = true;
@@ -702,7 +768,7 @@ public class PhoenixHBaseAccessor {
       DATE_TIERED_COMPACTION_POLICY);
     int blockingStoreFiles = hbaseConf.getInt(TIMELINE_METRICS_AGGREGATE_TABLE_HBASE_BLOCKING_STORE_FILES, 60);
 
-    if (tableName.equals(METRICS_RECORD_TABLE_NAME)) {
+    if (tableName.equals(METRICS_RECORD_TABLE_NAME) || tableName.equalsIgnoreCase(METRIC_TRANSIENT_TABLE_NAME)) {
       compactionPolicyKey = metricsConf.get(TIMELINE_METRICS_HBASE_PRECISION_TABLE_COMPACTION_POLICY_KEY,
         HSTORE_COMPACTION_CLASS_KEY);
       compactionPolicyClass = metricsConf.get(TIMELINE_METRICS_HBASE_PRECISION_TABLE_COMPACTION_POLICY_CLASS,
@@ -724,8 +790,7 @@ public class PhoenixHBaseAccessor {
     return modifyTable;
   }
 
-  private boolean setHbaseBlockingStoreFiles(TableDescriptorBuilder tableDescriptor,
-                                             String tableName, int value) {
+  private boolean setHbaseBlockingStoreFiles(TableDescriptorBuilder tableDescriptor, String tableName, int value) {
     int blockingStoreFiles = hbaseConf.getInt(HBASE_BLOCKING_STORE_FILES, value);
     if (blockingStoreFiles != value) {
       blockingStoreFiles = value;
@@ -736,6 +801,7 @@ public class PhoenixHBaseAccessor {
     }
     return false;
   }
+
   protected String getSplitPointsStr(String splitPoints) {
     if (StringUtils.isEmpty(splitPoints.trim())) {
       return "";
@@ -760,8 +826,7 @@ public class PhoenixHBaseAccessor {
   /**
    * Insert precision YARN container data.
    */
-  public void insertContainerMetrics(List<ContainerMetric> metrics)
-      throws SQLException, IOException {
+  public void insertContainerMetrics(List<ContainerMetric> metrics) throws SQLException, IOException {
     Connection conn = getConnection();
     PreparedStatement metricRecordStmt = null;
 
@@ -844,7 +909,7 @@ public class PhoenixHBaseAccessor {
       // Write to metadata cache on successful write to store
       if (metadataManager != null) {
         metadataManager.putIfModifiedTimelineMetricMetadata(
-                metadataManager.getTimelineMetricMetadata(tm, acceptMetric));
+                metadataManager.createTimelineMetricMetadata(tm, acceptMetric));
 
         metadataManager.putIfModifiedHostedAppsMetadata(
                 tm.getHostName(), tm.getAppId());
@@ -901,10 +966,23 @@ public class PhoenixHBaseAccessor {
         getLatestMetricRecords(condition, conn, metrics);
       } else {
         if (condition.getEndTime() >= condition.getStartTime()) {
-          stmt = PhoenixTransactSQL.prepareGetMetricsSqlStmt(conn, condition);
-          rs = stmt.executeQuery();
-          while (rs.next()) {
-            appendMetricFromResultSet(metrics, condition, metricFunctions, rs);
+
+          if (CollectionUtils.isNotEmpty(condition.getUuids())) {
+            stmt = PhoenixTransactSQL.prepareGetMetricsSqlStmt(conn, condition);
+            rs = stmt.executeQuery();
+            while (rs.next()) {
+              appendMetricFromResultSet(metrics, condition, metricFunctions, rs);
+            }
+          }
+
+          if (CollectionUtils.isNotEmpty(condition.getTransientMetricNames())) {
+            stmt = PhoenixTransactSQL.prepareTransientMetricsSqlStmt(conn, condition);
+            if (stmt != null) {
+              rs = stmt.executeQuery();
+              while (rs.next()) {
+                TransientMetricReadHelper.appendMetricFromResultSet(metrics, condition, metricFunctions, rs);
+              }
+            }
           }
         } else {
           LOG.warn("Skipping metrics query because endTime < startTime");
@@ -1018,7 +1096,8 @@ public class PhoenixHBaseAccessor {
     }
   }
 
-  private void getTimelineMetricsFromResultSet(TimelineMetrics metrics, Function f, Condition condition, ResultSet rs) throws SQLException, IOException {
+  private void getTimelineMetricsFromResultSet(TimelineMetrics metrics, Function f, Condition condition, ResultSet rs)
+    throws SQLException, IOException {
     if (condition.getPrecision().equals(Precision.SECONDS)) {
       TimelineMetric metric = TIMELINE_METRIC_READ_HELPER.getTimelineMetricFromResultSet(rs);
       if (f != null && f.getSuffix() != null) { //Case : Requesting "._rate" for precision data
@@ -1076,7 +1155,7 @@ public class PhoenixHBaseAccessor {
    * @throws SQLException
    */
   public TimelineMetrics getAggregateMetricRecords(final Condition condition,
-      Multimap<String, List<Function>> metricFunctions) throws SQLException {
+      Multimap<String, List<Function>> metricFunctions) throws SQLException, IOException {
 
     validateConditionIsNotEmpty(condition);
 
@@ -1090,11 +1169,23 @@ public class PhoenixHBaseAccessor {
       if(condition.isPointInTime()) {
         getLatestAggregateMetricRecords(condition, conn, metrics, metricFunctions);
       } else {
-        stmt = PhoenixTransactSQL.prepareGetAggregateSqlStmt(conn, condition);
 
-        rs = stmt.executeQuery();
-        while (rs.next()) {
-          appendAggregateMetricFromResultSet(metrics, condition, metricFunctions, rs);
+        if (CollectionUtils.isNotEmpty(condition.getUuids())) {
+          stmt = PhoenixTransactSQL.prepareGetAggregateSqlStmt(conn, condition);
+          rs = stmt.executeQuery();
+          while (rs.next()) {
+            appendAggregateMetricFromResultSet(metrics, condition, metricFunctions, rs);
+          }
+        }
+
+        if (CollectionUtils.isNotEmpty(condition.getTransientMetricNames())) {
+          stmt = PhoenixTransactSQL.prepareTransientMetricsSqlStmt(conn, condition);
+          if (stmt != null) {
+            rs = stmt.executeQuery();
+            while (rs.next()) {
+              TransientMetricReadHelper.appendMetricFromResultSet(metrics, condition, metricFunctions, rs);
+            }
+          }
         }
       }
     } finally {
@@ -1256,22 +1347,6 @@ public class PhoenixHBaseAccessor {
     }
   }
 
-  private Collection<List<Function>> findMetricFunctions(Multimap<String, List<Function>> metricFunctions,
-                                                         String metricName) {
-    if (metricFunctions.containsKey(metricName)) {
-      return metricFunctions.get(metricName);
-    }
-
-    for (String metricNameEntry : metricFunctions.keySet()) {
-      String metricRegEx = getJavaRegexFromSqlRegex(metricNameEntry);
-      if (metricName.matches(metricRegEx)) {
-        return metricFunctions.get(metricNameEntry);
-      }
-    }
-
-    return null;
-  }
-
   public void saveHostAggregateRecords(Map<TimelineMetric, MetricHostAggregate> hostAggregateMap,
                                        String phoenixTableName) throws SQLException {
 
@@ -1296,9 +1371,9 @@ public class PhoenixHBaseAccessor {
         TimelineMetric metric = metricAggregate.getKey();
         MetricHostAggregate hostAggregate = metricAggregate.getValue();
 
-        byte[] uuid = metadataManagerInstance.getUuid(metric);
+        byte[] uuid = metadataManagerInstance.getUuid(metric, false);
         if (uuid == null) {
-          LOG.error("Error computing UUID for metric. Cannot write metric : " + metric.toString());
+          LOG.error("Error computing UUID for metric. Cannot write aggregate metric : " + metric.toString());
           continue;
         }
         rowCount++;
@@ -1392,7 +1467,7 @@ public class PhoenixHBaseAccessor {
         }
 
         rowCount++;
-        byte[] uuid =  metadataManagerInstance.getUuid(clusterMetric);
+        byte[] uuid =  metadataManagerInstance.getUuid(clusterMetric, false);
         if (uuid == null) {
           LOG.error("Error computing UUID for metric. Cannot write metrics : " + clusterMetric.toString());
           continue;
@@ -1480,7 +1555,7 @@ public class PhoenixHBaseAccessor {
             "aggregate = " + aggregate);
         }
 
-        byte[] uuid = metadataManagerInstance.getUuid(clusterMetric);
+        byte[] uuid = metadataManagerInstance.getUuid(clusterMetric, false);
         if (uuid == null) {
           LOG.error("Error computing UUID for metric. Cannot write metric : " + clusterMetric.toString());
           continue;
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricConfiguration.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricConfiguration.java
index e680905..6ec2c6b 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricConfiguration.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricConfiguration.java
@@ -114,6 +114,9 @@ public class TimelineMetricConfiguration {
   public static final String CLUSTER_DAILY_TABLE_TTL =
     "timeline.metrics.cluster.aggregator.daily.ttl";
 
+  public static final String METRICS_TRANSIENT_TABLE_TTL =
+    "timeline.metrics.transient.table.ttl";
+
   public static final String CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL =
     "timeline.metrics.cluster.aggregator.second.timeslice.interval";
 
@@ -246,9 +249,6 @@ public class TimelineMetricConfiguration {
   public static final String TIMELINE_SERVICE_HTTP_POLICY =
     "timeline.metrics.service.http.policy";
 
-  public static final String DISABLE_METRIC_METADATA_MGMT =
-    "timeline.metrics.service.metadata.management.disabled";
-
   public static final String METRICS_METADATA_SYNC_INIT_DELAY =
     "timeline.metrics.service.metadata.sync.init.delay";
 
@@ -344,6 +344,8 @@ public class TimelineMetricConfiguration {
   public static final String DEFAULT_EXTERNAL_SINK_DIR =
     "timeline.metrics.external.sink.dir";
 
+  public static final String TRANSIENT_METRIC_PATTERNS = "timeline.metrics.transient.metric.patterns";
+
   public static final String KAFKA_SERVERS = "timeline.metrics.external.sink.kafka.bootstrap.servers";
   public static final String KAFKA_ACKS = "timeline.metrics.external.sink.kafka.acks";
   public static final String KAFKA_RETRIES = "timeline.metrics.external.sink.kafka.bootstrap.retries";
@@ -712,4 +714,12 @@ public class TimelineMetricConfiguration {
     }
     return false;
   }
+
+  public String getTransientMetricPatterns() {
+    if (metricsConf != null) {
+      return metricsConf.get(TRANSIENT_METRIC_PATTERNS, StringUtils.EMPTY);
+    }
+    return StringUtils.EMPTY;
+  }
+
 }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricServiceSummary.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricServiceSummary.java
new file mode 100644
index 0000000..5bc2fd4
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricServiceSummary.java
@@ -0,0 +1,74 @@
+/**
+ * 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.ambari.metrics.core.timeline;
+
+import org.apache.ambari.metrics.core.timeline.availability.MetricCollectorHAController;
+import org.apache.ambari.metrics.core.timeline.discovery.TimelineMetricMetadataManager;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+@XmlRootElement(name = "summary")
+@XmlAccessorType(XmlAccessType.NONE)
+/**
+ * Class to provide basic summary of the state for AMS.
+ * Can be used to add AMS health metrics in the future without trying to reach HBase.
+ */
+public class TimelineMetricServiceSummary {
+
+  private Date timestamp;
+  private Map<String, String> metadataSummary = new HashMap<>();
+  private Map<String, String> aggregationSummary = new HashMap<>();
+  static final Log LOG = LogFactory.getLog(TimelineMetricServiceSummary.class);
+
+  TimelineMetricServiceSummary(TimelineMetricMetadataManager metricMetadataManager,
+                               MetricCollectorHAController haController) {
+    timestamp = new Date(System.currentTimeMillis());
+    try {
+      metadataSummary = metricMetadataManager.getMetadataSummary();
+    } catch (Exception e) {
+      LOG.error("Error fetching metadata summary : " + e);
+    }
+    if (haController != null) {
+      aggregationSummary = haController.getAggregationSummary();
+    }
+  }
+
+  @XmlElement(name = "timestamp")
+  public Date getTimestamp() {
+    return timestamp;
+  }
+
+  @XmlElement(name = "metadata")
+  public Map<String, String> getMetadataSummary() {
+    return metadataSummary;
+  }
+
+  @XmlElement(name = "aggregation")
+  public Map<String, String> getAggregationSummary() {
+    return aggregationSummary;
+  }
+}
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricStore.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricStore.java
index 901d51e..a723a2c 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricStore.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricStore.java
@@ -107,4 +107,9 @@ public interface TimelineMetricStore {
      * @return [ hostname ]
      */
   List<String> getLiveInstances();
+
+  /**
+   * Returns a summary of the hosts, metrics and aggregator checkpoints for the service.
+   */
+  TimelineMetricServiceSummary getTimelineMetricServiceSummary();
 }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricsIgniteCache.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricsIgniteCache.java
index 0a8dcc5..e085fd8 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricsIgniteCache.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TimelineMetricsIgniteCache.java
@@ -26,10 +26,10 @@ import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguratio
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_COLLECTOR_IGNITE_NODES;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRIC_AGGREGATION_SQL_FILTERS;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_SERVICE_HTTP_POLICY;
-import static org.apache.ambari.metrics.core.timeline.aggregators.AggregatorUtils.getJavaRegexFromSqlRegex;
 import static org.apache.ambari.metrics.core.timeline.aggregators.AggregatorUtils.getRoundedCheckPointTimeMillis;
 import static org.apache.ambari.metrics.core.timeline.aggregators.AggregatorUtils.getTimeSlices;
 import static org.apache.ambari.metrics.core.timeline.aggregators.AggregatorUtils.sliceFromTimelineMetric;
+import static org.apache.hadoop.metrics2.sink.timeline.TimelineMetricUtils.getJavaMetricPatterns;
 
 import java.net.MalformedURLException;
 import java.net.URISyntaxException;
@@ -112,13 +112,18 @@ public class TimelineMetricsIgniteCache implements TimelineMetricDistributedCach
     cacheSliceIntervalMillis = SECONDS.toMillis(metricConf.getInt(CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL, 30));
     Long aggregationInterval = metricConf.getLong(CLUSTER_AGGREGATOR_SECOND_SLEEP_INTERVAL, 120L);
 
+    // Skip aggregation for metrics for which aggregating across hosts does not make sense.
     String filteredMetricPatterns = metricConf.get(TIMELINE_METRIC_AGGREGATION_SQL_FILTERS);
-    if (!StringUtils.isEmpty(filteredMetricPatterns)) {
-      LOG.info("Skipping aggregation for metric patterns : " + filteredMetricPatterns);
-      for (String patternString : filteredMetricPatterns.split(",")) {
-        String javaPatternString = getJavaRegexFromSqlRegex(patternString);
-        skipAggrPatternStrings.add(javaPatternString);
-      }
+    if (StringUtils.isNotEmpty(filteredMetricPatterns)) {
+      LOG.info("Skipping in memory cluster aggregation for metric patterns : " + filteredMetricPatterns);
+      skipAggrPatternStrings.addAll(getJavaMetricPatterns(filteredMetricPatterns));
+    }
+
+    // Skip aggregation for those metrics that are meant to be of high volume and get differential treatment.
+    String transientMetricPatterns = timelineMetricConfiguration.getTransientMetricPatterns();
+    if (StringUtils.isNotEmpty(transientMetricPatterns)) {
+      LOG.info("Skipping in memory cluster aggregation for transient metric patterns : " + transientMetricPatterns);
+      skipAggrPatternStrings.addAll(getJavaMetricPatterns(transientMetricPatterns));
     }
 
     if (metricConf.get(TIMELINE_METRICS_COLLECTOR_IGNITE_NODES) != null) {
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TransientMetricReadHelper.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TransientMetricReadHelper.java
new file mode 100644
index 0000000..11be127
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/TransientMetricReadHelper.java
@@ -0,0 +1,139 @@
+/**
+ * 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.ambari.metrics.core.timeline;
+
+import com.google.common.collect.Multimap;
+import org.apache.ambari.metrics.core.timeline.aggregators.Function;
+import org.apache.ambari.metrics.core.timeline.query.Condition;
+import org.apache.hadoop.metrics2.sink.timeline.SingleValuedTimelineMetric;
+import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
+import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
+
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.List;
+import java.util.TreeMap;
+
+import static org.apache.ambari.metrics.core.timeline.FunctionUtils.findMetricFunctions;
+
+class TransientMetricReadHelper {
+
+  static void appendMetricFromResultSet(TimelineMetrics metrics, Condition condition,
+                                        Multimap<String, List<Function>> metricFunctions,
+                                        ResultSet rs) throws SQLException, IOException {
+    String metricName = rs.getString("METRIC_NAME");
+    Collection<List<Function>> functionList = findMetricFunctions(metricFunctions, metricName);
+
+    for (List<Function> functions : functionList) {
+      // Apply aggregation function if present
+      if ((functions != null && !functions.isEmpty())) {
+        if (functions.size() > 1) {
+          throw new IllegalArgumentException("Multiple aggregate functions not supported.");
+        }
+        for (Function f : functions) {
+          if (f.getReadFunction() == Function.ReadFunction.VALUE) {
+            getTimelineMetricsFromResultSet(metrics, f, condition, rs);
+          } else {
+            SingleValuedTimelineMetric metric = getAggregatedTimelineMetricFromResultSet(rs, f);
+            if (condition.isGrouped()) {
+              metrics.addOrMergeTimelineMetric(metric);
+            } else {
+              metrics.getMetrics().add(metric.getTimelineMetric());
+            }
+          }
+        }
+      } else {
+        // No aggregation requested
+        // Execution never goes here, function always contain at least 1 element
+        getTimelineMetricsFromResultSet(metrics, null, condition, rs);
+      }
+    }
+  }
+
+  private static TimelineMetric getTimelineMetricFromResultSet(ResultSet rs)
+    throws SQLException, IOException {
+    TimelineMetric metric = getTimelineMetricCommonsFromResultSet(rs);
+    TreeMap<Long, Double> sortedByTimeMetrics = PhoenixHBaseAccessor.readMetricFromJSON(rs.getString("METRICS"));
+    metric.setMetricValues(sortedByTimeMetrics);
+    return metric;
+  }
+
+  private static TimelineMetric getTimelineMetricCommonsFromResultSet(ResultSet rs)
+    throws SQLException {
+    TimelineMetric metric = new TimelineMetric();
+    metric.setMetricName(rs.getString("METRIC_NAME"));
+    metric.setAppId(rs.getString("APP_ID"));
+    metric.setInstanceId(rs.getString("INSTANCE_ID"));
+    metric.setHostName(rs.getString("HOSTNAME"));
+    metric.setStartTime(rs.getLong("SERVER_TIME"));
+    metric.setType(rs.getString("UNITS"));
+    return metric;
+  }
+
+  private static void getTimelineMetricsFromResultSet(TimelineMetrics metrics, Function f, Condition condition, ResultSet rs) throws SQLException, IOException {
+    TimelineMetric metric = getTimelineMetricFromResultSet(rs);
+    if (f != null && f.getSuffix() != null) {
+      metric.setMetricName(metric.getMetricName() + f.getSuffix());
+    }
+    if (condition.isGrouped()) {
+      metrics.addOrMergeTimelineMetric(metric);
+    } else {
+      metrics.getMetrics().add(metric);
+    }
+  }
+
+
+  private static SingleValuedTimelineMetric getAggregatedTimelineMetricFromResultSet(ResultSet rs,
+                                                                                     Function f) throws SQLException, IOException {
+
+    Function function = (f != null) ? f : Function.DEFAULT_VALUE_FUNCTION;
+    SingleValuedTimelineMetric metric = new SingleValuedTimelineMetric(
+      rs.getString("METRIC_NAME") + function.getSuffix(),
+      rs.getString("APP_ID"),
+      rs.getString("INSTANCE_ID"),
+      rs.getString("HOSTNAME"),
+      rs.getLong("SERVER_TIME")
+    );
+
+    double value;
+    switch(function.getReadFunction()){
+      case AVG:
+        value = rs.getDouble("METRIC_SUM") / rs.getInt("METRIC_COUNT");
+        break;
+      case MIN:
+        value = rs.getDouble("METRIC_MIN");
+        break;
+      case MAX:
+        value = rs.getDouble("METRIC_MAX");
+        break;
+      case SUM:
+        value = rs.getDouble("METRIC_SUM");
+        break;
+      default:
+        value = rs.getDouble("METRIC_SUM") / rs.getInt("METRIC_COUNT");
+        break;
+    }
+
+    metric.setSingleTimeseriesValue(rs.getLong("SERVER_TIME"), value);
+
+    return metric;
+  }
+}
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/AggregatorUtils.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/AggregatorUtils.java
index 7cc81ce..9848e6b 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/AggregatorUtils.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/AggregatorUtils.java
@@ -252,17 +252,4 @@ public class AggregatorUtils {
     return currentTime - (currentTime % aggregatorPeriod);
   }
 
-  public static String getJavaRegexFromSqlRegex(String sqlRegex) {
-    String javaRegEx;
-    if (sqlRegex.contains("*") || sqlRegex.contains("__%")) {
-      //Special case handling for metric name with * and __%.
-      //For example, dfs.NNTopUserOpCounts.windowMs=300000.op=*.user=%.count
-      // or dfs.NNTopUserOpCounts.windowMs=300000.op=__%.user=%.count
-      String metricNameWithEscSeq = sqlRegex.replace("*", "\\*").replace("__%", "..%");
-      javaRegEx = metricNameWithEscSeq.replace("%", ".*");
-    } else {
-      javaRegEx = sqlRegex.replace("%", ".*");
-    }
-    return javaRegEx;
-  }
 }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricAggregatorFactory.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricAggregatorFactory.java
index b395b39..0f23ef1 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricAggregatorFactory.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricAggregatorFactory.java
@@ -479,6 +479,7 @@ public class TimelineMetricAggregatorFactory {
     );
   }
 
+  //TODO : Fix this.
   public static TimelineMetricAggregator createFilteringTimelineMetricAggregatorMinute(PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf, TimelineMetricMetadataManager metricMetadataManager, MetricCollectorHAController haController, ConcurrentHashMap<String, Long> postedAggregatedMap) {
     String checkpointDir = metricsConf.get(
       TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricAppAggregator.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricAppAggregator.java
index 4c62366..3ea585a 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricAppAggregator.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricAppAggregator.java
@@ -146,6 +146,8 @@ public class TimelineMetricAppAggregator {
             TimelineMetricMetadata timelineMetricMetadata = new TimelineMetricMetadata(clusterMetric.getMetricName(),
               appId, clusterMetric.getInstanceId(), hostMetricMetadata.getUnits(), hostMetricMetadata.getType(), hostMetricMetadata.getSeriesStartTime(),
               hostMetricMetadata.isSupportsAggregates(), TimelineMetricsFilter.acceptMetric(clusterMetric.getMetricName(), appId));
+            byte[] uuid = metadataManagerInstance.getUuid(clusterMetric.getMetricName(), appId, clusterMetric.getInstanceId(), StringUtils.EMPTY, true);
+            timelineMetricMetadata.setUuid(uuid);
             metadataManagerInstance.putIfModifiedTimelineMetricMetadata(timelineMetricMetadata);
           }
         }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricClusterAggregatorSecond.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricClusterAggregatorSecond.java
index 1fd6b7b..c9998d9 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricClusterAggregatorSecond.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricClusterAggregatorSecond.java
@@ -22,11 +22,11 @@ import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguratio
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_CLUSTER_AGGREGATOR_INTERPOLATION_ENABLED;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_EVENT_METRIC_PATTERNS;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRIC_AGGREGATION_SQL_FILTERS;
-import static org.apache.ambari.metrics.core.timeline.aggregators.AggregatorUtils.getJavaRegexFromSqlRegex;
 import static org.apache.ambari.metrics.core.timeline.aggregators.AggregatorUtils.getTimeSlices;
 import static org.apache.ambari.metrics.core.timeline.aggregators.AggregatorUtils.sliceFromTimelineMetric;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.GET_METRIC_SQL;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
+import static org.apache.hadoop.metrics2.sink.timeline.TimelineMetricUtils.getJavaMetricPatterns;
 
 import java.io.IOException;
 import java.sql.ResultSet;
@@ -37,9 +37,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.mutable.MutableInt;
@@ -69,7 +66,7 @@ public class TimelineMetricClusterAggregatorSecond extends AbstractTimelineAggre
   protected final boolean interpolationEnabled;
   private TimelineMetricMetadataManager metadataManagerInstance;
   private String skipAggrPatternStrings;
-  private Set<Pattern> skipInterpolationMetricPatterns = new HashSet<>();
+  private List<String> skipInterpolationMetricPatterns = new ArrayList<>();
   private final static String liveHostsMetricName = "live_hosts";
 
   public TimelineMetricClusterAggregatorSecond(AGGREGATOR_NAME aggregatorName,
@@ -98,12 +95,10 @@ public class TimelineMetricClusterAggregatorSecond extends AbstractTimelineAggre
     String skipInterpolationMetricPatternStrings = metricsConf.get(TIMELINE_METRICS_EVENT_METRIC_PATTERNS, "");
 
     if (StringUtils.isNotEmpty(skipInterpolationMetricPatternStrings)) {
-      for (String patternString : skipInterpolationMetricPatternStrings.split(",")) {
-        String javaPatternString = getJavaRegexFromSqlRegex(patternString);
-        LOG.info("SQL pattern " + patternString + " converted to Java pattern : " + javaPatternString);
-        skipInterpolationMetricPatterns.add(Pattern.compile(javaPatternString));
-      }
+      LOG.info("Skipping Interpolation for patterns : " + skipInterpolationMetricPatternStrings);
+      skipInterpolationMetricPatterns.addAll(getJavaMetricPatterns(skipInterpolationMetricPatternStrings));
     }
+
     this.timelineMetricReadHelper = new TimelineMetricReadHelper(metadataManager, true);
   }
 
@@ -221,14 +216,7 @@ public class TimelineMetricClusterAggregatorSecond extends AbstractTimelineAggre
       return 0;
     }
 
-    boolean skipInterpolationForMetric = false;
-    for (Pattern pattern : skipInterpolationMetricPatterns) {
-      Matcher m = pattern.matcher(metric.getMetricName());
-      if (m.matches()) {
-        skipInterpolationForMetric = true;
-        LOG.debug("Skipping interpolation for " + metric.getMetricName());
-      }
-    }
+    boolean skipInterpolationForMetric = shouldInterpolationBeSkipped(metric.getMetricName());
 
     Map<TimelineClusterMetric, Double> clusterMetrics = sliceFromTimelineMetric(metric, timeSlices, !skipInterpolationForMetric && interpolationEnabled);
 
@@ -279,10 +267,17 @@ public class TimelineMetricClusterAggregatorSecond extends AbstractTimelineAggre
       MetricClusterAggregate metricClusterAggregate = new MetricClusterAggregate(
         (double) numOfHosts, 1, null, (double) numOfHosts, (double) numOfHosts);
 
-      metadataManagerInstance.getUuid(timelineClusterMetric);
-
+      metadataManagerInstance.getUuid(timelineClusterMetric, true);
       aggregateClusterMetrics.put(timelineClusterMetric, metricClusterAggregate);
     }
+  }
 
+  private boolean shouldInterpolationBeSkipped(String metricName) {
+    for (String pattern : skipInterpolationMetricPatterns) {
+      if (metricName.matches(pattern)) {
+        return true;
+      }
+    }
+    return false;
   }
 }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricFilteringHostAggregator.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricFilteringHostAggregator.java
index b0aec2f..82fb605 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricFilteringHostAggregator.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/TimelineMetricFilteringHostAggregator.java
@@ -27,7 +27,6 @@ import org.apache.ambari.metrics.core.timeline.availability.MetricCollectorHACon
 import org.apache.ambari.metrics.core.timeline.discovery.TimelineMetricMetadataManager;
 import org.apache.ambari.metrics.core.timeline.query.Condition;
 import org.apache.ambari.metrics.core.timeline.query.DefaultCondition;
-import org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -80,7 +79,7 @@ public class TimelineMetricFilteringHostAggregator extends TimelineMetricHostAgg
       LOG.debug("Already aggregated hostnames based on postedAggregatedMap : " + aggregatedHostnames);
       LOG.debug("Hostnames that will be aggregated : " + notAggregatedHostnames);
     }
-    List<byte[]> uuids = metricMetadataManager.getUuids(new ArrayList<String>(), notAggregatedHostnames, "", "");
+    List<byte[]> uuids = metricMetadataManager.getUuidsForGetMetricQuery(new ArrayList<String>(), notAggregatedHostnames, "", "");
 
     Condition condition = new DefaultCondition(uuids, null, null, null, null, startTime,
       endTime, null, null, true);
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/v2/TimelineMetricFilteringHostAggregator.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/v2/TimelineMetricFilteringHostAggregator.java
index 1026cbe..72d7424 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/v2/TimelineMetricFilteringHostAggregator.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/aggregators/v2/TimelineMetricFilteringHostAggregator.java
@@ -77,7 +77,7 @@ public class TimelineMetricFilteringHostAggregator extends TimelineMetricHostAgg
       LOG.debug("Already aggregated hostnames based on postedAggregatedMap : " + aggregatedHostnames);
       LOG.debug("Hostnames that will be aggregated : " + notAggregatedHostnames);
     }
-    List<byte[]> uuids = metricMetadataManager.getUuids(new ArrayList<String>(), notAggregatedHostnames, "", "");
+    List<byte[]> uuids = metricMetadataManager.getUuidsForGetMetricQuery(new ArrayList<String>(), notAggregatedHostnames, "", "");
 
     EmptyCondition condition = new EmptyCondition();
     condition.setDoUpdate(true);
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/availability/MetricCollectorHAController.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/availability/MetricCollectorHAController.java
index ee28d87..adbe8e7 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/availability/MetricCollectorHAController.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/availability/MetricCollectorHAController.java
@@ -17,9 +17,12 @@
  */
 package org.apache.ambari.metrics.core.timeline.availability;
 
+import static org.apache.ambari.metrics.core.timeline.availability.AggregationTaskRunner.ACTUAL_AGGREGATOR_NAMES;
 import static org.apache.helix.model.IdealState.RebalanceMode.FULL_AUTO;
 
 import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeSet;
@@ -304,4 +307,24 @@ public class MetricCollectorHAController {
       }
     }
   }
+
+  public Map<String, String> getAggregationSummary() {
+    Map<String, String> summary = new HashMap<>();
+
+    CheckpointManager checkpointManager = aggregationTaskRunner.getCheckpointManager();
+
+    summary.put(ACTUAL_AGGREGATOR_NAMES.get(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_RECORD_MINUTE),
+      new Date(checkpointManager.readCheckpoint(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_RECORD_MINUTE)).toString());
+    summary.put(ACTUAL_AGGREGATOR_NAMES.get(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_RECORD_HOURLY),
+      new Date(checkpointManager.readCheckpoint(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_RECORD_HOURLY)).toString());
+
+    summary.put(ACTUAL_AGGREGATOR_NAMES.get(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_AGGREGATE_SECOND),
+      new Date(checkpointManager.readCheckpoint(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_AGGREGATE_SECOND)).toString());
+    summary.put(ACTUAL_AGGREGATOR_NAMES.get(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_AGGREGATE_MINUTE),
+      new Date(checkpointManager.readCheckpoint(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_AGGREGATE_MINUTE)).toString());
+    summary.put(ACTUAL_AGGREGATOR_NAMES.get(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_AGGREGATE_HOURLY),
+      new Date(checkpointManager.readCheckpoint(AggregationTaskRunner.AGGREGATOR_NAME.METRIC_AGGREGATE_HOURLY)).toString());
+
+    return summary;
+  }
 }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/discovery/TimelineMetricMetadataManager.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/discovery/TimelineMetricMetadataManager.java
index e13c884..1ca5bc0 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/discovery/TimelineMetricMetadataManager.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/discovery/TimelineMetricMetadataManager.java
@@ -17,12 +17,15 @@
  */
 package org.apache.ambari.metrics.core.timeline.discovery;
 
+import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URISyntaxException;
 import java.sql.SQLException;
 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;
@@ -32,10 +35,12 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration;
 import org.apache.ambari.metrics.core.timeline.uuid.MetricUuidGenStrategy;
-import org.apache.ambari.metrics.core.timeline.uuid.RandomUuidGenStrategy;
+import org.apache.ambari.metrics.core.timeline.uuid.MD5UuidGenStrategy;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang.ArrayUtils;
@@ -50,16 +55,16 @@ import org.apache.ambari.metrics.core.timeline.PhoenixHBaseAccessor;
 import org.apache.ambari.metrics.core.timeline.aggregators.TimelineClusterMetric;
 import org.apache.ambari.metrics.core.timeline.uuid.HashBasedUuidGenStrategy;
 
-import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.DISABLE_METRIC_METADATA_MGMT;
+import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TRANSIENT_METRIC_PATTERNS;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.METRICS_METADATA_SYNC_INIT_DELAY;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.METRICS_METADATA_SYNC_SCHEDULE_DELAY;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_UUID_GEN_STRATEGY;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRIC_METADATA_FILTERS;
-import static org.apache.ambari.metrics.core.timeline.aggregators.AggregatorUtils.getJavaRegexFromSqlRegex;
+import static org.apache.hadoop.metrics2.sink.timeline.TimelineMetricUtils.getJavaMetricPatterns;
+import static org.apache.hadoop.metrics2.sink.timeline.TimelineMetricUtils.getJavaRegexFromSqlRegex;
 
 public class TimelineMetricMetadataManager {
   private static final Log LOG = LogFactory.getLog(TimelineMetricMetadataManager.class);
-  private boolean isDisabled = false;
   // Cache all metadata on retrieval
   private final Map<TimelineMetricMetadataKey, TimelineMetricMetadata> METADATA_CACHE = new ConcurrentHashMap<>();
   private final Map<String, TimelineMetricMetadataKey> uuidKeyMap = new ConcurrentHashMap<>();
@@ -70,9 +75,13 @@ public class TimelineMetricMetadataManager {
   // Sync only when needed
   AtomicBoolean SYNC_HOSTED_APPS_METADATA = new AtomicBoolean(false);
   AtomicBoolean SYNC_HOSTED_INSTANCES_METADATA = new AtomicBoolean(false);
+
   private MetricUuidGenStrategy uuidGenStrategy = new HashBasedUuidGenStrategy();
   public static final int TIMELINE_METRIC_UUID_LENGTH = 16;
-  public static final int HOSTNAME_UUID_LENGTH = 4;
+  public static final int HOSTNAME_UUID_LENGTH = 16;
+
+  //Transient metric patterns. No UUID management and aggregation for such metrics.
+  private List<String> transientMetricPatterns = new ArrayList<>();
 
   // Single thread to sync back new writes to the store
   private final ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor();
@@ -82,7 +91,7 @@ public class TimelineMetricMetadataManager {
 
   TimelineMetricMetadataSync metricMetadataSync;
   // Filter metrics names matching given patterns, from metadata
-  final List<String> metricNameFilters = new ArrayList<>();
+  private final List<String> metricNameFilters = new ArrayList<>();
 
   // Test friendly construction since mock instrumentation is difficult to get
   // working with hadoop mini cluster
@@ -95,6 +104,12 @@ public class TimelineMetricMetadataManager {
     }
 
     uuidGenStrategy = getUuidStrategy(metricsConf);
+
+    String transientMetricPatternsString = metricsConf.get(TRANSIENT_METRIC_PATTERNS, StringUtils.EMPTY);
+    if (StringUtils.isNotEmpty(transientMetricPatternsString)) {
+      LOG.info("Skipping UUID for patterns : " + transientMetricPatternsString);
+      transientMetricPatterns.addAll(getJavaMetricPatterns(transientMetricPatternsString));
+    }
   }
 
   public TimelineMetricMetadataManager(PhoenixHBaseAccessor hBaseAccessor) throws MalformedURLException, URISyntaxException {
@@ -105,34 +120,30 @@ public class TimelineMetricMetadataManager {
    * Initialize Metadata from the store
    */
   public void initializeMetadata() {
-    if (metricsConf.getBoolean(DISABLE_METRIC_METADATA_MGMT, false)) {
-      isDisabled = true;
-    } else {
-      metricMetadataSync = new TimelineMetricMetadataSync(this);
-      // Schedule the executor to sync to store
-      executorService.scheduleWithFixedDelay(metricMetadataSync,
-        metricsConf.getInt(METRICS_METADATA_SYNC_INIT_DELAY, 120), // 2 minutes
-        metricsConf.getInt(METRICS_METADATA_SYNC_SCHEDULE_DELAY, 300), // 5 minutes
-        TimeUnit.SECONDS);
-      // Read from store and initialize map
-      try {
-        Map<TimelineMetricMetadataKey, TimelineMetricMetadata> metadata = getMetadataFromStore();
-
-        LOG.info("Retrieved " + metadata.size() + ", metadata objects from store.");
-        // Store in the cache
-        METADATA_CACHE.putAll(metadata);
-
-        Map<String, TimelineMetricHostMetadata> hostedAppData = getHostedAppsFromStore();
-
-        LOG.info("Retrieved " + hostedAppData.size() + " host objects from store.");
-        HOSTED_APPS_MAP.putAll(hostedAppData);
-
-        loadUuidMapsOnInit();
-
-        hBaseAccessor.setMetadataInstance(this);
-      } catch (SQLException e) {
-        LOG.warn("Exception loading metric metadata", e);
-      }
+    metricMetadataSync = new TimelineMetricMetadataSync(this);
+    // Schedule the executor to sync to store
+    executorService.scheduleWithFixedDelay(metricMetadataSync,
+      metricsConf.getInt(METRICS_METADATA_SYNC_INIT_DELAY, 120), // 2 minutes
+      metricsConf.getInt(METRICS_METADATA_SYNC_SCHEDULE_DELAY, 300), // 5 minutes
+      TimeUnit.SECONDS);
+    // Read from store and initialize map
+    try {
+      Map<TimelineMetricMetadataKey, TimelineMetricMetadata> metadata = getMetadataFromStore();
+
+      LOG.info("Retrieved " + metadata.size() + ", metadata objects from store.");
+      // Store in the cache
+      METADATA_CACHE.putAll(metadata);
+
+      Map<String, TimelineMetricHostMetadata> hostedAppData = getHostedAppsFromStore();
+
+      LOG.info("Retrieved " + hostedAppData.size() + " host objects from store.");
+      HOSTED_APPS_MAP.putAll(hostedAppData);
+
+      loadUuidMapsOnInit();
+
+      hBaseAccessor.setMetadataInstance(this);
+    } catch (SQLException e) {
+      LOG.warn("Exception loading metric metadata", e);
     }
   }
 
@@ -167,6 +178,7 @@ public class TimelineMetricMetadataManager {
   public void markSuccessOnSyncHostedInstanceMetadata() {
     SYNC_HOSTED_INSTANCES_METADATA.set(false);
   }
+
   /**
    * Test metric name for valid patterns and return true/false
    */
@@ -219,7 +231,9 @@ public class TimelineMetricMetadataManager {
     if (apps == null) {
       apps = new ConcurrentHashMap<>();
       if (timelineMetricHostMetadata == null) {
-        HOSTED_APPS_MAP.put(hostname, new TimelineMetricHostMetadata(apps));
+        TimelineMetricHostMetadata newHostMetadata = new TimelineMetricHostMetadata(apps);
+        newHostMetadata.setUuid(getUuidForHostname(hostname, true));
+        HOSTED_APPS_MAP.put(hostname, newHostMetadata);
       } else {
         HOSTED_APPS_MAP.get(hostname).setHostedApps(apps);
       }
@@ -260,8 +274,8 @@ public class TimelineMetricMetadataManager {
     hBaseAccessor.saveInstanceHostsMetadata(hostedInstancesMetadata);
   }
 
-  public TimelineMetricMetadata getTimelineMetricMetadata(TimelineMetric timelineMetric, boolean isWhitelisted) {
-    return new TimelineMetricMetadata(
+  public TimelineMetricMetadata createTimelineMetricMetadata(TimelineMetric timelineMetric, boolean isWhitelisted) {
+    TimelineMetricMetadata timelineMetricMetadata = new TimelineMetricMetadata(
       timelineMetric.getMetricName(),
       timelineMetric.getAppId(),
       timelineMetric.getInstanceId(),
@@ -271,10 +285,13 @@ public class TimelineMetricMetadataManager {
       supportAggregates(timelineMetric),
       isWhitelisted
     );
-  }
 
-  public boolean isDisabled() {
-    return isDisabled;
+    //Set UUID for metadata on the write path. Do not pass in hostname here since we only want Metric metadata, not host metadata.
+    if (!isTransientMetric(timelineMetric.getMetricName(), timelineMetric.getAppId())) {
+      byte[] uuid = getUuid(timelineMetric.getMetricName(), timelineMetric.getAppId(), timelineMetric.getInstanceId(), null, true);
+      timelineMetricMetadata.setUuid(uuid);
+    }
+    return timelineMetricMetadata;
   }
 
   boolean isDistributedModeEnabled() {
@@ -310,7 +327,6 @@ public class TimelineMetricMetadataManager {
   // UUID Management
   //////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
 
-
   /**
    * Load the UUID mappings from the UUID table on startup.
    */
@@ -333,24 +349,26 @@ public class TimelineMetricMetadataManager {
 
   /**
    * Returns the UUID gen strategy.
-   * @param configuration
-   * @return
+   * @param configuration the config
+   * @return the UUID generator of type org.apache.ambari.metrics.core.timeline.uuid.MetricUuidGenStrategy
    */
   private MetricUuidGenStrategy getUuidStrategy(Configuration configuration) {
     String strategy = configuration.get(TIMELINE_METRICS_UUID_GEN_STRATEGY, "");
-    if ("random".equalsIgnoreCase(strategy)) {
-      return new RandomUuidGenStrategy();
-    } else {
+    if ("hash".equalsIgnoreCase(strategy)) {
       return new HashBasedUuidGenStrategy();
+    } else {
+      //Default
+      return new MD5UuidGenStrategy();
     }
   }
 
   /**
    * Given the hostname, generates a byte array of length 'HOSTNAME_UUID_LENGTH'
-   * @param hostname
+   * @param hostname the hostname
+   * @param createIfNotPresent Generate UUID if not present.
    * @return uuid byte array of length 'HOSTNAME_UUID_LENGTH'
    */
-  private byte[] getUuidForHostname(String hostname) {
+  private byte[] getUuidForHostname(String hostname, boolean createIfNotPresent) {
 
     TimelineMetricHostMetadata timelineMetricHostMetadata = HOSTED_APPS_MAP.get(hostname);
     if (timelineMetricHostMetadata != null) {
@@ -360,13 +378,16 @@ public class TimelineMetricMetadataManager {
       }
     }
 
-    byte[] uuid = uuidGenStrategy.computeUuid(hostname, HOSTNAME_UUID_LENGTH);
+    if (!createIfNotPresent) {
+      LOG.warn("UUID not found for " + hostname + ", createIfNotPresent is false");
+      return null;
+    }
 
+    byte[] uuid = uuidGenStrategy.computeUuid(hostname, HOSTNAME_UUID_LENGTH);
     String uuidStr = new String(uuid);
     if (uuidHostMap.containsKey(uuidStr)) {
-      //TODO fix the collisions
       LOG.error("Duplicate key computed for " + hostname +", Collides with  " + uuidHostMap.get(uuidStr));
-      return uuid;
+      return null;
     }
 
     if (timelineMetricHostMetadata == null) {
@@ -381,10 +402,12 @@ public class TimelineMetricMetadataManager {
 
   /**
    * Given a timelineClusterMetric instance, generates a UUID for Metric-App-Instance combination.
-   * @param timelineClusterMetric
+   * @param timelineClusterMetric The timeline cluster metric for which the UUID needs to be generated.
+   * @param createIfNotPresent Generate UUID if not present.
    * @return uuid byte array of length 'TIMELINE_METRIC_UUID_LENGTH'
    */
-  public byte[] getUuid(TimelineClusterMetric timelineClusterMetric) {
+  public byte[] getUuid(TimelineClusterMetric timelineClusterMetric, boolean createIfNotPresent) {
+
     TimelineMetricMetadataKey key = new TimelineMetricMetadataKey(timelineClusterMetric.getMetricName(),
       timelineClusterMetric.getAppId(), timelineClusterMetric.getInstanceId());
 
@@ -396,20 +419,19 @@ public class TimelineMetricMetadataManager {
       }
     }
 
+    if (!createIfNotPresent) {
+      LOG.warn("UUID not found for " + key + ", createIfNotPresent is false");
+      return null;
+    }
+
     byte[] uuid = uuidGenStrategy.computeUuid(timelineClusterMetric, TIMELINE_METRIC_UUID_LENGTH);
 
     String uuidStr = new String(uuid);
     if (uuidKeyMap.containsKey(uuidStr) && !uuidKeyMap.get(uuidStr).equals(key)) {
       TimelineMetricMetadataKey collidingKey = (TimelineMetricMetadataKey)uuidKeyMap.get(uuidStr);
-      //TODO fix the collisions
-      /**
-       * 2017-08-23 14:12:35,922 ERROR TimelineMetricMetadataManager:
-       * Duplicate key [52, 50, 51, 53, 50, 53, 53, 53, 49, 54, 57, 50, 50, 54, 0, 0]([B@278a93f9) computed for
-       * TimelineClusterMetric{metricName='sdisk_dm-11_write_count', appId='hbase', instanceId='', timestamp=1503497400000}, Collides with
-       * TimelineMetricMetadataKey{metricName='sdisk_dm-20_write_count', appId='hbase', instanceId=''}
-       */
-      LOG.error("Duplicate key " + Arrays.toString(uuid) + "(" + uuid +  ") computed for " + timelineClusterMetric.toString() + ", Collides with  " + collidingKey.toString());
-      return uuid;
+      LOG.error("Duplicate key " + Arrays.toString(uuid) + "(" + uuid +  ") computed for " + timelineClusterMetric.toString()
+        + ", Collides with  " + collidingKey.toString());
+      return null;
     }
 
     if (timelineMetricMetadata == null) {
@@ -427,24 +449,36 @@ public class TimelineMetricMetadataManager {
   }
 
   /**
-   * Given a timelineMetric instance, generates a UUID for Metric-App-Instance combination.
-   * @param timelineMetric
+   * Given a timelineMetric instance, generates a UUID for Metric-App-Instance-Host combination.
+   * @param timelineMetric The timeline metric for which the UUID needs to be generated.
+   * @param createIfNotPresent Generate UUID if not present.
    * @return uuid byte array of length 'TIMELINE_METRIC_UUID_LENGTH' + 'HOSTNAME_UUID_LENGTH'
    */
-  public byte[] getUuid(TimelineMetric timelineMetric) {
+  public byte[] getUuid(TimelineMetric timelineMetric, boolean createIfNotPresent) {
 
     byte[] metricUuid = getUuid(new TimelineClusterMetric(timelineMetric.getMetricName(), timelineMetric.getAppId(),
-      timelineMetric.getInstanceId(), -1l));
-    byte[] hostUuid = getUuidForHostname(timelineMetric.getHostName());
+      timelineMetric.getInstanceId(), -1l), createIfNotPresent);
+    byte[] hostUuid = getUuidForHostname(timelineMetric.getHostName(), createIfNotPresent);
 
+    if (metricUuid == null || hostUuid == null) {
+      return null;
+    }
     return ArrayUtils.addAll(metricUuid, hostUuid);
   }
 
-  public byte[] getUuid(String metricName, String appId, String instanceId, String hostname) {
+  /**
+   * Given a metric name, appId, instanceId and hotname, generates a UUID for Metric-App-Instance-Host combination.
+   * @param createIfNotPresent Generate UUID if not present.
+   * @return uuid byte array of length 'TIMELINE_METRIC_UUID_LENGTH' + 'HOSTNAME_UUID_LENGTH'
+   */
+  public byte[] getUuid(String metricName, String appId, String instanceId, String hostname, boolean createIfNotPresent) {
 
-    byte[] metricUuid = getUuid(new TimelineClusterMetric(metricName, appId, instanceId, -1l));
+    byte[] metricUuid = getUuid(new TimelineClusterMetric(metricName, appId, instanceId, -1l), createIfNotPresent);
     if (StringUtils.isNotEmpty(hostname)) {
-      byte[] hostUuid = getUuidForHostname(hostname);
+      byte[] hostUuid = getUuidForHostname(hostname, createIfNotPresent);
+      if (hostUuid == null || metricUuid == null) {
+        return null;
+      }
       return ArrayUtils.addAll(metricUuid, hostUuid);
     }
     return metricUuid;
@@ -461,6 +495,11 @@ public class TimelineMetricMetadataManager {
     return key != null ? key.getMetricName() : null;
   }
 
+  /**
+   * Given a UUID (from DB hopefully), return the timeline metric it is associated with.
+   * @param uuid 'TIMELINE_METRIC_UUID_LENGTH' + 'HOSTNAME_UUID_LENGTH' byte UUID.
+   * @return TimelineMetric object if present in the metadata.
+   */
   public TimelineMetric getMetricFromUuid(byte[] uuid) {
     if (uuid == null) {
       return null;
@@ -487,15 +526,26 @@ public class TimelineMetricMetadataManager {
     }
   }
 
+  public List<byte[]> getUuidsForGetMetricQuery(Collection<String> metricNames,
+                                                List<String> hostnames,
+                                                String appId,
+                                                String instanceId) {
+    return getUuidsForGetMetricQuery(metricNames, hostnames, appId, instanceId, Collections.EMPTY_LIST);
+  }
   /**
    * Returns the set of UUIDs for a given GET request. If there are wildcards (%), resolves them based on UUID map.
+   * If metricName-App-Instance or hostname not present in Metadata, the combination will be skipped.
    * @param metricNames
    * @param hostnames
    * @param appId
    * @param instanceId
    * @return Set of UUIds
    */
-  public List<byte[]> getUuids(Collection<String> metricNames, List<String> hostnames, String appId, String instanceId) {
+  public List<byte[]> getUuidsForGetMetricQuery(Collection<String> metricNames,
+                                                List<String> hostnames,
+                                                String appId,
+                                                String instanceId,
+                                                List<String> transientMetricNames) {
 
     Collection<String> sanitizedMetricNames = new HashSet<>();
     List<byte[]> uuids = new ArrayList<>();
@@ -518,36 +568,28 @@ public class TimelineMetricMetadataManager {
       return uuids;
     }
 
-    Set<String> sanitizedHostNames = new HashSet<>();
-    if (CollectionUtils.isNotEmpty(hostnames)) {
-      for (String hostname : hostnames) {
-        if (hostname.contains("%")) {
-          String hostRegEx;
-          hostRegEx = hostname.replace("%", ".*");
-          for (String host : HOSTED_APPS_MAP.keySet()) {
-            if (host.matches(hostRegEx)) {
-              sanitizedHostNames.add(host);
-            }
-          }
-        } else {
-          sanitizedHostNames.add(hostname);
-        }
-      }
-    }
+    Set<String> sanitizedHostNames = getSanitizedHostnames(hostnames);
 
     if ( StringUtils.isNotEmpty(appId) && !(appId.equals("HOST") || appId.equals("FLUME_HANDLER"))) { //HACK.. Why??
       appId = appId.toLowerCase();
     }
     if (CollectionUtils.isNotEmpty(sanitizedHostNames)) {
       if (CollectionUtils.isNotEmpty(sanitizedMetricNames)) {
+
+        //Skip getting UUID if it is a transient metric.
+        //An attempt to get it will also be OK as we don't add null UUIDs.
         for (String metricName : sanitizedMetricNames) {
+          if (isTransientMetric(metricName, appId)) {
+            transientMetricNames.add(metricName);
+            continue;
+          }
           TimelineMetric metric = new TimelineMetric();
           metric.setMetricName(metricName);
           metric.setAppId(appId);
           metric.setInstanceId(instanceId);
           for (String hostname : sanitizedHostNames) {
             metric.setHostName(hostname);
-            byte[] uuid = getUuid(metric);
+            byte[] uuid = getUuid(metric, false);
             if (uuid != null) {
               uuids.add(uuid);
             }
@@ -555,7 +597,7 @@ public class TimelineMetricMetadataManager {
         }
       } else {
         for (String hostname : sanitizedHostNames) {
-          byte[] uuid = getUuidForHostname(hostname);
+          byte[] uuid = getUuidForHostname(hostname, false);
           if (uuid != null) {
             uuids.add(uuid);
           }
@@ -563,8 +605,13 @@ public class TimelineMetricMetadataManager {
       }
     } else {
       for (String metricName : sanitizedMetricNames) {
+        //Skip getting UUID if it is a transient metric. An attempt to get it will also be OK as we don't add null UUIDs.
+        if (isTransientMetric(metricName, appId)) {
+          transientMetricNames.add(metricName);
+          continue;
+        }
         TimelineClusterMetric metric = new TimelineClusterMetric(metricName, appId, instanceId, -1l);
-        byte[] uuid = getUuid(metric);
+        byte[] uuid = getUuid(metric, false);
         if (uuid != null) {
           uuids.add(uuid);
         }
@@ -574,15 +621,24 @@ public class TimelineMetricMetadataManager {
     return uuids;
   }
 
-  public Map<String, TimelineMetricMetadataKey> getUuidKeyMap() {
-    return uuidKeyMap;
-  }
+  //////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
 
   public List<String> getNotLikeHostnames(List<String> hostnames) {
     List<String> result = new ArrayList<>();
+    Set<String> sanitizedHostNames = getSanitizedHostnames(hostnames);
+    for (String hostname: HOSTED_APPS_MAP.keySet()) {
+      if (!sanitizedHostNames.contains(hostname)) {
+        result.add(hostname);
+      }
+    }
+    return result;
+  }
+
+  private Set<String> getSanitizedHostnames(List<String> hostnamedWithOrWithoutWildcard) {
+
     Set<String> sanitizedHostNames = new HashSet<>();
-    if (CollectionUtils.isNotEmpty(hostnames)) {
-      for (String hostname : hostnames) {
+    if (CollectionUtils.isNotEmpty(hostnamedWithOrWithoutWildcard)) {
+      for (String hostname : hostnamedWithOrWithoutWildcard) {
         if (hostname.contains("%")) {
           String hostRegEx;
           hostRegEx = hostname.replace("%", ".*");
@@ -596,12 +652,99 @@ public class TimelineMetricMetadataManager {
         }
       }
     }
+    return sanitizedHostNames;
+  }
 
-    for (String hostname: HOSTED_APPS_MAP.keySet()) {
-      if (!sanitizedHostNames.contains(hostname)) {
-        result.add(hostname);
+  /**
+   *
+   * @param appId
+   * @param metricPattern
+   * @param includeBlacklistedMetrics
+   * @return
+   * @throws SQLException
+   * @throws IOException
+   */
+  public Map<String, List<TimelineMetricMetadata>> getTimelineMetricMetadataByAppId(String appId, String metricPattern,
+                                                                             boolean includeBlacklistedMetrics) throws SQLException, IOException {
+
+    Map<TimelineMetricMetadataKey, TimelineMetricMetadata> metadata = getMetadataCache();
+
+    boolean filterByAppId = StringUtils.isNotEmpty(appId);
+    boolean filterByMetricName = StringUtils.isNotEmpty(metricPattern);
+    Pattern metricFilterPattern = null;
+    if (filterByMetricName) {
+      metricFilterPattern = Pattern.compile(metricPattern);
+    }
+
+    // Group Metadata by AppId
+    Map<String, List<TimelineMetricMetadata>> metadataByAppId = new HashMap<>();
+    for (TimelineMetricMetadata metricMetadata : metadata.values()) {
+
+      if (!includeBlacklistedMetrics && !metricMetadata.isWhitelisted()) {
+        continue;
+      }
+
+      String currentAppId = metricMetadata.getAppId();
+      if (filterByAppId && !currentAppId.equals(appId)) {
+        continue;
+      }
+
+      if (filterByMetricName) {
+        Matcher m = metricFilterPattern.matcher(metricMetadata.getMetricName());
+        if (!m.find()) {
+          continue;
+        }
+      }
+
+      List<TimelineMetricMetadata> metadataList = metadataByAppId.get(currentAppId);
+      if (metadataList == null) {
+        metadataList = new ArrayList<>();
+        metadataByAppId.put(currentAppId, metadataList);
       }
+
+      metadataList.add(metricMetadata);
     }
-    return result;
+
+    return metadataByAppId;
+  }
+
+  /**
+   * Returns metadata summary
+   * @return
+   * @throws IOException
+   * @throws SQLException
+   */
+  public Map<String, String> getMetadataSummary() throws IOException, SQLException {
+    Map<String, String> summary = new HashMap<>();
+    summary.put("Number of Hosts", String.valueOf(HOSTED_APPS_MAP.size()));
+    Map<String, List<TimelineMetricMetadata>> metadataMap = getTimelineMetricMetadataByAppId(StringUtils.EMPTY,
+      StringUtils.EMPTY,
+      true);
+
+    if (metadataMap != null) {
+      for (String appId : metadataMap.keySet()) {
+        summary.put(appId, String.valueOf(metadataMap.get(appId).size()));
+      }
+    }
+    return summary;
+  }
+
+
+  /**
+   *
+   * @param metricName
+   * @param appId
+   * @return
+   */
+  public boolean isTransientMetric(String metricName, String appId) {
+    //Currently we use only metric name. In the future we may use appId as well.
+
+    for (String pattern : transientMetricPatterns) {
+      if (metricName.matches(pattern)) {
+        return true;
+      }
+    }
+    return false;
   }
+
 }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/Condition.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/Condition.java
index fa118a3..6f07615 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/Condition.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/Condition.java
@@ -26,6 +26,7 @@ public interface Condition {
 
   List<byte[]> getUuids();
   List<String> getMetricNames();
+  List<String> getTransientMetricNames();
   boolean isPointInTime();
   boolean isGrouped();
   void setStatement(String statement);
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/ConditionBuilder.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/ConditionBuilder.java
index e779d77..9285c8b 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/ConditionBuilder.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/ConditionBuilder.java
@@ -21,6 +21,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.hadoop.metrics2.sink.timeline.Precision;
 import org.apache.ambari.metrics.core.timeline.aggregators.Function;
 
@@ -43,6 +44,7 @@ public class ConditionBuilder {
   private boolean isBottomN;
   private Function topNFunction;
   private List<byte[]> uuids;
+  private List<String> transientMetricNames;
 
   public ConditionBuilder(List<String> metricNames) {
     this.metricNames = metricNames;
@@ -128,16 +130,27 @@ public class ConditionBuilder {
     return this;
   }
 
+  public ConditionBuilder transientMetricNames(List<String> transientMetricNames) {
+    this.transientMetricNames = transientMetricNames;
+    return this;
+  }
+
   public Condition build() {
     if (topN == null) {
-      return new DefaultCondition(
-        uuids, metricNames,
-        hostnames, appId, instanceId, startTime, endTime,
-        precision, limit, grouped);
+      if (CollectionUtils.isEmpty(transientMetricNames)) {
+        return new DefaultCondition(
+          uuids, metricNames,
+          hostnames, appId, instanceId, startTime, endTime,
+          precision, limit, grouped);
+      } else {
+        return new TransientMetricCondition(
+          uuids, metricNames,
+          hostnames, appId, instanceId, startTime, endTime,
+          precision, limit, grouped, transientMetricNames);
+      }
     } else {
       return new TopNCondition(uuids, metricNames, hostnames, appId, instanceId,
         startTime, endTime, precision, limit, grouped, topN, topNFunction, isBottomN);
     }
   }
-
 }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/DefaultCondition.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/DefaultCondition.java
index 888f381..7f72187 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/DefaultCondition.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/DefaultCondition.java
@@ -18,6 +18,7 @@
 package org.apache.ambari.metrics.core.timeline.query;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
@@ -233,133 +234,28 @@ public class DefaultCondition implements Condition {
     boolean appendConjunction = false;
 
     if (CollectionUtils.isNotEmpty(uuids)) {
-
-      List<byte[]> uuidsHost = new ArrayList<>();
-      List<byte[]> uuidsMetric = new ArrayList<>();
-      List<byte[]> uuidsFull = new ArrayList<>();
-
-      if (getUuids() != null) {
-        for (byte[] uuid : uuids) {
-          if (uuid.length == TimelineMetricMetadataManager.TIMELINE_METRIC_UUID_LENGTH) {
-            uuidsMetric.add(uuid);
-          } else if (uuid.length == TimelineMetricMetadataManager.HOSTNAME_UUID_LENGTH) {
-            uuidsHost.add(uuid);
-          } else {
-            uuidsFull.add(uuid);
-          }
-        }
-
-        // Put a '(' first
-        sb.append("(");
-
-        //IN clause
-        // METRIC_NAME (NOT) IN (?,?,?,?)
-        if (CollectionUtils.isNotEmpty(uuidsFull)) {
-          sb.append("UUID");
-          if (uuidNotCondition) {
-            sb.append(" NOT");
-          }
-          sb.append(" IN (");
-          //Append ?,?,?,?
-          for (int i = 0; i < uuidsFull.size(); i++) {
-            sb.append("?");
-            if (i < uuidsFull.size() - 1) {
-              sb.append(", ");
-            }
-          }
-          sb.append(")");
-          appendConjunction = true;
-        }
-
-        //Put an AND if both types are present
-        if (CollectionUtils.isNotEmpty(uuidsFull) &&
-          CollectionUtils.isNotEmpty(uuidsMetric)) {
-          sb.append(" AND ");
-        }
-
-        // ( for OR
-        if (!metricNamesNotCondition && uuidsMetric.size() > 1 && (CollectionUtils.isNotEmpty(uuidsFull) || CollectionUtils.isNotEmpty(uuidsHost))) {
-          sb.append("(");
-        }
-
-        //LIKE clause for clusterMetric UUIDs
-        // UUID (NOT) LIKE ? OR(AND) UUID LIKE ?
-        if (CollectionUtils.isNotEmpty(uuidsMetric)) {
-
-          for (int i = 0; i < uuidsMetric.size(); i++) {
-            sb.append("UUID");
-            if (metricNamesNotCondition) {
-              sb.append(" NOT");
-            }
-            sb.append(" LIKE ");
-            sb.append("?");
-
-            if (i < uuidsMetric.size() - 1) {
-              if (metricNamesNotCondition) {
-                sb.append(" AND ");
-              } else {
-                sb.append(" OR ");
-              }
-            // ) for OR
-            } else if ((CollectionUtils.isNotEmpty(uuidsFull) || CollectionUtils.isNotEmpty(uuidsHost)) && !metricNamesNotCondition && uuidsMetric.size() > 1) {
-              sb.append(")");
-            }
-          }
-          appendConjunction = true;
+      // Put a '(' first
+      sb.append("(");
+
+      //IN clause
+      // UUID (NOT) IN (?,?,?,?)
+      if (CollectionUtils.isNotEmpty(uuids)) {
+        sb.append("UUID");
+        if (metricNamesNotCondition) {
+          sb.append(" NOT");
         }
-
-        //Put an AND if both types are present
-        if ((CollectionUtils.isNotEmpty(uuidsMetric) || (CollectionUtils.isNotEmpty(uuidsFull) && CollectionUtils.isEmpty(uuidsMetric)))
-          && CollectionUtils.isNotEmpty(uuidsHost)) {
-          sb.append(" AND ");
-        }
-        // ( for OR
-        if((CollectionUtils.isNotEmpty(uuidsFull) || CollectionUtils.isNotEmpty(uuidsMetric)) && !hostNamesNotCondition && uuidsHost.size() > 1){
-          sb.append("(");
-        }
-
-        //LIKE clause for HOST UUIDs
-        // UUID (NOT) LIKE ? OR(AND) UUID LIKE ?
-        if (CollectionUtils.isNotEmpty(uuidsHost)) {
-
-          for (int i = 0; i < uuidsHost.size(); i++) {
-            sb.append("UUID");
-            if (hostNamesNotCondition) {
-              sb.append(" NOT");
-            }
-            sb.append(" LIKE ");
-            sb.append("?");
-
-            if (i < uuidsHost.size() - 1) {
-              if (hostNamesNotCondition) {
-                sb.append(" AND ");
-              } else {
-                sb.append(" OR ");
-              }
-            // ) for OR
-            } else if ((CollectionUtils.isNotEmpty(uuidsFull) || CollectionUtils.isNotEmpty(uuidsMetric)) && !hostNamesNotCondition && uuidsHost.size() > 1) {
-              sb.append(")");
-            }
+        sb.append(" IN (");
+        //Append ?,?,?,?
+        for (int i = 0; i < uuids.size(); i++) {
+          sb.append("?");
+          if (i < uuids.size() - 1) {
+            sb.append(", ");
           }
-          appendConjunction = true;
-        }
-
-        // Finish with a ')'
-        if (appendConjunction) {
-          sb.append(")");
-        }
-
-        uuids = new ArrayList<>();
-        if (CollectionUtils.isNotEmpty(uuidsFull)) {
-          uuids.addAll(uuidsFull);
-        }
-        for (byte[] uuid: uuidsMetric) {
-          uuids.add(new String(uuid).concat("%").getBytes());
-        }
-        for (byte[] uuid: uuidsHost) {
-          uuids.add("%".concat(new String(uuid)).getBytes());
         }
+        sb.append(")");
       }
+      appendConjunction = true;
+      sb.append(")");
     }
 
     return appendConjunction;
@@ -418,4 +314,9 @@ public class DefaultCondition implements Condition {
   public List<byte[]> getUuids() {
     return uuids;
   }
+
+  @Override
+  public List<String> getTransientMetricNames() {
+    return Collections.EMPTY_LIST;
+  }
 }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/EmptyCondition.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/EmptyCondition.java
index 742b09b..ed53175 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/EmptyCondition.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/EmptyCondition.java
@@ -46,6 +46,11 @@ public class EmptyCondition implements Condition {
   }
 
   @Override
+  public List<String> getTransientMetricNames() {
+    return null;
+  }
+
+  @Override
   public boolean isPointInTime() {
     return false;
   }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/PhoenixTransactSQL.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/PhoenixTransactSQL.java
index 819a20e..eda6842 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/PhoenixTransactSQL.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/PhoenixTransactSQL.java
@@ -24,12 +24,14 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.metrics2.sink.timeline.Precision;
 import org.apache.hadoop.metrics2.sink.timeline.PrecisionLimitExceededException;
 import org.apache.ambari.metrics.core.timeline.PhoenixHBaseAccessor;
-import org.apache.ambari.metrics.core.timeline.discovery.TimelineMetricMetadataManager;
+
+import static org.apache.ambari.metrics.core.timeline.PhoenixHBaseAccessor.RESULTSET_LIMIT;
 
 /**
  * Encapsulate all metrics related SQL queries.
@@ -42,7 +44,7 @@ public class PhoenixTransactSQL {
    * Create table to store individual metric records.
    */
   public static final String CREATE_METRICS_TABLE_SQL = "CREATE TABLE IF NOT " +
-    "EXISTS METRIC_RECORD (UUID BINARY(20) NOT NULL, " +
+    "EXISTS METRIC_RECORD (UUID BINARY(32) NOT NULL, " +
     "SERVER_TIME BIGINT NOT NULL, " +
     "METRIC_SUM DOUBLE, " +
     "METRIC_COUNT UNSIGNED_INT, " +
@@ -81,7 +83,7 @@ public class PhoenixTransactSQL {
 
   public static final String CREATE_METRICS_AGGREGATE_TABLE_SQL =
     "CREATE TABLE IF NOT EXISTS %s " +
-      "(UUID BINARY(20) NOT NULL, " +
+      "(UUID BINARY(32) NOT NULL, " +
       "SERVER_TIME BIGINT NOT NULL, " +
       "METRIC_SUM DOUBLE," +
       "METRIC_COUNT UNSIGNED_INT, " +
@@ -113,6 +115,30 @@ public class PhoenixTransactSQL {
       "CONSTRAINT pk PRIMARY KEY (UUID, SERVER_TIME ROW_TIMESTAMP)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
       "TTL=%s, COMPRESSION='%s'";
 
+  /**
+   * Create table to store individual metric records.
+   */
+  public static final String METRIC_TRANSIENT_TABLE_NAME =
+    "METRIC_TRANSIENT";
+
+  public static final String CREATE_TRANSIENT_METRICS_TABLE_SQL = "CREATE TABLE IF NOT " +
+    "EXISTS " + METRIC_TRANSIENT_TABLE_NAME + " " +
+    "(METRIC_NAME VARCHAR, " +
+    "HOSTNAME VARCHAR, " +
+    "SERVER_TIME BIGINT NOT NULL, " +
+    "APP_ID VARCHAR, " +
+    "INSTANCE_ID VARCHAR, " +
+    "UNITS CHAR(20), " +
+    "METRIC_SUM DOUBLE, " +
+    "METRIC_COUNT UNSIGNED_INT, " +
+    "METRIC_MAX DOUBLE, " +
+    "METRIC_MIN DOUBLE, " +
+    "METRICS VARCHAR CONSTRAINT pk " +
+    "PRIMARY KEY (METRIC_NAME, HOSTNAME, SERVER_TIME ROW_TIMESTAMP, APP_ID, " +
+    "INSTANCE_ID)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
+    "TTL=%s, COMPRESSION='%s'";
+
+  //////// METATDATA TABLES ////////
   public static final String CREATE_METRICS_METADATA_TABLE_SQL =
     "CREATE TABLE IF NOT EXISTS METRICS_METADATA " +
       "(METRIC_NAME VARCHAR, " +
@@ -129,7 +155,7 @@ public class PhoenixTransactSQL {
 
   public static final String CREATE_HOSTED_APPS_METADATA_TABLE_SQL =
     "CREATE TABLE IF NOT EXISTS HOSTED_APPS_METADATA " +
-      "(HOSTNAME VARCHAR, UUID BINARY(4), APP_IDS VARCHAR, " +
+      "(HOSTNAME VARCHAR, UUID BINARY(16), APP_IDS VARCHAR, " +
       "CONSTRAINT pk PRIMARY KEY (HOSTNAME))" +
       "DATA_BLOCK_ENCODING='%s', COMPRESSION='%s'";
 
@@ -142,6 +168,8 @@ public class PhoenixTransactSQL {
   public static final String ALTER_METRICS_METADATA_TABLE =
     "ALTER TABLE METRICS_METADATA ADD IF NOT EXISTS IS_WHITELISTED BOOLEAN";
 
+  ////////////////////////////////
+
   /**
    * ALTER table to set new options
    */
@@ -211,6 +239,19 @@ public class PhoenixTransactSQL {
     "METRIC_COUNT) " +
     "VALUES (?, ?, ?, ?, ?, ?)";
 
+  /**
+   * Insert into transient metric table.
+   */
+  public static final String UPSERT_TRANSIENT_METRICS_SQL = "UPSERT INTO %s " +
+    "(METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID, SERVER_TIME, " +
+    "UNITS, " +
+    "METRIC_SUM, " +
+    "METRIC_MAX, " +
+    "METRIC_MIN, " +
+    "METRIC_COUNT, " +
+    "METRICS) VALUES " +
+    "(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+
   public static final String UPSERT_METADATA_SQL =
     "UPSERT INTO METRICS_METADATA (METRIC_NAME, APP_ID, INSTANCE_ID, UUID, UNITS, TYPE, " +
       "START_TIME, SUPPORTS_AGGREGATION, IS_WHITELISTED) " +
@@ -281,6 +322,43 @@ public class PhoenixTransactSQL {
     "METRIC_MIN " +
     "FROM %s";
 
+  /**
+   * Retrieve a set of rows from metrics records table.
+   */
+  public static final String GET_TRANSIENT_METRIC_SQL = "SELECT METRIC_NAME, " +
+    "HOSTNAME, APP_ID, INSTANCE_ID, SERVER_TIME, UNITS, " +
+    "METRIC_SUM, " +
+    "METRIC_MAX, " +
+    "METRIC_MIN, " +
+    "METRIC_COUNT, " +
+    "METRICS " +
+    "FROM %s";
+
+  /**
+   * Get latest transient metrics for a number of hosts
+   */
+  public static final String GET_LATEST_TRANSIENT_METRIC_SQL = "SELECT %s " +
+    "E.METRIC_NAME AS METRIC_NAME, E.HOSTNAME AS HOSTNAME, " +
+    "E.APP_ID AS APP_ID, E.INSTANCE_ID AS INSTANCE_ID, " +
+    "E.SERVER_TIME AS SERVER_TIME, E.START_TIME AS START_TIME, " +
+    "E.UNITS AS UNITS, E.METRIC_SUM AS METRIC_SUM, " +
+    "E.METRIC_MAX AS METRIC_MAX, E.METRIC_MIN AS METRIC_MIN, " +
+    "E.METRIC_COUNT AS METRIC_COUNT, E.METRICS AS METRICS " +
+    "FROM %s AS E " +
+    "INNER JOIN " +
+    "(SELECT METRIC_NAME, HOSTNAME, MAX(SERVER_TIME) AS MAX_SERVER_TIME, " +
+    "APP_ID, INSTANCE_ID " +
+    "FROM %s " +
+    "WHERE " +
+    "%s " +
+    "GROUP BY METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID) " +
+    "AS I " +
+    "ON E.METRIC_NAME=I.METRIC_NAME " +
+    "AND E.HOSTNAME=I.HOSTNAME " +
+    "AND E.SERVER_TIME=I.MAX_SERVER_TIME " +
+    "AND E.APP_ID=I.APP_ID " +
+    "AND E.INSTANCE_ID=I.INSTANCE_ID";
+
   public static final String TOP_N_INNER_SQL = "SELECT UUID " +
     "FROM %s WHERE %s GROUP BY UUID ORDER BY %s LIMIT %s";
 
@@ -384,16 +462,6 @@ public class PhoenixTransactSQL {
   private static boolean sortMergeJoinEnabled = false;
 
   /**
-   * Filter to optimize HBase scan by using file timestamps. This prevents
-   * a full table scan of metric records.
-   *
-   * @return Phoenix Hint String
-   */
-  public static String getNaiveTimeRangeHint(Long startTime, Long delta) {
-    return String.format("/*+ NATIVE_TIME_RANGE(%s) */", (startTime - delta));
-  }
-
-  /**
    * Falling back to sort merge join algorithm if default queries fail.
    *
    * @return Phoenix Hint String
@@ -422,10 +490,7 @@ public class PhoenixTransactSQL {
       String metricsTable;
       String query;
       if (condition.getPrecision() == null) {
-        long endTime = condition.getEndTime() == null ? System.currentTimeMillis() : condition.getEndTime();
-        long startTime = condition.getStartTime() == null ? 0 : condition.getStartTime();
-        Precision precision = Precision.getPrecision(startTime, endTime);
-        condition.setPrecision(precision);
+        condition.setPrecision(getBestPrecisionForCondition(condition));
       }
       switch (condition.getPrecision()) {
         case DAYS:
@@ -508,16 +573,35 @@ public class PhoenixTransactSQL {
       //aggregator can use empty metrics query
       return;
     }
-
     long range = condition.getEndTime() - condition.getStartTime();
-    long rowsPerMetric;
-
-    //Get Precision (passed in or computed) and estimate values returned based on that.
+    List<String> hostNames = condition.getHostnames();
+    int numHosts = (hostNames == null || hostNames.isEmpty()) ? 1 : condition.getHostnames().size();
     Precision precision = condition.getPrecision();
-    if (precision == null) {
-      precision = Precision.getPrecision(condition.getStartTime(), condition.getEndTime());
+
+    //Validate only if precision is passed in.
+    if (precision != null) {
+      long rowsPerMetric = getRowCountForPrecision(precision, range, CollectionUtils.isNotEmpty(hostNames));
+      long totalRowsRequested = rowsPerMetric * condition.getMetricNames().size() * numHosts;
+      if (totalRowsRequested > RESULTSET_LIMIT) {
+        throw new PrecisionLimitExceededException("Requested " + condition.getMetricNames().size() + " metrics for "
+          + numHosts + " hosts in " + precision + " precision for the time range of " + range / 1000
+          + " seconds. Estimated resultset size of " + totalRowsRequested + " is greater than the limit of "
+          + RESULTSET_LIMIT + ". Request lower precision or fewer number of metrics or hosts." +
+          " Alternatively, increase the limit value through ams-site:timeline.metrics.service.default.result.limit config");
+      }
     }
+  }
 
+  /**
+   * Given a precision and condition, calculate the max number of rows that may be returned from phoenix using actual
+   * aggregator intervals for determining data granularity in tables.
+   * @param precision
+   * @param range
+   * @param withHosts
+   * @return
+   */
+  private static long getRowCountForPrecision(Precision precision, long range, boolean withHosts) {
+    long rowsPerMetric;
     switch (precision) {
       case DAYS:
         rowsPerMetric = TimeUnit.MILLISECONDS.toDays(range);
@@ -526,63 +610,51 @@ public class PhoenixTransactSQL {
         rowsPerMetric = TimeUnit.MILLISECONDS.toHours(range);
         break;
       case MINUTES:
-        rowsPerMetric = TimeUnit.MILLISECONDS.toMinutes(range)/5; //5 minute data in METRIC_AGGREGATE_MINUTE table.
+        int minuteInterval = PhoenixHBaseAccessor.hostMinuteAggregatorDataInterval;
+        if (!withHosts) {
+          minuteInterval = PhoenixHBaseAccessor.clusterMinuteAggregatorDataInterval;
+        }
+        rowsPerMetric = TimeUnit.MILLISECONDS.toMinutes(range) / TimeUnit.SECONDS.toMinutes(minuteInterval);
         break;
       default:
-        rowsPerMetric = TimeUnit.MILLISECONDS.toSeconds(range)/10; //10 second data in METRIC_AGGREGATE table
+        int secondAggregatorInterval = PhoenixHBaseAccessor.clusterSecondAggregatorDataInterval;
+        if (withHosts) {
+          secondAggregatorInterval = 60;
+        }
+        rowsPerMetric = TimeUnit.MILLISECONDS.toSeconds(range) / secondAggregatorInterval;
     }
+    return rowsPerMetric;
+  }
 
+  /**
+   * Given a condition, find the highest precision that satisfies the limit condition.
+   * @param condition
+   * @return
+   */
+  private static Precision getBestPrecisionForCondition(Condition condition) {
+
+    List<String> metricNames = condition.getMetricNames();
     List<String> hostNames = condition.getHostnames();
+    long range = condition.getEndTime() - condition.getStartTime();
     int numHosts = (hostNames == null || hostNames.isEmpty()) ? 1 : condition.getHostnames().size();
 
-    long totalRowsRequested = rowsPerMetric * condition.getMetricNames().size() * numHosts;
-
-    if (totalRowsRequested > PhoenixHBaseAccessor.RESULTSET_LIMIT) {
-      throw new PrecisionLimitExceededException("Requested " +  condition.getMetricNames().size() + " metrics for "
-        + numHosts + " hosts in " + precision +  " precision for the time range of " + range/1000
-        + " seconds. Estimated resultset size of " + totalRowsRequested + " is greater than the limit of "
-        + PhoenixHBaseAccessor.RESULTSET_LIMIT + ". Request lower precision or fewer number of metrics or hosts." +
-        " Alternatively, increase the limit value through ams-site:timeline.metrics.service.default.result.limit config");
+    Precision[] precisions = {Precision.SECONDS, Precision.MINUTES, Precision.HOURS, Precision.DAYS};
+    int i = 0;
+    for (; i < precisions.length; i++) {
+      long rowsPerMetric = getRowCountForPrecision(precisions[i], range, CollectionUtils.isNotEmpty(hostNames));
+      if ((rowsPerMetric * metricNames.size() * numHosts) <= PhoenixHBaseAccessor.RESULTSET_LIMIT) {
+        break;
+      }
+    }
+    if (i >= precisions.length) {
+      return Precision.DAYS;
     }
+    return precisions[i];
   }
 
   public static PreparedStatement prepareGetLatestMetricSqlStmt(
     Connection connection, Condition condition) throws SQLException {
-
-    validateConditionIsNotEmpty(condition);
-
-    if (condition.getMetricNames() == null
-      || condition.getMetricNames().isEmpty()) {
-      throw new IllegalArgumentException("Point in time query without "
-        + "metric names not supported ");
-    }
-
-    String stmtStr;
-    if (condition.getStatement() != null) {
-      stmtStr = condition.getStatement();
-    } else {
-      stmtStr = String.format(GET_LATEST_METRIC_SQL,
-        getLatestMetricsHints(),
-        METRICS_RECORD_TABLE_NAME,
-        METRICS_RECORD_TABLE_NAME,
-        condition.getConditionClause());
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("SQL: " + stmtStr + ", condition: " + condition);
-    }
-    PreparedStatement stmt = null;
-    try {
-      stmt = connection.prepareStatement(stmtStr);
-      setQueryParameters(stmt, condition);
-    } catch (SQLException e) {
-      if (stmt != null) {
-        stmt.close();
-      }
-      throw e;
-    }
-
-    return stmt;
+    return prepareGetLatestMetricSqlStmtHelper(connection, condition, GET_LATEST_METRIC_SQL, METRICS_RECORD_TABLE_NAME);
   }
 
   private static PreparedStatement setQueryParameters(PreparedStatement stmt,
@@ -613,9 +685,7 @@ public class PhoenixTransactSQL {
     String metricsAggregateTable;
     String queryStmt;
     if (condition.getPrecision() == null) {
-      long endTime = condition.getEndTime() == null ? System.currentTimeMillis() : condition.getEndTime();
-      long startTime = condition.getStartTime() == null ? 0 : condition.getStartTime();
-      condition.setPrecision(Precision.getPrecision(startTime, endTime));
+      condition.setPrecision(getBestPrecisionForCondition(condition));
     }
     switch (condition.getPrecision()) {
       case DAYS:
@@ -724,6 +794,126 @@ public class PhoenixTransactSQL {
     return stmt;
   }
 
+  /**
+   *
+   * @param connection
+   * @param c
+   * @return
+   * @throws SQLException
+   */
+  public static PreparedStatement prepareTransientMetricsSqlStmt(Connection connection, Condition c)
+    throws SQLException {
+    validateConditionIsNotEmpty(c);
+    validateRowCountLimit(c);
+
+    if (!(c instanceof TransientMetricCondition)) {
+      LOG.error("Condition not instanceOf TransientMetricCondition");
+      return null;
+    }
+
+    TransientMetricCondition condition = (TransientMetricCondition) c;
+
+    String stmtStr;
+    if (condition.getStatement() != null) {
+      stmtStr = condition.getStatement();
+    } else {
+      stmtStr = String.format(GET_TRANSIENT_METRIC_SQL, METRIC_TRANSIENT_TABLE_NAME);
+    }
+
+    StringBuilder sb = new StringBuilder(stmtStr);
+
+    sb.append(" WHERE ");
+    sb.append(condition.getTransientConditionClause());
+    String orderByClause = condition.getOrderByClause(true);
+    if (orderByClause != null) {
+      sb.append(orderByClause);
+    } else {
+      sb.append(" ORDER BY METRIC_NAME, SERVER_TIME ");
+    }
+
+    if (condition.getLimit() != null) {
+      sb.append(" LIMIT ").append(condition.getLimit());
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("SQL: " + sb.toString() + ", condition: " + condition);
+    }
+
+    PreparedStatement stmt = null;
+    try {
+      stmt = connection.prepareStatement(sb.toString());
+      int pos = 1;
+      pos = addMetricNames(condition, pos, stmt);
+      pos = addHostNames(condition, pos, stmt);
+      pos = addAppId(condition, pos, stmt);
+      pos = addInstanceId(condition, pos, stmt);
+      pos = addStartTime(condition, pos, stmt);
+      addEndTime(condition, pos, stmt);
+
+      if (condition.getFetchSize() != null) {
+        stmt.setFetchSize(condition.getFetchSize());
+      }
+    } catch (SQLException e) {
+      if (stmt != null) {
+        stmt.close();
+      }
+      throw e;
+    }
+
+    return stmt;
+
+  }
+
+  /**
+   *
+   * @param connection
+   * @param condition
+   * @param sqlStmt
+   * @param tableName
+   * @return
+   * @throws SQLException
+   */
+  private static PreparedStatement prepareGetLatestMetricSqlStmtHelper(Connection connection,
+                                                                       Condition condition,
+                                                                       String sqlStmt,
+                                                                       String tableName) throws SQLException {
+    validateConditionIsNotEmpty(condition);
+
+    if (condition.getMetricNames() == null
+      || condition.getMetricNames().isEmpty()) {
+      throw new IllegalArgumentException("Point in time query without "
+        + "metric names not supported ");
+    }
+
+    String stmtStr;
+    if (condition.getStatement() != null) {
+      stmtStr = condition.getStatement();
+    } else {
+      stmtStr = String.format(sqlStmt,
+        getLatestMetricsHints(),
+        tableName,
+        tableName,
+        condition.getConditionClause());
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("SQL: " + stmtStr + ", condition: " + condition);
+    }
+    PreparedStatement stmt = null;
+    try {
+      stmt = connection.prepareStatement(stmtStr);
+      setQueryParameters(stmt, condition);
+    } catch (SQLException e) {
+      if (stmt != null) {
+        stmt.close();
+      }
+      throw e;
+    }
+
+    return stmt;
+
+  }
+
   public static String getTargetTableUsingPrecision(Precision precision, boolean withHosts) {
 
     String inputTable = null;
@@ -770,17 +960,58 @@ public class PhoenixTransactSQL {
   private static int addUuids(Condition condition, int pos, PreparedStatement stmt) throws SQLException {
     if (condition.getUuids() != null) {
       for (int pos2 = 1 ; pos2 <= condition.getUuids().size(); pos2++,pos++) {
-        byte[] uuid = condition.getUuids().get(pos2 - 1);
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Setting pos: " + pos + ", value = " + new String(uuid));
+          LOG.debug("Setting pos: " + pos + ", value = " + condition.getUuids().get(pos2 - 1));
         }
+        stmt.setBytes(pos, condition.getUuids().get(pos2 - 1));
+      }
+    }
+    return pos;
+  }
 
-        if (uuid.length != TimelineMetricMetadataManager.HOSTNAME_UUID_LENGTH + TimelineMetricMetadataManager.TIMELINE_METRIC_UUID_LENGTH) {
-          stmt.setString(pos, new String(uuid));
-        } else {
-          stmt.setBytes(pos, uuid);
+  private static int addMetricNames(Condition condition, int pos, PreparedStatement stmt) throws SQLException {
+    if (condition.getMetricNames() != null) {
+      for (int pos2 = 1 ; pos2 <= condition.getMetricNames().size(); pos2++,pos++) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Setting pos: " + pos + ", value = " + condition.getMetricNames().get(pos2 - 1));
+        }
+        stmt.setString(pos, condition.getMetricNames().get(pos2 - 1));
+      }
+    }
+    return pos;
+  }
+
+  private static int addHostNames(Condition condition, int pos, PreparedStatement stmt) throws SQLException {
+    int i = pos;
+    if (condition.getHostnames() != null) {
+      for (String hostname : condition.getHostnames()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Setting pos: " + pos + ", value: " + hostname);
         }
+        stmt.setString(i++, hostname);
+      }
+    }
+    return i;
+  }
+
+  private static int addAppId(Condition condition, int pos, PreparedStatement stmt) throws SQLException {
+
+    if (condition.getAppId() != null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Setting pos: " + pos + ", value: " + condition.getAppId());
+      }
+      stmt.setString(pos++, condition.getAppId());
+    }
+    return pos;
+  }
+
+  private static int addInstanceId(Condition condition, int pos, PreparedStatement stmt) throws SQLException {
+
+    if (condition.getInstanceId() != null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Setting pos: " + pos + ", value: " + condition.getInstanceId());
       }
+      stmt.setString(pos++, condition.getInstanceId());
     }
     return pos;
   }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/SplitByMetricNamesCondition.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/SplitByMetricNamesCondition.java
index 2f1697f..21274fd 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/SplitByMetricNamesCondition.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/SplitByMetricNamesCondition.java
@@ -47,6 +47,11 @@ public class SplitByMetricNamesCondition implements Condition {
   }
 
   @Override
+  public List<String> getTransientMetricNames() {
+    return null;
+  }
+
+  @Override
   public boolean isPointInTime() {
     return adaptee.isPointInTime();
   }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/TransientMetricCondition.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/TransientMetricCondition.java
new file mode 100644
index 0000000..c26d18d
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/TransientMetricCondition.java
@@ -0,0 +1,206 @@
+/**
+ * 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.ambari.metrics.core.timeline.query;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.metrics2.sink.timeline.Precision;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Class to encapsulate condition to query transient metrics.
+ */
+public class TransientMetricCondition extends DefaultCondition {
+
+  private List<String> transientMetricNames = new ArrayList<>();
+
+  public TransientMetricCondition(List<byte[]> uuids, List<String> metricNames, List<String> hostnames, String appId,
+                          String instanceId, Long startTime, Long endTime, Precision precision,
+                          Integer limit, boolean grouped, List<String> transientMetricNames) {
+    super(uuids, metricNames, hostnames, appId, instanceId, startTime, endTime, precision, limit, grouped);
+    this.transientMetricNames = transientMetricNames;
+    if (CollectionUtils.isEmpty(hostnames)) {
+      this.hostnames = Collections.singletonList("%");
+    }
+  }
+
+  public StringBuilder getTransientConditionClause() {
+    StringBuilder sb = new StringBuilder();
+
+    boolean appendConjunction = appendMetricNameClause(sb);
+
+    appendConjunction = appendHostnameClause(sb, appendConjunction);
+
+    appendConjunction = append(sb, appendConjunction, getAppId(), " APP_ID = ?");
+    appendConjunction = append(sb, appendConjunction, getInstanceId(), " INSTANCE_ID = ?");
+    appendConjunction = append(sb, appendConjunction, getStartTime(), " SERVER_TIME >= ?");
+    append(sb, appendConjunction, getEndTime(), " SERVER_TIME < ?");
+
+    return sb;
+  }
+
+  private boolean appendMetricNameClause(StringBuilder sb) {
+    boolean appendConjunction = false;
+    List<String> metricsLike = new ArrayList<>();
+    List<String> metricsIn = new ArrayList<>();
+
+    if (getMetricNames() != null) {
+      for (String name : getMetricNames()) {
+        if (name.contains("%")) {
+          metricsLike.add(name);
+        } else {
+          metricsIn.add(name);
+        }
+      }
+
+      // Put a '(' first
+      sb.append("(");
+
+      //IN clause
+      // METRIC_NAME (NOT) IN (?,?,?,?)
+      if (CollectionUtils.isNotEmpty(metricsIn)) {
+        sb.append("METRIC_NAME");
+        if (metricNamesNotCondition) {
+          sb.append(" NOT");
+        }
+        sb.append(" IN (");
+        //Append ?,?,?,?
+        for (int i = 0; i < metricsIn.size(); i++) {
+          sb.append("?");
+          if (i < metricsIn.size() - 1) {
+            sb.append(", ");
+          }
+        }
+        sb.append(")");
+        appendConjunction = true;
+      }
+
+      //Put an OR/AND if both types are present
+      if (CollectionUtils.isNotEmpty(metricsIn) &&
+        CollectionUtils.isNotEmpty(metricsLike)) {
+        if (metricNamesNotCondition) {
+          sb.append(" AND ");
+        } else {
+          sb.append(" OR ");
+        }
+      }
+
+      //LIKE clause
+      // METRIC_NAME (NOT) LIKE ? OR(AND) METRIC_NAME LIKE ?
+      if (CollectionUtils.isNotEmpty(metricsLike)) {
+
+        for (int i = 0; i < metricsLike.size(); i++) {
+          sb.append("METRIC_NAME");
+          if (metricNamesNotCondition) {
+            sb.append(" NOT");
+          }
+          sb.append(" LIKE ");
+          sb.append("?");
+
+          if (i < metricsLike.size() - 1) {
+            if (metricNamesNotCondition) {
+              sb.append(" AND ");
+            } else {
+              sb.append(" OR ");
+            }
+          }
+        }
+        appendConjunction = true;
+      }
+
+      // Finish with a ')'
+      if (appendConjunction) {
+        sb.append(")");
+      }
+
+      metricNames.clear();
+      if (CollectionUtils.isNotEmpty(metricsIn)) {
+        metricNames.addAll(metricsIn);
+      }
+      if (CollectionUtils.isNotEmpty(metricsLike)) {
+        metricNames.addAll(metricsLike);
+      }
+    }
+    return appendConjunction;
+  }
+
+  protected boolean appendHostnameClause(StringBuilder sb, boolean appendConjunction) {
+    boolean hostnameContainsRegex = false;
+    if (hostnames != null) {
+      for (String hostname : hostnames) {
+        if (hostname.contains("%")) {
+          hostnameContainsRegex = true;
+          break;
+        }
+      }
+    }
+
+    StringBuilder hostnamesCondition = new StringBuilder();
+    if (hostnameContainsRegex) {
+      hostnamesCondition.append(" (");
+      for (String hostname : getHostnames()) {
+        if (hostnamesCondition.length() > 2) {
+          hostnamesCondition.append(" OR ");
+        }
+        hostnamesCondition.append("HOSTNAME LIKE ?");
+      }
+      hostnamesCondition.append(")");
+
+      appendConjunction = append(sb, appendConjunction, getHostnames(), hostnamesCondition.toString());
+    } else if (CollectionUtils.isNotEmpty(hostnames)) {
+      for (String hostname : getHostnames()) {
+        if (hostnamesCondition.length() > 0) {
+          hostnamesCondition.append(" ,");
+        } else {
+          hostnamesCondition.append(" HOSTNAME IN (");
+        }
+        hostnamesCondition.append('?');
+      }
+      hostnamesCondition.append(')');
+      appendConjunction = append(sb, appendConjunction, getHostnames(), hostnamesCondition.toString());
+
+    } else {
+      appendConjunction = append(sb, appendConjunction, getHostnames(), " HOSTNAME = ?");
+    }
+    return appendConjunction;
+  }
+
+  @Override
+  public String toString() {
+    return "Condition{" +
+      "metricNames=" + metricNames +
+      ", hostnames='" + hostnames + '\'' +
+      ", appId='" + appId + '\'' +
+      ", instanceId='" + instanceId + '\'' +
+      ", startTime=" + startTime +
+      ", endTime=" + endTime +
+      ", limit=" + limit +
+      ", grouped=" + grouped +
+      ", orderBy=" + orderByColumns +
+      ", noLimit=" + noLimit +
+      '}';
+  }
+
+  @Override
+  public List<String> getTransientMetricNames() {
+    return transientMetricNames;
+  }
+}
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/HashBasedUuidGenStrategy.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/HashBasedUuidGenStrategy.java
index 0d87c55..fdfeb79 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/HashBasedUuidGenStrategy.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/HashBasedUuidGenStrategy.java
@@ -179,11 +179,15 @@ public class HashBasedUuidGenStrategy implements MetricUuidGenStrategy {
       return null;
     }
 
-    int customAsciiSum = 1489 + (int) computeWeightedNumericalAsciiSum(value); //seed = 1489
+    int customAsciiSum = 27177289 + (int) computeWeightedNumericalAsciiSum(value); //seed = 1489
 
-    String customAsciiSumStr = String.valueOf(customAsciiSum);
+    StringBuilder customAsciiSumStr = new StringBuilder(String.valueOf(customAsciiSum));
     if (customAsciiSumStr.length() < maxLength) {
-      return null;
+      String originalcustomAsciiSumStr = customAsciiSumStr.toString();
+      while (customAsciiSumStr.length() < maxLength) {
+        customAsciiSumStr.append(originalcustomAsciiSumStr);
+      }
+      return customAsciiSumStr.substring(0, maxLength).getBytes();
     } else {
       return customAsciiSumStr.substring(customAsciiSumStr.length() - maxLength, customAsciiSumStr.length()).getBytes();
     }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/RandomUuidGenStrategy.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/MD5UuidGenStrategy.java
similarity index 52%
rename from ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/RandomUuidGenStrategy.java
rename to ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/MD5UuidGenStrategy.java
index 613b566..baa2dce 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/RandomUuidGenStrategy.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/MD5UuidGenStrategy.java
@@ -18,36 +18,43 @@
 
 package org.apache.ambari.metrics.core.timeline.uuid;
 
-import java.security.SecureRandom;
+import java.nio.ByteBuffer;
+import java.util.UUID;
 
 import org.apache.ambari.metrics.core.timeline.aggregators.TimelineClusterMetric;
+import org.apache.commons.lang.StringUtils;
 
-import com.google.common.primitives.Longs;
+public class MD5UuidGenStrategy implements MetricUuidGenStrategy {
 
-public class RandomUuidGenStrategy implements MetricUuidGenStrategy {
-  private static SecureRandom randomGenerator;
-
-  public RandomUuidGenStrategy() {
-    randomGenerator = new SecureRandom(
-      Longs.toByteArray(System.currentTimeMillis()));
+  public MD5UuidGenStrategy() {
   }
 
   @Override
   public byte[] computeUuid(TimelineClusterMetric timelineClusterMetric, int maxLength) {
-    final byte[] bytes = new byte[maxLength];
-    randomGenerator.nextBytes(bytes);
-    return bytes;
-  }
 
-//  @Override
-//  public byte[] computeUuid(TimelineMetric timelineMetric, int maxLength) {
-//    return new byte[10];
-//  }
+    String metricString = timelineClusterMetric.getMetricName() + timelineClusterMetric.getAppId();
+    if (StringUtils.isNotEmpty(timelineClusterMetric.getInstanceId())) {
+      metricString += timelineClusterMetric.getInstanceId();
+    }
+    byte[] metricBytes = metricString.getBytes();
+
+    UUID uuid = UUID.nameUUIDFromBytes(metricBytes);
+    ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[maxLength]);
+    byteBuffer.putLong(uuid.getMostSignificantBits());
+    byteBuffer.putLong(uuid.getLeastSignificantBits());
+
+    return byteBuffer.array();
+  }
 
   @Override
   public byte[] computeUuid(String value, int maxLength) {
-    final byte[] bytes = new byte[maxLength];
-    randomGenerator.nextBytes(bytes);
-    return bytes;
+
+    byte[] valueBytes = value.getBytes();
+    UUID uuid = UUID.nameUUIDFromBytes(valueBytes);
+    ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[maxLength]);
+    byteBuffer.putLong(uuid.getMostSignificantBits());
+    byteBuffer.putLong(uuid.getLeastSignificantBits());
+
+    return byteBuffer.array();
   }
 }
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/MetricUuidGenStrategy.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/MetricUuidGenStrategy.java
index 5464d10..24af7c3 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/MetricUuidGenStrategy.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/uuid/MetricUuidGenStrategy.java
@@ -23,14 +23,6 @@ public interface MetricUuidGenStrategy {
 
   /**
    * Compute UUID for a given value
-   * @param timelineMetric instance
-   * @param maxLength
-   * @return
-   */
-//  byte[] computeUuid(TimelineMetric timelineMetric, int maxLength);
-
-  /**
-   * Compute UUID for a given value
    * @param value
    * @param maxLength
    * @return
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/webapp/TimelineWebServices.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/webapp/TimelineWebServices.java
index a36b038..9c88b1a 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/webapp/TimelineWebServices.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/webapp/TimelineWebServices.java
@@ -47,6 +47,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.ambari.metrics.core.timeline.TimelineMetricServiceSummary;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -281,6 +282,21 @@ public class TimelineWebServices {
   }
 
   @GET
+  @Path("/metrics/summary")
+  @Produces({ MediaType.APPLICATION_JSON })
+  public TimelineMetricServiceSummary getTimelineMetricServiceSummary(
+    @Context HttpServletRequest req,
+    @Context HttpServletResponse res) {
+    init(res);
+
+    try {
+      return timelineMetricStore.getTimelineMetricServiceSummary();
+    } catch (Exception e) {
+      throw new WebApplicationException(e, Response.Status.INTERNAL_SERVER_ERROR);
+    }
+  }
+
+  @GET
   @Path("/metrics/metadata")
   @Produces({ MediaType.APPLICATION_JSON })
   public Map<String, List<TimelineMetricMetadata>> getTimelineMetricMetadata(
@@ -349,6 +365,10 @@ public class TimelineWebServices {
     ) {
     init(res);
 
+    if (metricName == null || appId == null) {
+      throw new WebApplicationException(new IllegalArgumentException("Non null values needed for metricName and appId")
+        , Response.Status.BAD_REQUEST);
+    }
     try {
       return timelineMetricStore.getUuid(metricName, appId, instanceId, hostname);
     } catch (Exception e) {
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/resources/metrics_def/KAFKA_BROKER.dat b/ambari-metrics/ambari-metrics-timelineservice/src/main/resources/metrics_def/KAFKA_BROKER.dat
index b9848d7..455fa01 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/resources/metrics_def/KAFKA_BROKER.dat
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/resources/metrics_def/KAFKA_BROKER.dat
@@ -1101,4 +1101,18273 @@ kafka.network.RequestMetrics.RequestsPerSec.request.Heartbeat.count
 kafka.network.RequestMetrics.RequestsPerSec.request.Heartbeat.1MinuteRate
 kafka.network.RequestMetrics.RequestsPerSec.request.Heartbeat.meanRate
 kafka.network.RequestMetrics.RequestsPerSec.request.Heartbeat.5MinuteRate
-kafka.network.RequestMetrics.RequestsPerSec.request.Heartbeat.5MinuteRate
\ No newline at end of file
+kafka.network.RequestMetrics.RequestsPerSec.request.Heartbeat.5MinuteRatetopology.stockton-storm-1.kafka-spout.g1c1n1.pc01.cls04.6701.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c4n1.pc01.cls04.6713.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c5n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c6n3.pc01.cls04.6702.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c2n1.pc01.cls04.6708.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c2n2.pc01.cls04.6704.-1.--complete-latency.default
+topology.orlando-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c3n2.pc01.cls04.6708.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g1c3n1.pc01.cls04.6726.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c7n3.pc01.cls04.6720.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c7n2.pc01.cls04.6707.-1.--emit-count.--system
+topology.harrison-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c2n4.pc01.cls04.6700.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c4n2.pc01.cls04.6708.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c5n2.pc01.cls04.6707.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n2.pc01.cls04.6707.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c4n2.pc01.cls04.6710.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c2n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c6n3.pc01.cls04.6710.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c2n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c3n4.pc01.cls04.6711.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c6n4.pc01.cls04.6721.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c7n4.pc01.cls04.6714.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g1c4n2.pc01.cls04.6714.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c4n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g2c3n1.pc01.cls04.6720.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6723.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6703.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c4n3.pc01.cls04.6720.-1.--emit-count.--system
+topology.elkridge-storm-1.kafka-spout.g1c5n3.pc01.cls04.6712.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g1c3n1.pc01.cls04.6715.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c6n1.pc01.cls04.6723.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-spout.g1c7n1.pc01.cls04.6712.-1.--emit-count.--system
+topology.harrison-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n1.pc01.cls04.6711.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6709.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c3n4.pc01.cls04.6706.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c5n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c4n2.pc01.cls04.6700.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c3n1.pc01.cls04.6709.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c1n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c7n4.pc01.cls04.6721.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6727.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c3n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c4n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c1n3.pc01.cls04.6727.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c7n2.pc01.cls04.6724.-1.--receive.population
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-29.spoutLag
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c7n3.pc01.cls04.6701.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-spout.g1c2n3.pc01.cls04.6711.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c5n3.pc01.cls04.6705.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c6n3.pc01.cls04.6725.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c5n2.pc01.cls04.6725.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c7n4.pc01.cls04.6726.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6708.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c6n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.omaha-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c2n3.pc01.cls04.6708.-1.--ack-count.default
+topology.tacoma-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c5n4.pc01.cls04.6723.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6712.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c6n2.pc01.cls04.6704.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c1n3.pc01.cls04.6702.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c2n2.pc01.cls04.6706.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c5n3.pc01.cls04.6720.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6704.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c5n3.pc01.cls04.6709.-1.--ack-count.default
+topology.seattle-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c6n4.pc01.cls04.6702.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g2c6n4.pc01.cls04.6724.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g1c4n4.pc01.cls04.6713.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6727.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6707.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c4n1.pc01.cls04.6723.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c7n4.pc01.cls04.6722.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c1n2.pc01.cls04.6727.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c5n1.pc01.cls04.6703.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g1c4n4.pc01.cls04.6708.-1.--sendqueue.population
+topology.omaha-storm-1.kafka-spout.g1c5n2.pc01.cls04.6704.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c2n1.pc01.cls04.6725.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6722.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c5n3.pc01.cls04.6701.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g1c3n3.pc01.cls04.6710.-1.--ack-count.default
+topology.chicago-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c7n4.pc01.cls04.6723.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c3n2.pc01.cls04.6712.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n2.pc01.cls04.6712.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-topic.phoenix-storm-1.partition-9.earliestTimeOffset
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6723.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c5n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c1n4.pc01.cls04.6708.-1.--emit-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c7n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.houston-storm-1.kafka-spout.g2c4n4.pc01.cls04.6701.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6704.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c5n4.pc01.cls04.6723.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g2c1n1.pc01.cls04.6700.-1.--emit-count.--system
+topology.seattle-storm-1.kafka-spout.g2c1n2.pc01.cls04.6703.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n4.pc01.cls04.6709.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6702.-1.--ack-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6710.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c7n3.pc01.cls04.6705.-1.--complete-latency.default
+topology.buffalo-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c5n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c6n4.pc01.cls04.6703.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c4n4.pc01.cls04.6715.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c2n2.pc01.cls04.6711.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c2n3.pc01.cls04.6715.-1.--complete-latency.default
+topology.miami-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c1n3.pc01.cls04.6723.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6709.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c7n1.pc01.cls04.6711.-1.--emit-count.--system
+topology.fort-worth-storm-1.kafka-spout.g2c4n4.pc01.cls04.6725.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g1c6n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c5n4.pc01.cls04.6710.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c4n1.pc01.cls04.6723.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c7n4.pc01.cls04.6723.-1.--receive.population
+topology.chicago-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c5n1.pc01.cls04.6713.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g2c6n4.pc01.cls04.6714.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c4n1.pc01.cls04.6723.-1.--emit-count.--system
+topology.tacoma-storm-1.kafka-spout.g2c5n2.pc01.cls04.6706.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c2n3.pc01.cls04.6712.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g2c5n1.pc01.cls04.6703.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c3n3.pc01.cls04.6705.-1.--emit-count.default
+topology.seattle-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c6n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c7n1.pc01.cls04.6723.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6710.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g1c2n1.pc01.cls04.6707.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g2c4n1.pc01.cls04.6701.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6707.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c7n3.pc01.cls04.6706.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c5n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c6n2.pc01.cls04.6708.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c4n1.pc01.cls04.6709.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c6n2.pc01.cls04.6720.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c5n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c6n2.pc01.cls04.6726.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--emit-count.--system
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c5n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g2c3n2.pc01.cls04.6710.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6721.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c7n2.pc01.cls04.6712.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c3n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c5n2.pc01.cls04.6724.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c5n4.pc01.cls04.6702.-1.--ack-count.default
+topology.tacoma-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g1c6n4.pc01.cls04.6704.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c6n4.pc01.cls04.6705.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g1c2n4.pc01.cls04.6707.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c2n3.pc01.cls04.6702.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c1n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c3n2.pc01.cls04.6712.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g2c2n4.pc01.cls04.6712.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c4n2.pc01.cls04.6727.-1.--complete-latency.default
+topology.puerto-rico-storm-1.kafka-spout.g1c3n3.pc01.cls04.6712.-1.--emit-count.--system
+topology.springfield-storm-1.kafka-spout.g2c5n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c2n4.pc01.cls04.6710.-1.--emit-count.--system
+topology.washington-dc-storm-1.kafka-spout.g1c5n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g1c4n4.pc01.cls04.6705.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6711.-1.--emit-count.default
+topology.springfield-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6714.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6726.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6715.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c5n3.pc01.cls04.6707.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c6n2.pc01.cls04.6712.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6713.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c4n2.pc01.cls04.6714.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c3n3.pc01.cls04.6710.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c4n1.pc01.cls04.6714.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6709.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.washington-dc-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c6n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c2n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c7n2.pc01.cls04.6706.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c5n1.pc01.cls04.6713.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c2n3.pc01.cls04.6702.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c6n4.pc01.cls04.6705.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6721.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c2n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.tacoma-storm-1.kafka-spout.g2c2n3.pc01.cls04.6713.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c3n1.pc01.cls04.6710.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.buffalo-storm-1.kafka-spout.g2c2n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c1n4.pc01.cls04.6712.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c7n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c6n3.pc01.cls04.6703.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c2n3.pc01.cls04.6725.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c4n1.pc01.cls04.6724.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g1c2n1.pc01.cls04.6720.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c2n1.pc01.cls04.6725.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c7n4.pc01.cls04.6703.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c4n4.pc01.cls04.6701.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c7n1.pc01.cls04.6704.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c6n4.pc01.cls04.6713.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c6n3.pc01.cls04.6709.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c2n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.cheyenne-storm-1.kafka-topic.cheyenne-storm-1.partition-4.latestCompletedOffset
+topology.fort-worth-storm-1.kafka-spout.g1c4n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.phoenix-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c5n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g1c4n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c3n4.pc01.cls04.6725.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g1c1n4.pc01.cls04.6724.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g1c2n3.pc01.cls04.6713.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g1c3n2.pc01.cls04.6708.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c1n2.pc01.cls04.6721.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c4n3.pc01.cls04.6701.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c3n2.pc01.cls04.6712.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c3n2.pc01.cls04.6700.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c6n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c3n2.pc01.cls04.6720.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c4n4.pc01.cls04.6722.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c5n2.pc01.cls04.6712.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g1c1n4.pc01.cls04.6701.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c1n2.pc01.cls04.6727.-1.--ack-count.default
+topology.tacoma-storm-1.kafka-spout.g1c3n3.pc01.cls04.6710.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c5n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c6n4.pc01.cls04.6721.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c5n2.pc01.cls04.6720.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6704.-1.--emit-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c1n3.pc01.cls04.6709.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c2n2.pc01.cls04.6708.-1.--receive.population
+topology.omaha-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.omaha-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n1.pc01.cls04.6720.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c1n3.pc01.cls04.6726.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g2c4n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c5n2.pc01.cls04.6705.-1.--receive.population
+topology.phoenix-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c3n4.pc01.cls04.6704.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c7n4.pc01.cls04.6721.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n3.pc01.cls04.6703.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c2n3.pc01.cls04.6727.-1.--emit-count.default
+topology.buffalo-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c3n3.pc01.cls04.6726.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6723.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c6n4.pc01.cls04.6715.-1.--emit-count.default
+topology.tacoma-storm-1.kafka-spout.g2c2n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c3n4.pc01.cls04.6706.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6710.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c3n3.pc01.cls04.6707.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g2c4n4.pc01.cls04.6713.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c5n1.pc01.cls04.6720.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c7n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.buffalo-storm-1.kafka-spout.g2c1n3.pc01.cls04.6707.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g1c3n4.pc01.cls04.6725.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c3n1.pc01.cls04.6715.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--emit-count.--system
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6707.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g1c5n4.pc01.cls04.6703.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c1n3.pc01.cls04.6708.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c5n2.pc01.cls04.6720.-1.--complete-latency.default
+topology.nashville-storm-1.kafka-spout.g1c7n3.pc01.cls04.6700.-1.--emit-count.--system
+topology.fort-worth-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6700.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c3n4.pc01.cls04.6722.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c5n4.pc01.cls04.6700.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c4n4.pc01.cls04.6708.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c2n2.pc01.cls04.6713.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c7n4.pc01.cls04.6724.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6713.-1.--receive.population
+topology.omaha-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c3n1.pc01.cls04.6701.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c6n1.pc01.cls04.6725.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c6n3.pc01.cls04.6701.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c2n2.pc01.cls04.6708.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c1n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g2c2n3.pc01.cls04.6722.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6720.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c6n4.pc01.cls04.6724.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c1n2.pc01.cls04.6706.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6720.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c4n2.pc01.cls04.6709.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c5n4.pc01.cls04.6723.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c3n2.pc01.cls04.6714.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c4n3.pc01.cls04.6711.-1.--emit-count.--system
+topology.washington-dc-storm-1.kafka-spout.g2c2n4.pc01.cls04.6704.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c3n3.pc01.cls04.6711.-1.--emit-count.--system
+topology.anaheim-storm-1.kafka-spout.g2c7n1.pc01.cls04.6725.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c2n1.pc01.cls04.6726.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c3n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c6n1.pc01.cls04.6723.-1.--sendqueue.population
+topology.nashville-storm-1.kafka-spout.g2c1n1.pc01.cls04.6712.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c2n4.pc01.cls04.6725.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g1c7n3.pc01.cls04.6705.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g2c4n4.pc01.cls04.6701.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c5n2.pc01.cls04.6701.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g1c3n4.pc01.cls04.6714.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6703.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c6n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c4n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c3n3.pc01.cls04.6726.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n4.pc01.cls04.6724.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--complete-latency.default
+topology.cheyenne-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c7n2.pc01.cls04.6705.-1.--receive.population
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c1n2.pc01.cls04.6726.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6726.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c5n1.pc01.cls04.6726.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c6n4.pc01.cls04.6708.-1.--ack-count.default
+topology.buffalo-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c7n1.pc01.cls04.6709.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g2c6n4.pc01.cls04.6712.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c5n2.pc01.cls04.6724.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c2n1.pc01.cls04.6707.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--emit-count.--system
+topology.anaheim-storm-1.kafka-spout.g1c3n3.pc01.cls04.6708.-1.--emit-count.--system
+topology.burbank-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--emit-count.--system
+topology.nyc-storm-1.kafka-spout.g2c2n2.pc01.cls04.6711.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c2n4.pc01.cls04.6727.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c7n4.pc01.cls04.6703.-1.--receive.population
+topology.tacoma-storm-1.kafka-spout.g1c1n3.pc01.cls04.6713.-1.--receive.population
+topology.phoenix-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c5n2.pc01.cls04.6712.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-spout.g1c6n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c7n2.pc01.cls04.6722.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g1c1n1.pc01.cls04.6713.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g2c5n4.pc01.cls04.6725.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c3n4.pc01.cls04.6706.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c6n3.pc01.cls04.6723.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g1c3n1.pc01.cls04.6722.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c4n1.pc01.cls04.6723.-1.--emit-count.default
+topology.tacoma-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g2c4n1.pc01.cls04.6710.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c6n4.pc01.cls04.6707.-1.--emit-count.--system
+topology.burbank-storm-1.kafka-spout.g1c4n1.pc01.cls04.6701.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-topic.pearl-city-storm-1.partition-37.earliestTimeOffset
+topology.fairfax-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6711.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--receive.population
+topology.nyc-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c3n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.buffalo-storm-1.kafka-spout.g2c6n1.pc01.cls04.6704.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c5n1.pc01.cls04.6700.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c6n4.pc01.cls04.6700.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c1n4.pc01.cls04.6711.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c5n2.pc01.cls04.6714.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6720.-1.--receive.population
+topology.buffalo-storm-1.kafka-spout.g1c5n2.pc01.cls04.6703.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g2c5n4.pc01.cls04.6703.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n2.pc01.cls04.6703.-1.--complete-latency.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6713.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g1c5n3.pc01.cls04.6725.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c5n3.pc01.cls04.6713.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c5n3.pc01.cls04.6708.-1.--sendqueue.population
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-5.spoutLag
+topology.harrison-storm-1.kafka-spout.g2c2n3.pc01.cls04.6727.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c4n4.pc01.cls04.6722.-1.--sendqueue.population
+topology.washington-dc-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c4n4.pc01.cls04.6723.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g1c5n2.pc01.cls04.6726.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c4n4.pc01.cls04.6724.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--ack-count.default
+topology.pearl-city-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c2n1.pc01.cls04.6711.-1.--receive.population
+topology.puerto-rico-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c1n4.pc01.cls04.6715.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6707.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g2c3n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c5n3.pc01.cls04.6708.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c3n2.pc01.cls04.6721.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c2n3.pc01.cls04.6712.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g2c2n2.pc01.cls04.6702.-1.--receive.population
+topology.pearl-city-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c1n4.pc01.cls04.6711.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c5n1.pc01.cls04.6727.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g1c7n1.pc01.cls04.6704.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g1c1n1.pc01.cls04.6701.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c2n4.pc01.cls04.6710.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c1n3.pc01.cls04.6722.-1.--emit-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c2n3.pc01.cls04.6724.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c6n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g2c4n3.pc01.cls04.6714.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c1n3.pc01.cls04.6705.-1.--ack-count.default
+topology.cheyenne-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c4n1.pc01.cls04.6703.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g2c1n2.pc01.cls04.6723.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n1.pc01.cls04.6711.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c3n4.pc01.cls04.6711.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c1n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c3n2.pc01.cls04.6723.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.tacoma-storm-1.kafka-spout.g1c2n1.pc01.cls04.6714.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c5n3.pc01.cls04.6722.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n3.pc01.cls04.6704.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6705.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c3n4.pc01.cls04.6713.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g1c3n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c6n3.pc01.cls04.6724.-1.--ack-count.default
+topology.miami-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c6n2.pc01.cls04.6714.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6703.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c6n2.pc01.cls04.6700.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--ack-count.default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c1n3.pc01.cls04.6706.-1.--emit-count.default
+topology.cheyenne-storm-1.kafka-spout.g1c4n4.pc01.cls04.6703.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c6n1.pc01.cls04.6726.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c1n3.pc01.cls04.6711.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c1n1.pc01.cls04.6710.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c2n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c7n2.pc01.cls04.6721.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c6n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6704.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6702.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n1.pc01.cls04.6709.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c2n1.pc01.cls04.6707.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6701.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c4n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c7n2.pc01.cls04.6714.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c7n4.pc01.cls04.6720.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c3n1.pc01.cls04.6708.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c4n2.pc01.cls04.6714.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c4n1.pc01.cls04.6706.-1.--emit-count.--system
+topology.phoenix-storm-1.kafka-spout.g1c6n3.pc01.cls04.6703.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c2n4.pc01.cls04.6722.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c3n1.pc01.cls04.6711.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c6n4.pc01.cls04.6727.-1.--ack-count.default
+topology.burbank-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n3.pc01.cls04.6725.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6707.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c3n2.pc01.cls04.6723.-1.--emit-count.--system
+topology.san-jose-storm-1.kafka-spout.g1c2n4.pc01.cls04.6711.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.pearl-city-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6700.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c2n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c1n1.pc01.cls04.6701.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c1n1.pc01.cls04.6712.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c1n1.pc01.cls04.6708.-1.--ack-count.default
+topology.atlanta-storm-1.kafka-spout.g1c3n2.pc01.cls04.6708.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c6n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c6n2.pc01.cls04.6726.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c1n4.pc01.cls04.6723.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c1n3.pc01.cls04.6723.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c1n4.pc01.cls04.6727.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g1c4n4.pc01.cls04.6714.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c5n4.pc01.cls04.6725.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c7n1.pc01.cls04.6725.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c3n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.pearl-city-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g1c3n3.pc01.cls04.6714.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c3n3.pc01.cls04.6710.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g1c6n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c6n2.pc01.cls04.6720.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g2c5n4.pc01.cls04.6708.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c5n3.pc01.cls04.6704.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n3.pc01.cls04.6700.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c5n2.pc01.cls04.6721.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6714.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c6n4.pc01.cls04.6703.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c7n4.pc01.cls04.6705.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c7n3.pc01.cls04.6706.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g1c2n2.pc01.cls04.6711.-1.--ack-count.default
+topology.omaha-storm-1.kafka-spout.g2c3n1.pc01.cls04.6713.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g1c3n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c7n3.pc01.cls04.6713.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g1c1n3.pc01.cls04.6715.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c2n2.pc01.cls04.6701.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.nashville-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g1c3n3.pc01.cls04.6714.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-11.latestEmittedOffset
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6724.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c7n2.pc01.cls04.6707.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g1c6n1.pc01.cls04.6713.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-spout.g1c1n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c6n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.nyc-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.cheyenne-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c1n3.pc01.cls04.6727.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c2n4.pc01.cls04.6706.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c6n2.pc01.cls04.6723.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g1c5n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c5n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c7n1.pc01.cls04.6720.-1.--ack-count.default
+topology.pearl-city-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c6n1.pc01.cls04.6710.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c4n1.pc01.cls04.6700.-1.--ack-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c2n2.pc01.cls04.6701.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c7n3.pc01.cls04.6702.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-29.latestEmittedOffset
+topology.elkridge-storm-1.kafka-spout.g2c1n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c3n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c3n4.pc01.cls04.6704.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n2.pc01.cls04.6707.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n2.pc01.cls04.6704.-1.--emit-count.default
+topology.akron-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c3n2.pc01.cls04.6710.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g1c2n1.pc01.cls04.6705.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c2n2.pc01.cls04.6714.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c3n4.pc01.cls04.6713.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c6n2.pc01.cls04.6726.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c4n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.pearl-city-storm-1.kafka-spout.g1c3n3.pc01.cls04.6721.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c7n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c5n2.pc01.cls04.6723.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c4n2.pc01.cls04.6727.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n4.pc01.cls04.6720.-1.--sendqueue.population
+topology.nyc-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c5n3.pc01.cls04.6714.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c1n1.pc01.cls04.6727.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c6n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g1c1n1.pc01.cls04.6720.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c6n3.pc01.cls04.6713.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-spout.g2c3n1.pc01.cls04.6710.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6713.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c5n3.pc01.cls04.6707.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6714.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c5n2.pc01.cls04.6701.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c7n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c4n3.pc01.cls04.6707.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c6n2.pc01.cls04.6726.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c4n4.pc01.cls04.6709.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n3.pc01.cls04.6700.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g2c1n3.pc01.cls04.6727.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c2n1.pc01.cls04.6712.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g1c7n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c6n2.pc01.cls04.6711.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c5n2.pc01.cls04.6712.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c2n3.pc01.cls04.6708.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c1n1.pc01.cls04.6711.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g2c6n3.pc01.cls04.6710.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c7n4.pc01.cls04.6705.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g2c6n3.pc01.cls04.6723.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c4n3.pc01.cls04.6702.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c1n1.pc01.cls04.6708.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c4n4.pc01.cls04.6714.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c1n3.pc01.cls04.6722.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g1c6n3.pc01.cls04.6726.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--ack-count.default
+topology.nyc-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c1n2.pc01.cls04.6700.-1.--ack-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c1n1.pc01.cls04.6723.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c4n2.pc01.cls04.6727.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c5n2.pc01.cls04.6703.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6723.-1.--emit-count.--system
+topology.seattle-storm-1.kafka-spout.g2c7n1.pc01.cls04.6722.-1.--emit-count.default
+topology.springfield-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n3.pc01.cls04.6705.-1.--receive.population
+topology.tacoma-storm-1.kafka-spout.g1c3n3.pc01.cls04.6710.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c7n4.pc01.cls04.6712.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c3n1.pc01.cls04.6726.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c1n4.pc01.cls04.6709.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c7n2.pc01.cls04.6702.-1.--emit-count.--system
+topology.burbank-storm-1.kafka-spout.g1c5n3.pc01.cls04.6701.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6713.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g2c2n4.pc01.cls04.6723.-1.--sendqueue.population
+topology.cheyenne-storm-1.kafka-spout.g1c3n2.pc01.cls04.6720.-1.--emit-count.default
+topology.cheyenne-storm-1.kafka-spout.g1c3n2.pc01.cls04.6720.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--ack-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c3n1.pc01.cls04.6704.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c6n2.pc01.cls04.6705.-1.--receive.population
+topology.harrison-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c4n1.pc01.cls04.6713.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c1n2.pc01.cls04.6727.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g2c3n3.pc01.cls04.6706.-1.--complete-latency.default
+topology.omaha-storm-1.kafka-spout.g2c1n1.pc01.cls04.6727.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g1c7n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--ack-count.default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6707.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.buffalo-storm-1.kafka-spout.g2c2n1.pc01.cls04.6707.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c4n1.pc01.cls04.6711.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c7n3.pc01.cls04.6709.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g1c5n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c1n2.pc01.cls04.6706.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c1n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c3n3.pc01.cls04.6725.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g1c5n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g2c7n1.pc01.cls04.6727.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n4.pc01.cls04.6721.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c6n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c3n2.pc01.cls04.6700.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c1n4.pc01.cls04.6700.-1.--receive.population
+topology.cheyenne-storm-1.kafka-spout.g1c5n4.pc01.cls04.6703.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c6n2.pc01.cls04.6725.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c1n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c1n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c6n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c1n3.pc01.cls04.6713.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c2n1.pc01.cls04.6704.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c6n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6702.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c5n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c7n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6724.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c3n1.pc01.cls04.6706.-1.--ack-count.default
+topology.cheyenne-storm-1.kafka-spout.g1c5n1.pc01.cls04.6703.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c4n3.pc01.cls04.6709.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c6n1.pc01.cls04.6724.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6702.-1.--sendqueue.population
+topology.atlanta-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c6n2.pc01.cls04.6709.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6704.-1.--emit-count.--system
+topology.orlando-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c5n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c7n1.pc01.cls04.6725.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n4.pc01.cls04.6709.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c4n2.pc01.cls04.6720.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c1n3.pc01.cls04.6704.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c5n3.pc01.cls04.6720.-1.--receive.population
+topology.springfield-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c2n3.pc01.cls04.6708.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c4n3.pc01.cls04.6700.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c6n3.pc01.cls04.6701.-1.--receive.population
+topology.akron-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c3n1.pc01.cls04.6706.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g1c2n3.pc01.cls04.6706.-1.--receive.population
+topology.harrison-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c6n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n3.pc01.cls04.6702.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c3n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c3n1.pc01.cls04.6723.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g1c6n1.pc01.cls04.6710.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c3n1.pc01.cls04.6710.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c2n4.pc01.cls04.6721.-1.--ack-count.default
+topology.harrison-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c6n4.pc01.cls04.6724.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c7n2.pc01.cls04.6704.-1.--ack-count.default
+topology.harrison-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6711.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g1c2n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.tacoma-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c5n2.pc01.cls04.6722.-1.--complete-latency.default
+topology.omaha-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.cheyenne-storm-1.kafka-topic.cheyenne-storm-1.partition-34.latestCompletedOffset
+topology.springfield-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6701.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c2n4.pc01.cls04.6711.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6725.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g2c2n2.pc01.cls04.6708.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g2c4n2.pc01.cls04.6704.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c1n4.pc01.cls04.6727.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c4n1.pc01.cls04.6704.-1.--receive.population
+topology.buffalo-storm-1.kafka-spout.g1c5n1.pc01.cls04.6710.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c1n2.pc01.cls04.6713.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c1n1.pc01.cls04.6710.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g1c2n4.pc01.cls04.6700.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c7n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6713.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c3n3.pc01.cls04.6713.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6713.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6725.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c5n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c4n1.pc01.cls04.6705.-1.--complete-latency.default
+topology.akron-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c4n2.pc01.cls04.6720.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c7n4.pc01.cls04.6720.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6713.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c4n4.pc01.cls04.6724.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c7n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g1c5n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c7n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g2c2n4.pc01.cls04.6723.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c6n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c4n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c4n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c6n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c5n3.pc01.cls04.6707.-1.--emit-count.--system
+topology.fairfax-storm-1.kafka-spout.g1c4n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g2c2n1.pc01.cls04.6708.-1.--complete-latency.default
+topology.akron-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n3.pc01.cls04.6726.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-spout.g1c6n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c2n3.pc01.cls04.6727.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c2n4.pc01.cls04.6706.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c2n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.omaha-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c5n4.pc01.cls04.6723.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c7n2.pc01.cls04.6727.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c3n1.pc01.cls04.6726.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c1n4.pc01.cls04.6702.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c6n1.pc01.cls04.6724.-1.--ack-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c3n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c4n3.pc01.cls04.6702.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c6n4.pc01.cls04.6708.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c7n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c5n2.pc01.cls04.6722.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c2n3.pc01.cls04.6724.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c2n2.pc01.cls04.6710.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c7n2.pc01.cls04.6702.-1.--emit-count.--system
+topology.puerto-rico-storm-1.kafka-spout.g2c3n3.pc01.cls04.6707.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--ack-count.default
+topology.nyc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g1c6n4.pc01.cls04.6704.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c4n4.pc01.cls04.6707.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c3n4.pc01.cls04.6724.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c5n2.pc01.cls04.6725.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c5n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c5n2.pc01.cls04.6706.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c4n4.pc01.cls04.6704.-1.--ack-count.default
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-36.latestEmittedOffset
+topology.tacoma-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c6n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c5n3.pc01.cls04.6727.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g2c2n4.pc01.cls04.6723.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6703.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c7n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c7n2.pc01.cls04.6711.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g1c3n1.pc01.cls04.6724.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c4n4.pc01.cls04.6727.-1.--ack-count.default
+topology.tacoma-storm-1.kafka-spout.g2c5n2.pc01.cls04.6707.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c7n3.pc01.cls04.6702.-1.--receive.population
+topology.harrison-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c5n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c3n2.pc01.cls04.6721.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c3n3.pc01.cls04.6725.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c2n2.pc01.cls04.6711.-1.--emit-count.--system
+topology.san-jose-storm-1.kafka-spout.g1c4n2.pc01.cls04.6727.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g1c5n2.pc01.cls04.6724.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c2n1.pc01.cls04.6710.-1.--ack-count.default
+topology.omaha-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c3n4.pc01.cls04.6726.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c4n2.pc01.cls04.6727.-1.--complete-latency.default
+topology.tacoma-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n4.pc01.cls04.6708.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c4n3.pc01.cls04.6709.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c1n1.pc01.cls04.6710.-1.--emit-count.--system
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c1n4.pc01.cls04.6700.-1.--ack-count.default
+topology.chicago-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c5n1.pc01.cls04.6701.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c2n1.pc01.cls04.6727.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c2n2.pc01.cls04.6708.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c3n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6700.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c1n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c2n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c7n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g2c3n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6726.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c1n2.pc01.cls04.6709.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g1c2n4.pc01.cls04.6700.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g1c1n4.pc01.cls04.6724.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c5n1.pc01.cls04.6724.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c1n2.pc01.cls04.6707.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c1n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.tacoma-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c3n4.pc01.cls04.6722.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c3n1.pc01.cls04.6725.-1.--emit-count.default
+topology.springfield-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c1n2.pc01.cls04.6711.-1.--emit-count.--system
+topology.fairfax-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c1n1.pc01.cls04.6700.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c2n4.pc01.cls04.6721.-1.--complete-latency.default
+topology.houston-storm-1.kafka-spout.g2c4n2.pc01.cls04.6704.-1.--emit-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6712.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c4n4.pc01.cls04.6708.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c6n4.pc01.cls04.6700.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c6n4.pc01.cls04.6704.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c7n3.pc01.cls04.6701.-1.--receive.population
+topology.tacoma-storm-1.kafka-spout.g2c6n3.pc01.cls04.6703.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g1c5n3.pc01.cls04.6723.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c5n1.pc01.cls04.6724.-1.--ack-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c3n2.pc01.cls04.6724.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c5n4.pc01.cls04.6720.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c3n4.pc01.cls04.6713.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g1c7n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c6n4.pc01.cls04.6708.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6707.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c2n1.pc01.cls04.6713.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c3n3.pc01.cls04.6706.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n1.pc01.cls04.6725.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6704.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g2c2n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c1n1.pc01.cls04.6723.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c7n1.pc01.cls04.6724.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g1c5n1.pc01.cls04.6724.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6701.-1.--emit-count.default
+topology.houston-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c3n3.pc01.cls04.6706.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c3n1.pc01.cls04.6724.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c6n1.pc01.cls04.6722.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c5n3.pc01.cls04.6724.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n1.pc01.cls04.6707.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c2n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c3n3.pc01.cls04.6724.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c3n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c4n2.pc01.cls04.6714.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-36.spoutLag
+topology.elkridge-storm-1.kafka-spout.g2c2n4.pc01.cls04.6700.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c4n1.pc01.cls04.6702.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c6n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6727.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c1n2.pc01.cls04.6701.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c4n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c3n2.pc01.cls04.6700.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6706.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n4.pc01.cls04.6721.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c3n2.pc01.cls04.6712.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c2n4.pc01.cls04.6703.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g2c4n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c4n4.pc01.cls04.6720.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g1c6n1.pc01.cls04.6707.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c3n1.pc01.cls04.6708.-1.--emit-count.--system
+topology.atlanta-storm-1.kafka-spout.g2c4n3.pc01.cls04.6714.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g1c2n1.pc01.cls04.6714.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c4n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c5n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c3n4.pc01.cls04.6707.-1.--receive.population
+topology.fort-worth-storm-1.kafka-topic.fort-worth-storm-1.partition-13.spoutLag
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6700.-1.--receive.population
+topology.omaha-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c5n4.pc01.cls04.6706.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c2n2.pc01.cls04.6712.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c5n4.pc01.cls04.6726.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c7n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.cheyenne-storm-1.kafka-spout.g1c4n4.pc01.cls04.6703.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c2n3.pc01.cls04.6713.-1.--receive.population
+topology.cheyenne-storm-1.kafka-spout.g2c4n4.pc01.cls04.6703.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c6n4.pc01.cls04.6715.-1.--ack-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c3n1.pc01.cls04.6720.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c5n4.pc01.cls04.6708.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6720.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c3n1.pc01.cls04.6702.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g1c6n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c6n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g1c4n1.pc01.cls04.6709.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c7n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c1n1.pc01.cls04.6720.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6713.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c6n2.pc01.cls04.6710.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c6n3.pc01.cls04.6703.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n1.pc01.cls04.6706.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c4n4.pc01.cls04.6707.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c1n4.pc01.cls04.6705.-1.--emit-count.--system
+topology.st-paul-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c7n3.pc01.cls04.6720.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n3.pc01.cls04.6700.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6720.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c6n2.pc01.cls04.6700.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c5n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c4n2.pc01.cls04.6707.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6710.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c7n1.pc01.cls04.6702.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c2n1.pc01.cls04.6707.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c4n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c4n3.pc01.cls04.6722.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6709.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c2n4.pc01.cls04.6704.-1.--emit-count.default
+topology.tacoma-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c5n4.pc01.cls04.6722.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c7n3.pc01.cls04.6712.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g2c2n4.pc01.cls04.6720.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g1c5n1.pc01.cls04.6727.-1.--emit-count.default
+topology.springfield-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g1c2n1.pc01.cls04.6713.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c2n1.pc01.cls04.6703.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c6n1.pc01.cls04.6725.-1.--receive.population
+topology.harrison-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6707.-1.--emit-count.--system
+topology.seattle-storm-1.kafka-spout.g1c3n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n4.pc01.cls04.6711.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c7n4.pc01.cls04.6702.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c4n4.pc01.cls04.6723.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g2c6n2.pc01.cls04.6701.-1.--ack-count.default
+topology.springfield-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6722.-1.--complete-latency.default
+topology.akron-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c3n2.pc01.cls04.6704.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g1c5n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g1c5n1.pc01.cls04.6721.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g1c6n3.pc01.cls04.6702.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c2n3.pc01.cls04.6720.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c7n2.pc01.cls04.6722.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c2n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c2n4.pc01.cls04.6720.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c2n2.pc01.cls04.6701.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c1n4.pc01.cls04.6724.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g2c6n2.pc01.cls04.6701.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g2c2n1.pc01.cls04.6707.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c2n4.pc01.cls04.6712.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6724.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.omaha-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n2.pc01.cls04.6700.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c3n4.pc01.cls04.6701.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c2n3.pc01.cls04.6713.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c7n3.pc01.cls04.6727.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c7n2.pc01.cls04.6707.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c2n2.pc01.cls04.6704.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c2n4.pc01.cls04.6726.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g1c2n2.pc01.cls04.6703.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c5n4.pc01.cls04.6727.-1.--ack-count.default
+topology.omaha-storm-1.kafka-spout.g2c5n2.pc01.cls04.6710.-1.--receive.population
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c2n3.pc01.cls04.6701.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c1n4.pc01.cls04.6705.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n2.pc01.cls04.6710.-1.--emit-count.--system
+topology.anaheim-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n2.pc01.cls04.6727.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c2n1.pc01.cls04.6703.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c2n2.pc01.cls04.6724.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c2n1.pc01.cls04.6704.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c5n3.pc01.cls04.6703.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c2n3.pc01.cls04.6725.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.nashville-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6713.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6719.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6725.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g2c2n2.pc01.cls04.6721.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c2n4.pc01.cls04.6710.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c1n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c3n3.pc01.cls04.6723.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g2c7n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c1n1.pc01.cls04.6701.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c6n4.pc01.cls04.6722.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g1c1n3.pc01.cls04.6701.-1.--emit-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c6n2.pc01.cls04.6725.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g2c1n3.pc01.cls04.6710.-1.--complete-latency.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c2n3.pc01.cls04.6707.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c6n1.pc01.cls04.6724.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c5n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c4n4.pc01.cls04.6712.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c5n4.pc01.cls04.6708.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c7n3.pc01.cls04.6723.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c1n3.pc01.cls04.6725.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c2n2.pc01.cls04.6702.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--ack-count.default
+topology.nashville-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6713.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g2c1n1.pc01.cls04.6710.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c3n3.pc01.cls04.6722.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g2c4n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c5n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g2c5n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.tacoma-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c1n3.pc01.cls04.6723.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.buffalo-storm-1.kafka-spout.g1c5n2.pc01.cls04.6703.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6704.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c1n4.pc01.cls04.6723.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g1c7n3.pc01.cls04.6724.-1.--emit-count.default
+topology.tacoma-storm-1.kafka-spout.g1c6n2.pc01.cls04.6713.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c5n3.pc01.cls04.6705.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c1n2.pc01.cls04.6721.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c1n3.pc01.cls04.6703.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6709.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c5n4.pc01.cls04.6710.-1.--receive.population
+topology.springfield-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g2c5n2.pc01.cls04.6701.-1.--ack-count.default
+topology.pearl-city-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--ack-count.default
+topology.akron-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c6n3.pc01.cls04.6723.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c5n3.pc01.cls04.6701.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c1n3.pc01.cls04.6702.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g2c2n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g2c7n3.pc01.cls04.6709.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6709.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c7n2.pc01.cls04.6709.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.pearl-city-storm-1.kafka-spout.g2c5n2.pc01.cls04.6721.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g2c7n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g2c4n1.pc01.cls04.6706.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c5n2.pc01.cls04.6706.-1.--emit-count.--system
+topology.burbank-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c5n4.pc01.cls04.6713.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c3n2.pc01.cls04.6724.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c3n4.pc01.cls04.6701.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c2n3.pc01.cls04.6724.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c5n3.pc01.cls04.6703.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6721.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g1c3n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.cheyenne-storm-1.kafka-spout.g1c3n2.pc01.cls04.6720.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c3n3.pc01.cls04.6712.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g1c6n1.pc01.cls04.6720.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c1n1.pc01.cls04.6727.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6726.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c7n4.pc01.cls04.6705.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6704.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c1n2.pc01.cls04.6727.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c3n4.pc01.cls04.6704.-1.--emit-count.--system
+topology.orlando-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g1c4n3.pc01.cls04.6713.-1.--emit-count.--system
+topology.san-jose-storm-1.kafka-spout.g1c1n2.pc01.cls04.6713.-1.--complete-latency.default
+topology.orlando-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c2n3.pc01.cls04.6724.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g1c1n2.pc01.cls04.6705.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c7n3.pc01.cls04.6723.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c4n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6707.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c2n4.pc01.cls04.6702.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c1n4.pc01.cls04.6724.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c3n2.pc01.cls04.6700.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c3n4.pc01.cls04.6706.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c7n3.pc01.cls04.6701.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c2n3.pc01.cls04.6725.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--receive.population
+topology.pearl-city-storm-1.kafka-spout.g1c2n1.pc01.cls04.6713.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c4n4.pc01.cls04.6710.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c7n2.pc01.cls04.6706.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c1n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c1n2.pc01.cls04.6727.-1.--sendqueue.population
+topology.nyc-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c3n1.pc01.cls04.6727.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c6n1.pc01.cls04.6707.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c3n1.pc01.cls04.6720.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g2c4n2.pc01.cls04.6708.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.houston-storm-1.kafka-spout.g2c6n4.pc01.cls04.6714.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6723.-1.--ack-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6721.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c7n2.pc01.cls04.6704.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c6n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c6n2.pc01.cls04.6709.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c6n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6702.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c3n2.pc01.cls04.6727.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c1n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c1n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c7n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c4n1.pc01.cls04.6709.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c7n1.pc01.cls04.6707.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c6n4.pc01.cls04.6711.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c3n4.pc01.cls04.6700.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g2c3n2.pc01.cls04.6726.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.puerto-rico-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n1.pc01.cls04.6702.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n4.pc01.cls04.6711.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c3n4.pc01.cls04.6724.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c2n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c1n1.pc01.cls04.6714.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c2n2.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c3n2.pc01.cls04.6725.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c7n1.pc01.cls04.6704.-1.--emit-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c1n2.pc01.cls04.6727.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c5n2.pc01.cls04.6724.-1.--emit-count.default
+topology.cheyenne-storm-1.kafka-topic.cheyenne-storm-1.partition-4.spoutLag
+topology.anaheim-storm-1.kafka-spout.g1c7n1.pc01.cls04.6703.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g1c5n1.pc01.cls04.6705.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c5n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c2n1.pc01.cls04.6722.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c6n2.pc01.cls04.6713.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c4n3.pc01.cls04.6706.-1.--ack-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c2n2.pc01.cls04.6705.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c1n3.pc01.cls04.6727.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g1c1n3.pc01.cls04.6707.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g1c7n1.pc01.cls04.6701.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c5n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c4n2.pc01.cls04.6713.-1.--ack-count.default
+topology.chicago-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--emit-count.default
+topology.houston-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c7n4.pc01.cls04.6705.-1.--emit-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6723.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c3n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c5n1.pc01.cls04.6726.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6701.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c6n2.pc01.cls04.6714.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c1n2.pc01.cls04.6700.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c4n3.pc01.cls04.6701.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c2n4.pc01.cls04.6701.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g2c5n3.pc01.cls04.6707.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c5n4.pc01.cls04.6709.-1.--ack-count.default
+topology.tacoma-storm-1.kafka-spout.g1c6n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g2c7n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c1n4.pc01.cls04.6700.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c3n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c5n4.pc01.cls04.6708.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c4n4.pc01.cls04.6709.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c1n4.pc01.cls04.6723.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c6n1.pc01.cls04.6727.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c6n2.pc01.cls04.6726.-1.--ack-count.default
+topology.atlanta-storm-1.kafka-spout.g2c1n1.pc01.cls04.6705.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c1n2.pc01.cls04.6703.-1.--ack-count.default
+topology.atlanta-storm-1.kafka-spout.g1c4n1.pc01.cls04.6721.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6704.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c4n3.pc01.cls04.6700.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g1c1n4.pc01.cls04.6703.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c7n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c1n4.pc01.cls04.6723.-1.--ack-count.default
+topology.atlanta-storm-1.kafka-spout.g2c2n2.pc01.cls04.6722.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c6n4.pc01.cls04.6706.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c3n2.pc01.cls04.6720.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c3n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g2c6n4.pc01.cls04.6721.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c1n3.pc01.cls04.6723.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c1n3.pc01.cls04.6708.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c5n1.pc01.cls04.6727.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c4n4.pc01.cls04.6714.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c7n2.pc01.cls04.6722.-1.--ack-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c5n3.pc01.cls04.6711.-1.--ack-count.default
+topology.cheyenne-storm-1.kafka-spout.g1c6n3.pc01.cls04.6724.-1.--sendqueue.population
+topology.burbank-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c5n4.pc01.cls04.6727.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c7n2.pc01.cls04.6721.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c3n4.pc01.cls04.6727.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c6n1.pc01.cls04.6727.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c5n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c7n1.pc01.cls04.6710.-1.--ack-count.default
+topology.springfield-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c4n3.pc01.cls04.6709.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c2n2.pc01.cls04.6700.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.puerto-rico-storm-1.kafka-spout.g1c7n1.pc01.cls04.6712.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c6n3.pc01.cls04.6714.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c1n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g1c4n3.pc01.cls04.6712.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-spout.g2c1n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c5n2.pc01.cls04.6707.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c4n4.pc01.cls04.6712.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g2c4n3.pc01.cls04.6710.-1.--complete-latency.default
+topology.puerto-rico-storm-1.kafka-spout.g2c1n1.pc01.cls04.6708.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6727.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.puerto-rico-storm-1.kafka-spout.g1c3n4.pc01.cls04.6707.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6702.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c3n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g2c1n2.pc01.cls04.6723.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c4n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c7n1.pc01.cls04.6720.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c7n2.pc01.cls04.6702.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c5n1.pc01.cls04.6713.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c5n2.pc01.cls04.6710.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c1n3.pc01.cls04.6720.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c3n1.pc01.cls04.6701.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c7n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c2n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c5n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c5n4.pc01.cls04.6725.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c3n4.pc01.cls04.6727.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n2.pc01.cls04.6711.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c2n3.pc01.cls04.6708.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c4n1.pc01.cls04.6713.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6703.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c3n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c4n3.pc01.cls04.6706.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g2c1n3.pc01.cls04.6705.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6709.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6713.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c1n1.pc01.cls04.6720.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g1c1n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c4n3.pc01.cls04.6702.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g2c2n3.pc01.cls04.6712.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c7n4.pc01.cls04.6725.-1.--ack-count.default
+topology.cheyenne-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g2c3n4.pc01.cls04.6726.-1.--sendqueue.population
+topology.omaha-storm-1.kafka-spout.g1c5n3.pc01.cls04.6712.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c5n3.pc01.cls04.6725.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c1n3.pc01.cls04.6704.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c1n2.pc01.cls04.6722.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-topic.phoenix-storm-1.partition-28.spoutLag
+topology.nashville-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c7n1.pc01.cls04.6703.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c2n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6701.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6722.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c6n1.pc01.cls04.6708.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c5n1.pc01.cls04.6725.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.omaha-storm-1.kafka-spout.g2c3n1.pc01.cls04.6713.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g1c3n3.pc01.cls04.6702.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c6n2.pc01.cls04.6711.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c3n3.pc01.cls04.6725.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c6n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c5n1.pc01.cls04.6711.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n3.pc01.cls04.6703.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c3n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.burbank-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c4n2.pc01.cls04.6707.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c2n3.pc01.cls04.6713.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c2n2.pc01.cls04.6725.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c6n2.pc01.cls04.6723.-1.--ack-count.default
+topology.puerto-rico-storm-1.kafka-spout.g2c3n1.pc01.cls04.6702.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c1n1.pc01.cls04.6707.-1.--receive.population
+topology.tacoma-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c3n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6713.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c2n3.pc01.cls04.6726.-1.--emit-count.--system
+topology.san-jose-storm-1.kafka-spout.g2c3n4.pc01.cls04.6700.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6709.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c3n2.pc01.cls04.6704.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c7n2.pc01.cls04.6724.-1.--ack-count.default
+topology.pearl-city-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c6n4.pc01.cls04.6726.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g2c4n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6707.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c1n4.pc01.cls04.6711.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g2c5n3.pc01.cls04.6701.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c6n4.pc01.cls04.6704.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g1c6n1.pc01.cls04.6700.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c2n3.pc01.cls04.6724.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g1c6n2.pc01.cls04.6720.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g2c6n2.pc01.cls04.6703.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c1n2.pc01.cls04.6706.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g2c6n3.pc01.cls04.6714.-1.--sendqueue.population
+topology.cheyenne-storm-1.kafka-spout.g1c4n4.pc01.cls04.6703.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g1c5n2.pc01.cls04.6714.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-topic.phoenix-storm-1.partition-28.latestTimeOffset
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c2n2.pc01.cls04.6708.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c5n3.pc01.cls04.6705.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c4n3.pc01.cls04.6708.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c3n2.pc01.cls04.6713.-1.--sendqueue.population
+topology.burbank-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c6n4.pc01.cls04.6705.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c7n3.pc01.cls04.6726.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g2c5n3.pc01.cls04.6703.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c2n4.pc01.cls04.6706.-1.--ack-count.default
+topology.miami-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c1n1.pc01.cls04.6710.-1.--ack-count.default
+topology.cheyenne-storm-1.kafka-spout.g1c4n4.pc01.cls04.6723.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c3n4.pc01.cls04.6713.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c1n4.pc01.cls04.6706.-1.--ack-count.default
+topology.akron-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6700.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c2n3.pc01.cls04.6725.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g1c7n1.pc01.cls04.6701.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c6n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6727.-1.--receive.population
+topology.tacoma-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c3n1.pc01.cls04.6706.-1.--complete-latency.default
+topology.miami-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c2n1.pc01.cls04.6714.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g2c5n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c1n3.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c1n2.pc01.cls04.6726.-1.--emit-count.--system
+topology.kansas-city-storm-1.kafka-spout.g1c4n3.pc01.cls04.6710.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c4n3.pc01.cls04.6701.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c3n4.pc01.cls04.6725.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6722.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g2c3n2.pc01.cls04.6720.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6721.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g1c5n1.pc01.cls04.6721.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c4n3.pc01.cls04.6710.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g2c7n2.pc01.cls04.6725.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c3n1.pc01.cls04.6725.-1.--receive.population
+topology.washington-dc-storm-1.kafka-spout.g1c3n4.pc01.cls04.6706.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c4n2.pc01.cls04.6706.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c2n4.pc01.cls04.6721.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c1n3.pc01.cls04.6706.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g1c2n1.pc01.cls04.6704.-1.--sendqueue.population
+topology.omaha-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c6n1.pc01.cls04.6722.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g2c2n1.pc01.cls04.6700.-1.--emit-count.--system
+topology.stockton-storm-1.kafka-spout.g1c2n1.pc01.cls04.6710.-1.--emit-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c7n3.pc01.cls04.6705.-1.--complete-latency.default
+topology.houston-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c3n2.pc01.cls04.6712.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c6n3.pc01.cls04.6700.-1.--emit-count.default
+topology.omaha-storm-1.kafka-spout.g2c6n1.pc01.cls04.6715.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g1c5n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c4n1.pc01.cls04.6722.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c5n3.pc01.cls04.6722.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c2n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g2c7n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c2n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.puerto-rico-storm-1.kafka-spout.g1c7n1.pc01.cls04.6712.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c4n4.pc01.cls04.6701.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c7n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c2n4.pc01.cls04.6703.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c6n3.pc01.cls04.6713.-1.--sendqueue.population
+topology.nyc-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c7n1.pc01.cls04.6701.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c4n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c2n1.pc01.cls04.6704.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c3n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.pearl-city-storm-1.kafka-topic.pearl-city-storm-1.partition-37.latestTimeOffset
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c7n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c3n4.pc01.cls04.6707.-1.--receive.population
+topology.phoenix-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c2n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.phoenix-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c5n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.omaha-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c1n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g1c6n2.pc01.cls04.6727.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c7n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.tacoma-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6700.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n2.pc01.cls04.6726.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c4n1.pc01.cls04.6701.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c1n2.pc01.cls04.6700.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c5n3.pc01.cls04.6713.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g1c1n3.pc01.cls04.6726.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c7n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.cheyenne-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6725.-1.--emit-count.--system
+topology.kansas-city-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c1n3.pc01.cls04.6703.-1.--emit-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c2n2.pc01.cls04.6703.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g2c1n3.pc01.cls04.6722.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c2n2.pc01.cls04.6702.-1.--receive.population
+topology.nyc-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c4n3.pc01.cls04.6711.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c4n2.pc01.cls04.6709.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c3n3.pc01.cls04.6703.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c1n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g2c2n4.pc01.cls04.6702.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c6n4.pc01.cls04.6700.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c2n2.pc01.cls04.6727.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g2c3n1.pc01.cls04.6701.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n3.pc01.cls04.6706.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6722.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c6n2.pc01.cls04.6706.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g1c1n4.pc01.cls04.6720.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c6n4.pc01.cls04.6727.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g2c6n2.pc01.cls04.6725.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6709.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c2n3.pc01.cls04.6725.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c5n4.pc01.cls04.6706.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-7.latestTimeOffset
+topology.burbank-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--emit-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c6n1.pc01.cls04.6724.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c3n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c7n2.pc01.cls04.6726.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c2n4.pc01.cls04.6714.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g2c2n1.pc01.cls04.6704.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6713.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.houston-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g1c6n2.pc01.cls04.6708.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c5n1.pc01.cls04.6705.-1.--complete-latency.default
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-18.latestTimeOffset
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6724.-1.--emit-count.--system
+topology.pearl-city-storm-1.kafka-spout.g1c3n3.pc01.cls04.6721.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c5n1.pc01.cls04.6706.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c1n1.pc01.cls04.6720.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c3n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.orlando-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c2n2.pc01.cls04.6727.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c6n1.pc01.cls04.6720.-1.--receive.population
+topology.tacoma-storm-1.kafka-spout.g2c1n4.pc01.cls04.6715.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n2.pc01.cls04.6712.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6718.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c6n3.pc01.cls04.6712.-1.--complete-latency.default
+topology.pearl-city-storm-1.kafka-spout.g1c5n2.pc01.cls04.6721.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g2c4n2.pc01.cls04.6724.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c2n2.pc01.cls04.6700.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c2n1.pc01.cls04.6726.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c6n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c7n4.pc01.cls04.6723.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c5n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c4n4.pc01.cls04.6702.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c5n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c5n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c3n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g2c6n2.pc01.cls04.6702.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c2n4.pc01.cls04.6722.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6724.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c4n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c4n1.pc01.cls04.6702.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.orlando-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c7n4.pc01.cls04.6722.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6706.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c4n2.pc01.cls04.6708.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g2c6n4.pc01.cls04.6726.-1.--sendqueue.population
+topology.buffalo-storm-1.kafka-spout.g1c4n4.pc01.cls04.6714.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c7n2.pc01.cls04.6714.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g2c6n1.pc01.cls04.6724.-1.--emit-count.--system
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c1n4.pc01.cls04.6723.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n1.pc01.cls04.6702.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g2c4n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c2n1.pc01.cls04.6708.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c2n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6702.-1.--ack-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c1n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c6n2.pc01.cls04.6723.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6713.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c3n2.pc01.cls04.6714.-1.--emit-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6703.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c4n3.pc01.cls04.6711.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c6n2.pc01.cls04.6704.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c4n1.pc01.cls04.6724.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6724.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c7n1.pc01.cls04.6707.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c4n2.pc01.cls04.6727.-1.--sendqueue.population
+topology.pearl-city-storm-1.kafka-spout.g1c3n4.pc01.cls04.6721.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g2c1n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c7n3.pc01.cls04.6726.-1.--ack-count.default
+topology.omaha-storm-1.kafka-spout.g1c5n2.pc01.cls04.6704.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c3n2.pc01.cls04.6720.-1.--receive.population
+topology.nyc-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6712.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c2n1.pc01.cls04.6713.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c6n4.pc01.cls04.6708.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c3n1.pc01.cls04.6707.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c5n3.pc01.cls04.6709.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c3n1.pc01.cls04.6724.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6704.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c2n2.pc01.cls04.6711.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c5n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.houston-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--emit-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c4n2.pc01.cls04.6707.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c3n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.buffalo-storm-1.kafka-spout.g1c5n1.pc01.cls04.6710.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c3n3.pc01.cls04.6712.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g2c5n2.pc01.cls04.6713.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c2n3.pc01.cls04.6721.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6700.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c5n1.pc01.cls04.6706.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c2n2.pc01.cls04.6708.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g2c7n3.pc01.cls04.6714.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--ack-count.default
+topology.springfield-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g1c5n3.pc01.cls04.6715.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6712.-1.--receive.population
+topology.akron-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c3n1.pc01.cls04.6705.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c5n1.pc01.cls04.6704.-1.--receive.population
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-29.latestTimeOffset
+topology.stockton-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c2n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c3n1.pc01.cls04.6706.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c3n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c3n3.pc01.cls04.6700.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6704.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c7n1.pc01.cls04.6710.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c1n1.pc01.cls04.6711.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g1c5n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c6n4.pc01.cls04.6702.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g1c6n2.pc01.cls04.6702.-1.--ack-count.default
+topology.pearl-city-storm-1.kafka-spout.g1c3n3.pc01.cls04.6721.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c4n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c4n4.pc01.cls04.6703.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c2n2.pc01.cls04.6711.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g1c4n4.pc01.cls04.6701.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6705.-1.--emit-count.default
+topology.houston-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g1c1n3.pc01.cls04.6711.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c7n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c1n3.pc01.cls04.6705.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g1c6n3.pc01.cls04.6727.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c1n2.pc01.cls04.6720.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c5n1.pc01.cls04.6726.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c2n2.pc01.cls04.6710.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c5n3.pc01.cls04.6706.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c6n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.houston-storm-1.kafka-spout.g2c4n2.pc01.cls04.6704.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c5n1.pc01.cls04.6718.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c1n1.pc01.cls04.6708.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6723.-1.--ack-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c2n4.pc01.cls04.6727.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6719.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n4.pc01.cls04.6705.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c6n1.pc01.cls04.6712.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c4n4.pc01.cls04.6707.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c3n4.pc01.cls04.6701.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6706.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g1c7n3.pc01.cls04.6705.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g2c7n4.pc01.cls04.6720.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c4n3.pc01.cls04.6706.-1.--sendqueue.population
+topology.omaha-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.houston-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c6n3.pc01.cls04.6720.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--complete-latency.default
+topology.tacoma-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c1n2.pc01.cls04.6727.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g1c4n3.pc01.cls04.6700.-1.--ack-count.default
+topology.atlanta-storm-1.kafka-spout.g1c5n3.pc01.cls04.6723.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c7n4.pc01.cls04.6706.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c6n1.pc01.cls04.6713.-1.--ack-count.default
+topology.nyc-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c2n1.pc01.cls04.6715.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6714.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6713.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c7n4.pc01.cls04.6701.-1.--receive.population
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c5n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n4.pc01.cls04.6705.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6727.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c2n1.pc01.cls04.6707.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c7n1.pc01.cls04.6709.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c4n3.pc01.cls04.6712.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c7n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g2c1n3.pc01.cls04.6722.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c1n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g2c1n3.pc01.cls04.6722.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c6n1.pc01.cls04.6725.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c3n1.pc01.cls04.6724.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c1n4.pc01.cls04.6708.-1.--sendqueue.population
+topology.phoenix-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c7n3.pc01.cls04.6703.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c5n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c7n4.pc01.cls04.6705.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c4n2.pc01.cls04.6701.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c2n1.pc01.cls04.6713.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c5n4.pc01.cls04.6702.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6713.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c4n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.phoenix-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c1n1.pc01.cls04.6715.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c5n2.pc01.cls04.6724.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c5n2.pc01.cls04.6701.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c3n4.pc01.cls04.6713.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n1.pc01.cls04.6720.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g1c4n4.pc01.cls04.6712.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c3n1.pc01.cls04.6710.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c7n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c7n2.pc01.cls04.6700.-1.--emit-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c5n2.pc01.cls04.6712.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c3n2.pc01.cls04.6712.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c3n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c7n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c1n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c5n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c4n1.pc01.cls04.6721.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c5n4.pc01.cls04.6711.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c7n2.pc01.cls04.6711.-1.--emit-count.default
+topology.nashville-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g1c5n3.pc01.cls04.6715.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c2n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c6n4.pc01.cls04.6727.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g1c5n1.pc01.cls04.6725.-1.--receive.population
+topology.washington-dc-storm-1.kafka-spout.g1c5n2.pc01.cls04.6714.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n1.pc01.cls04.6706.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c1n1.pc01.cls04.6713.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c7n3.pc01.cls04.6713.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c3n4.pc01.cls04.6712.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c4n4.pc01.cls04.6701.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c4n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c1n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g1c5n2.pc01.cls04.6723.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c3n2.pc01.cls04.6704.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c6n4.pc01.cls04.6704.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c5n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c4n1.pc01.cls04.6723.-1.--receive.population
+topology.nashville-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c4n1.pc01.cls04.6725.-1.--ack-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c1n3.pc01.cls04.6711.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6713.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c5n4.pc01.cls04.6703.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c2n1.pc01.cls04.6709.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g1c7n4.pc01.cls04.6706.-1.--receive.population
+topology.akron-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c5n3.pc01.cls04.6704.-1.--receive.population
+topology.washington-dc-storm-1.kafka-spout.g1c5n1.pc01.cls04.6715.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c3n4.pc01.cls04.6709.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g1c5n2.pc01.cls04.6707.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n1.pc01.cls04.6725.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c6n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c4n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g2c3n3.pc01.cls04.6703.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c2n4.pc01.cls04.6720.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c3n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g2c5n4.pc01.cls04.6712.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c2n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c2n4.pc01.cls04.6703.-1.--sendqueue.population
+topology.cheyenne-storm-1.kafka-spout.g2c2n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c4n3.pc01.cls04.6720.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c5n1.pc01.cls04.6708.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c5n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c2n3.pc01.cls04.6713.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g1c4n4.pc01.cls04.6709.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c4n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c4n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c2n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g1c3n3.pc01.cls04.6709.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c3n2.pc01.cls04.6727.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6707.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c4n3.pc01.cls04.6711.-1.--receive.population
+topology.pearl-city-storm-1.kafka-spout.g2c3n3.pc01.cls04.6721.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g1c1n1.pc01.cls04.6726.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c3n4.pc01.cls04.6727.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c5n3.pc01.cls04.6722.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c7n1.pc01.cls04.6722.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c2n1.pc01.cls04.6704.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g2c1n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c1n1.pc01.cls04.6720.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.pearl-city-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c2n3.pc01.cls04.6723.-1.--sendqueue.population
+topology.nashville-storm-1.kafka-spout.g1c5n3.pc01.cls04.6714.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g2c3n3.pc01.cls04.6727.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c7n4.pc01.cls04.6725.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6703.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.phoenix-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c5n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c5n1.pc01.cls04.6721.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6725.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c5n1.pc01.cls04.6707.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c1n1.pc01.cls04.6727.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c6n2.pc01.cls04.6707.-1.--ack-count.default
+topology.nashville-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c6n3.pc01.cls04.6709.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c3n3.pc01.cls04.6722.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.buffalo-storm-1.kafka-spout.g1c5n4.pc01.cls04.6715.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g1c4n3.pc01.cls04.6700.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6704.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c4n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c4n2.pc01.cls04.6704.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c5n4.pc01.cls04.6720.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6708.-1.--ack-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.omaha-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c7n2.pc01.cls04.6726.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c3n4.pc01.cls04.6713.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c3n4.pc01.cls04.6727.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c4n2.pc01.cls04.6703.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c5n2.pc01.cls04.6700.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c6n3.pc01.cls04.6725.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c6n1.pc01.cls04.6723.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c4n3.pc01.cls04.6702.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c6n3.pc01.cls04.6725.-1.--emit-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c3n3.pc01.cls04.6725.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c5n1.pc01.cls04.6714.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g1c2n1.pc01.cls04.6704.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g1c1n3.pc01.cls04.6722.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c3n4.pc01.cls04.6704.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g1c1n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.orlando-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c4n1.pc01.cls04.6723.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g1c3n1.pc01.cls04.6720.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c2n1.pc01.cls04.6721.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c7n1.pc01.cls04.6710.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6707.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6706.-1.--ack-count.default
+topology.akron-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n2.pc01.cls04.6703.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c1n2.pc01.cls04.6727.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g1c7n3.pc01.cls04.6726.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c6n4.pc01.cls04.6706.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c7n2.pc01.cls04.6724.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g2c4n2.pc01.cls04.6704.-1.--emit-count.--system
+topology.washington-dc-storm-1.kafka-spout.g1c5n1.pc01.cls04.6715.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c3n2.pc01.cls04.6712.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c1n4.pc01.cls04.6700.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6700.-1.--complete-latency.default
+topology.buffalo-storm-1.kafka-spout.g2c2n4.pc01.cls04.6707.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-topic.fort-worth-storm-1.partition-20.spoutLag
+topology.chicago-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c3n4.pc01.cls04.6704.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c4n4.pc01.cls04.6704.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c6n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c5n2.pc01.cls04.6708.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g2c2n2.pc01.cls04.6710.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g2c5n3.pc01.cls04.6720.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-spout.g1c6n2.pc01.cls04.6707.-1.--emit-count.--system
+topology.phoenix-storm-1.kafka-spout.g2c5n2.pc01.cls04.6722.-1.--complete-latency.default
+topology.cheyenne-storm-1.kafka-spout.g1c6n1.pc01.cls04.6704.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c1n2.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c2n3.pc01.cls04.6702.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c6n3.pc01.cls04.6705.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c3n1.pc01.cls04.6704.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c3n4.pc01.cls04.6701.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.puerto-rico-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c6n2.pc01.cls04.6720.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c6n1.pc01.cls04.6726.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c2n2.pc01.cls04.6707.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c5n3.pc01.cls04.6706.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c7n1.pc01.cls04.6711.-1.--ack-count.default
+topology.tacoma-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c3n3.pc01.cls04.6706.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c5n4.pc01.cls04.6702.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g2c4n2.pc01.cls04.6711.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6713.-1.--emit-count.--system
+topology.cheyenne-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c5n4.pc01.cls04.6727.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c7n2.pc01.cls04.6726.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6707.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c2n4.pc01.cls04.6722.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c2n2.pc01.cls04.6705.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c7n2.pc01.cls04.6703.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c3n3.pc01.cls04.6709.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.buffalo-storm-1.kafka-spout.g2c4n4.pc01.cls04.6705.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g1c2n4.pc01.cls04.6707.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c7n4.pc01.cls04.6723.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g1c2n3.pc01.cls04.6713.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c3n4.pc01.cls04.6708.-1.--ack-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c7n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6706.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c4n4.pc01.cls04.6715.-1.--sendqueue.population
+topology.pearl-city-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c4n3.pc01.cls04.6723.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6724.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g1c2n4.pc01.cls04.6702.-1.--receive.population
+topology.omaha-storm-1.kafka-spout.g2c3n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c2n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g2c2n1.pc01.cls04.6708.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6708.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g2c2n1.pc01.cls04.6726.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c4n2.pc01.cls04.6723.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c5n2.pc01.cls04.6726.-1.--ack-count.default
+topology.chicago-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c7n3.pc01.cls04.6724.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g1c1n3.pc01.cls04.6705.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.omaha-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c1n3.pc01.cls04.6704.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6706.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n1.pc01.cls04.6709.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g1c5n2.pc01.cls04.6724.-1.--ack-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c4n1.pc01.cls04.6727.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c3n4.pc01.cls04.6704.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c6n3.pc01.cls04.6714.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c3n2.pc01.cls04.6713.-1.--emit-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c6n4.pc01.cls04.6721.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g1c3n4.pc01.cls04.6708.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c3n4.pc01.cls04.6709.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c3n3.pc01.cls04.6720.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c6n3.pc01.cls04.6706.-1.--sendqueue.population
+topology.burbank-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c7n1.pc01.cls04.6704.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g2c3n2.pc01.cls04.6720.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g2c4n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c4n1.pc01.cls04.6721.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6709.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c5n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c7n3.pc01.cls04.6702.-1.--sendqueue.population
+topology.nyc-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c1n1.pc01.cls04.6714.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c5n4.pc01.cls04.6723.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g2c7n4.pc01.cls04.6721.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c4n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c7n2.pc01.cls04.6707.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c1n2.pc01.cls04.6727.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c5n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n1.pc01.cls04.6725.-1.--sendqueue.population
+topology.tacoma-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c2n1.pc01.cls04.6726.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c3n2.pc01.cls04.6709.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6708.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c4n1.pc01.cls04.6713.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-spout.g1c3n4.pc01.cls04.6707.-1.--emit-count.--system
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6707.-1.--ack-count.default
+topology.buffalo-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c1n1.pc01.cls04.6706.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c2n1.pc01.cls04.6725.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n2.pc01.cls04.6720.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c3n4.pc01.cls04.6706.-1.--ack-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-topic.phoenix-storm-1.partition-22.latestTimeOffset
+topology.anaheim-storm-1.kafka-spout.g2c3n2.pc01.cls04.6710.-1.--emit-count.default
+topology.pearl-city-storm-1.kafka-topic.pearl-city-storm-1.partition-37.latestCompletedOffset
+topology.miami-storm-1.kafka-spout.g1c5n2.pc01.cls04.6704.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g1c1n3.pc01.cls04.6726.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6711.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c5n4.pc01.cls04.6721.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g1c7n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c4n2.pc01.cls04.6720.-1.--sendqueue.population
+topology.nashville-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c5n3.pc01.cls04.6727.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6708.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c2n2.pc01.cls04.6713.-1.--sendqueue.population
+topology.tacoma-storm-1.kafka-spout.g1c3n3.pc01.cls04.6714.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6711.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c2n4.pc01.cls04.6722.-1.--complete-latency.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6720.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6723.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c1n3.pc01.cls04.6708.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c7n2.pc01.cls04.6705.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n3.pc01.cls04.6704.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c1n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c3n2.pc01.cls04.6712.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c5n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c6n3.pc01.cls04.6705.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c2n3.pc01.cls04.6711.-1.--sendqueue.population
+topology.tacoma-storm-1.kafka-spout.g1c1n3.pc01.cls04.6707.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g1c6n4.pc01.cls04.6727.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c7n3.pc01.cls04.6705.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c2n1.pc01.cls04.6722.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c2n4.pc01.cls04.6701.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c1n2.pc01.cls04.6715.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g1c4n4.pc01.cls04.6704.-1.--emit-count.default
+topology.nashville-storm-1.kafka-spout.g2c5n2.pc01.cls04.6714.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c7n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6724.-1.--emit-count.default
+topology.nashville-storm-1.kafka-spout.g2c6n4.pc01.cls04.6702.-1.--receive.population
+topology.springfield-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c7n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6715.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c6n4.pc01.cls04.6702.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c4n2.pc01.cls04.6704.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c5n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n1.pc01.cls04.6720.-1.--complete-latency.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c3n3.pc01.cls04.6703.-1.--complete-latency.default
+topology.miami-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g1c5n3.pc01.cls04.6712.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c4n4.pc01.cls04.6712.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.phoenix-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c5n4.pc01.cls04.6702.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--receive.population
+topology.akron-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g1c3n4.pc01.cls04.6711.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g2c6n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g2c6n4.pc01.cls04.6708.-1.--receive.population
+topology.chicago-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c6n3.pc01.cls04.6706.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-topic.phoenix-storm-1.partition-22.earliestTimeOffset
+topology.burbank-storm-1.kafka-spout.g2c6n4.pc01.cls04.6720.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c4n4.pc01.cls04.6705.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c7n1.pc01.cls04.6707.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c5n2.pc01.cls04.6713.-1.--receive.population
+topology.fort-worth-storm-1.kafka-topic.fort-worth-storm-1.partition-20.latestCompletedOffset
+topology.miami-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.pearl-city-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c2n3.pc01.cls04.6711.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c1n4.pc01.cls04.6726.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c1n2.pc01.cls04.6713.-1.--emit-count.default
+topology.houston-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n1.pc01.cls04.6710.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c3n1.pc01.cls04.6720.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c7n4.pc01.cls04.6702.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c3n4.pc01.cls04.6709.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c1n1.pc01.cls04.6724.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c1n1.pc01.cls04.6727.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c5n2.pc01.cls04.6713.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c4n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n1.pc01.cls04.6709.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c3n2.pc01.cls04.6712.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c5n2.pc01.cls04.6709.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g2c1n4.pc01.cls04.6712.-1.--ack-count.default
+topology.tacoma-storm-1.kafka-spout.g2c1n1.pc01.cls04.6706.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c4n1.pc01.cls04.6710.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c5n4.pc01.cls04.6708.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6715.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6713.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c7n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.houston-storm-1.kafka-spout.g2c2n3.pc01.cls04.6710.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c6n2.pc01.cls04.6709.-1.--emit-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6709.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g2c3n3.pc01.cls04.6704.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c1n2.pc01.cls04.6711.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c5n2.pc01.cls04.6720.-1.--emit-count.--system
+topology.atlanta-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c7n4.pc01.cls04.6702.-1.--receive.population
+topology.puerto-rico-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--ack-count.default
+topology.burbank-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c4n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.orlando-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c6n4.pc01.cls04.6724.-1.--receive.population
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-0.latestCompletedOffset
+topology.orlando-storm-1.kafka-spout.g2c4n2.pc01.cls04.6727.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c3n4.pc01.cls04.6706.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c4n1.pc01.cls04.6724.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c6n3.pc01.cls04.6711.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c5n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c1n4.pc01.cls04.6714.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6711.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g1c4n4.pc01.cls04.6701.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c1n3.pc01.cls04.6725.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g1c7n1.pc01.cls04.6722.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c4n3.pc01.cls04.6707.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c1n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c4n4.pc01.cls04.6715.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6709.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n4.pc01.cls04.6721.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c2n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g2c2n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g2c3n2.pc01.cls04.6711.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--emit-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c7n1.pc01.cls04.6710.-1.--emit-count.default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c2n3.pc01.cls04.6715.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6702.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c5n2.pc01.cls04.6710.-1.--ack-count.default
+topology.houston-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c6n1.pc01.cls04.6702.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g2c4n1.pc01.cls04.6721.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--receive.population
+topology.buffalo-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c5n1.pc01.cls04.6705.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n2.pc01.cls04.6714.-1.--emit-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c2n4.pc01.cls04.6701.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c7n2.pc01.cls04.6722.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g1c1n4.pc01.cls04.6711.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c4n1.pc01.cls04.6701.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c7n4.pc01.cls04.6703.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c1n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c2n2.pc01.cls04.6701.-1.--ack-count.default
+topology.nashville-storm-1.kafka-spout.g2c6n1.pc01.cls04.6720.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c4n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c4n2.pc01.cls04.6706.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c7n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.nyc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c4n3.pc01.cls04.6710.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c7n4.pc01.cls04.6714.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c2n3.pc01.cls04.6701.-1.--ack-count.default
+topology.pearl-city-storm-1.kafka-spout.g1c5n2.pc01.cls04.6721.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c5n3.pc01.cls04.6711.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c1n3.pc01.cls04.6723.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c4n4.pc01.cls04.6721.-1.--ack-count.default
+topology.houston-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c5n1.pc01.cls04.6706.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c6n1.pc01.cls04.6720.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6724.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-6.spoutLag
+topology.stockton-storm-1.kafka-spout.g2c4n4.pc01.cls04.6721.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c4n2.pc01.cls04.6701.-1.--receive.population
+topology.springfield-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--ack-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c5n2.pc01.cls04.6720.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6702.-1.--ack-count.default
+topology.tacoma-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g2c1n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c3n3.pc01.cls04.6725.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6726.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c7n4.pc01.cls04.6727.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c4n4.pc01.cls04.6715.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c6n4.pc01.cls04.6707.-1.--emit-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c1n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c1n2.pc01.cls04.6721.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g1c3n3.pc01.cls04.6705.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c6n2.pc01.cls04.6704.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c3n4.pc01.cls04.6722.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c3n2.pc01.cls04.6712.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c7n2.pc01.cls04.6707.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6701.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c5n1.pc01.cls04.6718.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c6n4.pc01.cls04.6708.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c6n4.pc01.cls04.6714.-1.--complete-latency.default
+topology.pearl-city-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n2.pc01.cls04.6724.-1.--emit-count.--system
+topology.kansas-city-storm-1.kafka-spout.g2c5n2.pc01.cls04.6715.-1.--emit-count.default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6720.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c2n1.pc01.cls04.6706.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c5n4.pc01.cls04.6702.-1.--complete-latency.default
+topology.phoenix-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6708.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c4n3.pc01.cls04.6706.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c2n1.pc01.cls04.6711.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c6n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g2c5n3.pc01.cls04.6727.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g1c1n3.pc01.cls04.6712.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c4n1.pc01.cls04.6714.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c5n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c3n3.pc01.cls04.6726.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c4n4.pc01.cls04.6726.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c2n1.pc01.cls04.6725.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g1c4n4.pc01.cls04.6702.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c2n2.pc01.cls04.6707.-1.--emit-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c6n1.pc01.cls04.6704.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c6n3.pc01.cls04.6703.-1.--receive.population
+topology.nyc-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6711.-1.--receive.population
+topology.phoenix-storm-1.kafka-topic.phoenix-storm-1.partition-28.latestCompletedOffset
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c2n2.pc01.cls04.6710.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c2n2.pc01.cls04.6703.-1.--complete-latency.default
+topology.pearl-city-storm-1.kafka-spout.g1c5n3.pc01.cls04.6715.-1.--emit-count.--system
+topology.chicago-storm-1.kafka-spout.g1c4n3.pc01.cls04.6702.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c4n3.pc01.cls04.6700.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c3n1.pc01.cls04.6700.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c3n2.pc01.cls04.6701.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c3n1.pc01.cls04.6713.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6708.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c1n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n2.pc01.cls04.6714.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g1c2n3.pc01.cls04.6702.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c2n4.pc01.cls04.6712.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g1c5n3.pc01.cls04.6721.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.pearl-city-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c4n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c1n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g1c6n3.pc01.cls04.6709.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c2n1.pc01.cls04.6721.-1.--sendqueue.population
+topology.burbank-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c5n3.pc01.cls04.6717.-1.--emit-count.default
+topology.cheyenne-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c5n4.pc01.cls04.6723.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c1n1.pc01.cls04.6710.-1.--emit-count.default
+topology.cheyenne-storm-1.kafka-spout.g1c6n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.pearl-city-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c7n4.pc01.cls04.6706.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c4n3.pc01.cls04.6726.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c3n1.pc01.cls04.6721.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-7.latestEmittedOffset
+topology.stockton-storm-1.kafka-spout.g1c3n1.pc01.cls04.6726.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g2c2n2.pc01.cls04.6720.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c6n3.pc01.cls04.6711.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c4n1.pc01.cls04.6702.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c4n4.pc01.cls04.6724.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6705.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c1n1.pc01.cls04.6710.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c1n3.pc01.cls04.6711.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c1n4.pc01.cls04.6711.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c3n1.pc01.cls04.6725.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c1n2.pc01.cls04.6722.-1.--ack-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c7n4.pc01.cls04.6711.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c3n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c5n1.pc01.cls04.6711.-1.--receive.population
+topology.cheyenne-storm-1.kafka-spout.g1c5n4.pc01.cls04.6703.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c5n1.pc01.cls04.6707.-1.--emit-count.default
+topology.buffalo-storm-1.kafka-spout.g2c5n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c1n4.pc01.cls04.6727.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c4n1.pc01.cls04.6711.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g2c1n3.pc01.cls04.6727.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6712.-1.--emit-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c3n1.pc01.cls04.6724.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c3n3.pc01.cls04.6700.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c6n3.pc01.cls04.6723.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g1c1n3.pc01.cls04.6715.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c7n2.pc01.cls04.6709.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6700.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c2n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g2c7n2.pc01.cls04.6721.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c6n4.pc01.cls04.6724.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c4n1.pc01.cls04.6721.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6714.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c1n2.pc01.cls04.6709.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g2c3n2.pc01.cls04.6711.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c4n4.pc01.cls04.6715.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c4n4.pc01.cls04.6725.-1.--receive.population
+topology.cheyenne-storm-1.kafka-topic.cheyenne-storm-1.partition-34.latestTimeOffset
+topology.cheyenne-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c7n4.pc01.cls04.6720.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--emit-count.default
+topology.houston-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g1c1n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c4n2.pc01.cls04.6713.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c3n4.pc01.cls04.6714.-1.--emit-count.default
+topology.akron-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c4n4.pc01.cls04.6709.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c2n1.pc01.cls04.6709.-1.--sendqueue.population
+topology.chicago-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c5n3.pc01.cls04.6700.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g2c3n3.pc01.cls04.6706.-1.--emit-count.--system
+topology.fairfax-storm-1.kafka-spout.g2c2n3.pc01.cls04.6700.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c7n1.pc01.cls04.6709.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c1n4.pc01.cls04.6700.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c4n1.pc01.cls04.6723.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g1c4n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c5n4.pc01.cls04.6713.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c5n3.pc01.cls04.6717.-1.--ack-count.default
+topology.chicago-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n1.pc01.cls04.6725.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g1c5n4.pc01.cls04.6702.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c6n3.pc01.cls04.6712.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6708.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c2n2.pc01.cls04.6710.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c1n3.pc01.cls04.6708.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c2n1.pc01.cls04.6725.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c2n1.pc01.cls04.6722.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6726.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n2.pc01.cls04.6720.-1.--emit-count.default
+topology.seattle-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-37.latestEmittedOffset
+topology.fairfax-storm-1.kafka-spout.g2c1n1.pc01.cls04.6701.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g2c2n4.pc01.cls04.6704.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6702.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n1.pc01.cls04.6723.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c3n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.puerto-rico-storm-1.kafka-spout.g1c7n1.pc01.cls04.6712.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g1c6n1.pc01.cls04.6702.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6706.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g1c7n1.pc01.cls04.6722.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c2n4.pc01.cls04.6703.-1.--ack-count.default
+topology.pearl-city-storm-1.kafka-spout.g2c5n2.pc01.cls04.6721.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c1n1.pc01.cls04.6702.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c2n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c2n1.pc01.cls04.6720.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c7n2.pc01.cls04.6711.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c4n1.pc01.cls04.6701.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c3n1.pc01.cls04.6724.-1.--complete-latency.default
+topology.orlando-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c6n3.pc01.cls04.6709.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c3n3.pc01.cls04.6721.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c7n3.pc01.cls04.6724.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c7n1.pc01.cls04.6713.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c2n3.pc01.cls04.6701.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c2n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g2c1n4.pc01.cls04.6700.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g2c6n3.pc01.cls04.6710.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c5n4.pc01.cls04.6703.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c2n3.pc01.cls04.6710.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-30.spoutLag
+topology.san-jose-storm-1.kafka-spout.g2c7n2.pc01.cls04.6723.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6707.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c5n3.pc01.cls04.6700.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c4n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c4n2.pc01.cls04.6705.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c7n3.pc01.cls04.6707.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g1c5n2.pc01.cls04.6714.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g1c5n1.pc01.cls04.6700.-1.--sendqueue.population
+topology.akron-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c2n4.pc01.cls04.6701.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g2c6n2.pc01.cls04.6727.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g2c2n3.pc01.cls04.6707.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6706.-1.--sendqueue.population
+topology.washington-dc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g2c4n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g1c6n2.pc01.cls04.6711.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c4n3.pc01.cls04.6703.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c1n2.pc01.cls04.6722.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c5n2.pc01.cls04.6708.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g2c6n3.pc01.cls04.6720.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c3n2.pc01.cls04.6705.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c4n2.pc01.cls04.6700.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c4n4.pc01.cls04.6704.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n2.pc01.cls04.6701.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g2c6n4.pc01.cls04.6725.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6713.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g1c6n1.pc01.cls04.6723.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--emit-count.default
+topology.pearl-city-storm-1.kafka-spout.g2c5n4.pc01.cls04.6721.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c1n4.pc01.cls04.6700.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c3n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g1c5n3.pc01.cls04.6705.-1.--emit-count.default
+topology.omaha-storm-1.kafka-spout.g1c5n1.pc01.cls04.6705.-1.--emit-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g2c3n3.pc01.cls04.6703.-1.--receive.population
+topology.omaha-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n3.pc01.cls04.6709.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--emit-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6703.-1.--receive.population
+topology.cheyenne-storm-1.kafka-spout.g2c6n2.pc01.cls04.6708.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c7n3.pc01.cls04.6722.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c6n2.pc01.cls04.6725.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g1c1n4.pc01.cls04.6702.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g1c3n3.pc01.cls04.6725.-1.--emit-count.--system
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6705.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g2c6n1.pc01.cls04.6713.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n2.pc01.cls04.6706.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c7n3.pc01.cls04.6708.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g1c2n4.pc01.cls04.6706.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c5n2.pc01.cls04.6710.-1.--ack-count.default
+topology.burbank-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g2c4n4.pc01.cls04.6703.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c2n1.pc01.cls04.6705.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c5n2.pc01.cls04.6703.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g1c5n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c4n1.pc01.cls04.6706.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-topic.fort-worth-storm-1.partition-2.latestCompletedOffset
+topology.chicago-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c5n1.pc01.cls04.6708.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c4n3.pc01.cls04.6706.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g2c4n1.pc01.cls04.6724.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c7n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g1c3n3.pc01.cls04.6712.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c1n1.pc01.cls04.6722.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g1c5n4.pc01.cls04.6704.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g1c5n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.cheyenne-storm-1.kafka-spout.g1c6n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.nyc-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.buffalo-storm-1.kafka-topic.buffalo-storm-1.partition-21.latestTimeOffset
+topology.englewood-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g2c2n4.pc01.cls04.6725.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c7n4.pc01.cls04.6720.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c3n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c5n1.pc01.cls04.6705.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6701.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c5n2.pc01.cls04.6723.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g2c2n4.pc01.cls04.6700.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6706.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c7n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6708.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c4n3.pc01.cls04.6725.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.houston-storm-1.kafka-spout.g2c5n2.pc01.cls04.6707.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c3n1.pc01.cls04.6706.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c5n1.pc01.cls04.6706.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c6n3.pc01.cls04.6706.-1.--complete-latency.default
+topology.akron-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c7n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c7n3.pc01.cls04.6722.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g2c3n2.pc01.cls04.6701.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6722.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6705.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g2c7n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c4n4.pc01.cls04.6712.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c3n3.pc01.cls04.6724.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g2c2n3.pc01.cls04.6701.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c7n2.pc01.cls04.6702.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6706.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c3n3.pc01.cls04.6702.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c2n1.pc01.cls04.6705.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c3n4.pc01.cls04.6722.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c5n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.cheyenne-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g2c1n3.pc01.cls04.6711.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c2n4.pc01.cls04.6726.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c5n4.pc01.cls04.6725.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c5n4.pc01.cls04.6706.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6706.-1.--receive.population
+topology.omaha-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c2n1.pc01.cls04.6705.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c3n2.pc01.cls04.6705.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c5n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g2c1n4.pc01.cls04.6727.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c3n2.pc01.cls04.6704.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c6n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c7n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c2n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c5n2.pc01.cls04.6713.-1.--emit-count.--system
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c3n3.pc01.cls04.6702.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6726.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c6n1.pc01.cls04.6713.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c4n1.pc01.cls04.6704.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--receive.population
+topology.omaha-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g1c5n3.pc01.cls04.6721.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c1n3.pc01.cls04.6715.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--ack-count.default
+topology.springfield-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c3n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c5n2.pc01.cls04.6712.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c6n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g2c2n4.pc01.cls04.6723.-1.--ack-count.default
+topology.springfield-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c4n4.pc01.cls04.6702.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c2n2.pc01.cls04.6709.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g1c7n3.pc01.cls04.6713.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c3n3.pc01.cls04.6701.-1.--receive.population
+topology.tacoma-storm-1.kafka-spout.g2c1n1.pc01.cls04.6706.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g1c7n3.pc01.cls04.6708.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c5n4.pc01.cls04.6702.-1.--ack-count.default
+topology.burbank-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6712.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6709.-1.--emit-count.--system
+topology.anaheim-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n1.pc01.cls04.6727.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g1c1n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c1n3.pc01.cls04.6727.-1.--emit-count.default
+topology.springfield-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c6n3.pc01.cls04.6724.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c4n2.pc01.cls04.6724.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g1c6n2.pc01.cls04.6711.-1.--emit-count.--system
+topology.st-paul-storm-1.kafka-spout.g1c5n2.pc01.cls04.6702.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c1n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c4n1.pc01.cls04.6714.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c2n4.pc01.cls04.6703.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c3n2.pc01.cls04.6704.-1.--emit-count.--system
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c5n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6701.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g2c2n2.pc01.cls04.6701.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c7n4.pc01.cls04.6702.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g1c2n4.pc01.cls04.6723.-1.--sendqueue.population
+topology.nyc-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c5n4.pc01.cls04.6722.-1.--sendqueue.population
+topology.chicago-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c5n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.burbank-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6716.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n2.pc01.cls04.6724.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c6n3.pc01.cls04.6725.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c7n2.pc01.cls04.6725.-1.--ack-count.default
+topology.tacoma-storm-1.kafka-spout.g1c1n3.pc01.cls04.6707.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g1c7n1.pc01.cls04.6701.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g1c4n4.pc01.cls04.6715.-1.--sendqueue.population
+topology.nashville-storm-1.kafka-spout.g1c3n1.pc01.cls04.6709.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c7n2.pc01.cls04.6724.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c3n2.pc01.cls04.6705.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c6n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g1c7n3.pc01.cls04.6715.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-spout.g2c4n2.pc01.cls04.6708.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6709.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c4n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c5n3.pc01.cls04.6717.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c2n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6708.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g1c4n4.pc01.cls04.6708.-1.--emit-count.--system
+topology.anaheim-storm-1.kafka-spout.g1c1n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c7n3.pc01.cls04.6713.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g2c6n2.pc01.cls04.6727.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c4n3.pc01.cls04.6702.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c3n4.pc01.cls04.6706.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c1n3.pc01.cls04.6707.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g2c2n4.pc01.cls04.6711.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c5n1.pc01.cls04.6706.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c5n1.pc01.cls04.6723.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n1.pc01.cls04.6705.-1.--emit-count.default
+topology.puerto-rico-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c7n1.pc01.cls04.6727.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c3n3.pc01.cls04.6721.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g1c3n1.pc01.cls04.6715.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c4n4.pc01.cls04.6703.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c1n2.pc01.cls04.6722.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6712.-1.--emit-count.--system
+topology.stockton-storm-1.kafka-spout.g1c7n2.pc01.cls04.6709.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6702.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g1c7n3.pc01.cls04.6705.-1.--emit-count.--system
+topology.stockton-storm-1.kafka-spout.g1c2n3.pc01.cls04.6713.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c5n4.pc01.cls04.6700.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g2c1n2.pc01.cls04.6702.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c3n4.pc01.cls04.6713.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c3n2.pc01.cls04.6725.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c7n1.pc01.cls04.6711.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g1c1n2.pc01.cls04.6721.-1.--emit-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n2.pc01.cls04.6703.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c2n2.pc01.cls04.6701.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g2c4n4.pc01.cls04.6708.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c2n1.pc01.cls04.6707.-1.--ack-count.default
+topology.nashville-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g2c7n3.pc01.cls04.6709.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c7n2.pc01.cls04.6721.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c7n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n1.pc01.cls04.6710.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g2c5n3.pc01.cls04.6713.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c2n4.pc01.cls04.6705.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6720.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c4n4.pc01.cls04.6705.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c3n4.pc01.cls04.6709.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c5n2.pc01.cls04.6706.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c3n2.pc01.cls04.6712.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6700.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c7n4.pc01.cls04.6725.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g1c6n1.pc01.cls04.6700.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g2c6n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c3n4.pc01.cls04.6714.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c6n2.pc01.cls04.6704.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c3n4.pc01.cls04.6700.-1.--emit-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c2n1.pc01.cls04.6702.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c3n1.pc01.cls04.6707.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c6n4.pc01.cls04.6721.-1.--emit-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c5n1.pc01.cls04.6713.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c3n1.pc01.cls04.6707.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c5n1.pc01.cls04.6700.-1.--ack-count.default
+topology.tacoma-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c2n1.pc01.cls04.6724.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c6n4.pc01.cls04.6727.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c4n1.pc01.cls04.6703.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c2n4.pc01.cls04.6706.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n1.pc01.cls04.6710.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c1n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6710.-1.--complete-latency.default
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-18.spoutLag
+topology.springfield-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g2c3n4.pc01.cls04.6714.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-spout.g2c6n1.pc01.cls04.6707.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g2c4n2.pc01.cls04.6708.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c5n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.chicago-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n4.pc01.cls04.6703.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c5n3.pc01.cls04.6700.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c7n4.pc01.cls04.6723.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6705.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g2c6n1.pc01.cls04.6720.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c5n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g1c5n4.pc01.cls04.6712.-1.--emit-count.--system
+topology.nyc-storm-1.kafka-spout.g2c4n4.pc01.cls04.6725.-1.--ack-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c4n4.pc01.cls04.6727.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g2c6n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g2c5n4.pc01.cls04.6712.-1.--emit-count.default
+topology.cheyenne-storm-1.kafka-spout.g2c2n2.pc01.cls04.6708.-1.--sendqueue.population
+topology.phoenix-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6703.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c1n4.pc01.cls04.6724.-1.--ack-count.default
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c5n2.pc01.cls04.6726.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c3n4.pc01.cls04.6713.-1.--complete-latency.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c4n1.pc01.cls04.6701.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c6n1.pc01.cls04.6723.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c1n4.pc01.cls04.6726.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c2n3.pc01.cls04.6720.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g2c2n4.pc01.cls04.6721.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c4n2.pc01.cls04.6704.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c4n2.pc01.cls04.6713.-1.--complete-latency.default
+topology.cheyenne-storm-1.kafka-spout.g2c4n4.pc01.cls04.6703.-1.--emit-count.--system
+topology.kansas-city-storm-1.kafka-spout.g2c3n3.pc01.cls04.6706.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c5n3.pc01.cls04.6701.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g1c1n3.pc01.cls04.6706.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c7n3.pc01.cls04.6702.-1.--ack-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c5n2.pc01.cls04.6708.-1.--ack-count.default
+topology.houston-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n2.pc01.cls04.6714.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c5n1.pc01.cls04.6718.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n2.pc01.cls04.6723.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c1n3.pc01.cls04.6708.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c2n4.pc01.cls04.6709.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g1c6n4.pc01.cls04.6724.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c6n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6720.-1.--ack-count.default
+topology.buffalo-storm-1.kafka-spout.g2c6n1.pc01.cls04.6704.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c1n2.pc01.cls04.6702.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c5n1.pc01.cls04.6703.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c4n4.pc01.cls04.6704.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c3n1.pc01.cls04.6711.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g1c4n3.pc01.cls04.6706.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c5n4.pc01.cls04.6703.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n1.pc01.cls04.6707.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c1n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c5n1.pc01.cls04.6714.-1.--emit-count.default
+topology.tacoma-storm-1.kafka-spout.g2c4n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6704.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g1c4n4.pc01.cls04.6705.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c4n4.pc01.cls04.6703.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c1n4.pc01.cls04.6703.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g1c2n1.pc01.cls04.6722.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c7n2.pc01.cls04.6723.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n3.pc01.cls04.6709.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c1n3.pc01.cls04.6722.-1.--receive.population
+topology.chicago-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6724.-1.--emit-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c6n1.pc01.cls04.6724.-1.--receive.population
+topology.chicago-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c2n4.pc01.cls04.6723.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g2c1n1.pc01.cls04.6709.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c1n1.pc01.cls04.6707.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g1c6n3.pc01.cls04.6727.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g1c1n3.pc01.cls04.6712.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c5n1.pc01.cls04.6707.-1.--complete-latency.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c1n1.pc01.cls04.6726.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c1n3.pc01.cls04.6707.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6720.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c3n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--receive.population
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-37.spoutLag
+topology.englewood-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c1n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c1n1.pc01.cls04.6706.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6704.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g2c6n1.pc01.cls04.6706.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c5n4.pc01.cls04.6702.-1.--sendqueue.population
+topology.cheyenne-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c1n4.pc01.cls04.6726.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-11.spoutLag
+topology.atlanta-storm-1.kafka-spout.g2c2n3.pc01.cls04.6722.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c4n2.pc01.cls04.6700.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c6n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c4n4.pc01.cls04.6724.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--receive.population
+topology.chicago-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6725.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c3n1.pc01.cls04.6714.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6701.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6700.-1.--emit-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c6n4.pc01.cls04.6711.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c4n1.pc01.cls04.6724.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-topic.phoenix-storm-1.partition-39.latestEmittedOffset
+topology.burbank-storm-1.kafka-spout.g2c2n3.pc01.cls04.6724.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g2c3n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c7n1.pc01.cls04.6706.-1.--emit-count.default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n1.pc01.cls04.6711.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c6n3.pc01.cls04.6725.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c7n2.pc01.cls04.6727.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c2n4.pc01.cls04.6714.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c3n1.pc01.cls04.6726.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6723.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6725.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g2c6n1.pc01.cls04.6706.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c2n3.pc01.cls04.6707.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n3.pc01.cls04.6726.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c2n4.pc01.cls04.6705.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--emit-count.--system
+topology.harrison-storm-1.kafka-spout.g2c1n1.pc01.cls04.6707.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c6n4.pc01.cls04.6720.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g1c3n3.pc01.cls04.6705.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c4n4.pc01.cls04.6721.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c4n3.pc01.cls04.6701.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6713.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g1c6n4.pc01.cls04.6702.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.pearl-city-storm-1.kafka-spout.g2c2n2.pc01.cls04.6721.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c3n2.pc01.cls04.6700.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c3n4.pc01.cls04.6706.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c7n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g2c5n3.pc01.cls04.6713.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c3n3.pc01.cls04.6725.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c7n2.pc01.cls04.6723.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.buffalo-storm-1.kafka-spout.g2c4n4.pc01.cls04.6705.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c5n4.pc01.cls04.6726.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c7n1.pc01.cls04.6720.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c4n4.pc01.cls04.6726.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n1.pc01.cls04.6709.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c3n4.pc01.cls04.6707.-1.--sendqueue.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g1c7n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c2n4.pc01.cls04.6706.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c1n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c6n4.pc01.cls04.6722.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c1n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c3n3.pc01.cls04.6701.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c1n3.pc01.cls04.6722.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c3n1.pc01.cls04.6702.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c4n4.pc01.cls04.6721.-1.--emit-count.--system
+topology.orlando-storm-1.kafka-spout.g1c2n2.pc01.cls04.6708.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c2n3.pc01.cls04.6720.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6703.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c7n1.pc01.cls04.6726.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c1n4.pc01.cls04.6713.-1.--emit-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c3n2.pc01.cls04.6701.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n3.pc01.cls04.6710.-1.--ack-count.default
+topology.pearl-city-storm-1.kafka-spout.g2c6n1.pc01.cls04.6721.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c2n2.pc01.cls04.6727.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c4n2.pc01.cls04.6707.-1.--sendqueue.population
+topology.buffalo-storm-1.kafka-topic.buffalo-storm-1.partition-21.earliestTimeOffset
+topology.elkridge-storm-1.kafka-spout.g2c5n2.pc01.cls04.6703.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g2c7n3.pc01.cls04.6720.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c4n1.pc01.cls04.6704.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.puerto-rico-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c3n1.pc01.cls04.6706.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6707.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c2n2.pc01.cls04.6700.-1.--complete-latency.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c5n2.pc01.cls04.6707.-1.--emit-count.default
+topology.buffalo-storm-1.kafka-spout.g2c2n2.pc01.cls04.6705.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c1n3.pc01.cls04.6723.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g1c4n3.pc01.cls04.6702.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c4n2.pc01.cls04.6706.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c5n4.pc01.cls04.6707.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6706.-1.--complete-latency.default
+topology.pearl-city-storm-1.kafka-spout.g2c5n4.pc01.cls04.6721.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c3n4.pc01.cls04.6727.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c6n1.pc01.cls04.6700.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-topic.phoenix-storm-1.partition-22.spoutLag
+topology.fort-worth-storm-1.kafka-spout.g2c4n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c2n1.pc01.cls04.6712.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c5n3.pc01.cls04.6725.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c2n4.pc01.cls04.6722.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c6n3.pc01.cls04.6714.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n2.pc01.cls04.6703.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c3n4.pc01.cls04.6706.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c1n4.pc01.cls04.6708.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-spout.g2c3n1.pc01.cls04.6700.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c4n4.pc01.cls04.6727.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g2c3n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.nashville-storm-1.kafka-spout.g2c5n3.pc01.cls04.6727.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c7n2.pc01.cls04.6725.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.pearl-city-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c4n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c2n3.pc01.cls04.6726.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c2n2.pc01.cls04.6700.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c2n1.pc01.cls04.6707.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c4n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c1n1.pc01.cls04.6702.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c4n2.pc01.cls04.6724.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g2c1n3.pc01.cls04.6702.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6725.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c5n1.pc01.cls04.6709.-1.--emit-count.default
+topology.seattle-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c2n3.pc01.cls04.6700.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c2n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c6n1.pc01.cls04.6700.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c6n3.pc01.cls04.6721.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6724.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6701.-1.--sendqueue.population
+topology.akron-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6711.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c4n3.pc01.cls04.6710.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6720.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n3.pc01.cls04.6710.-1.--sendqueue.population
+topology.nyc-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c3n3.pc01.cls04.6723.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--ack-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c4n3.pc01.cls04.6712.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g2c4n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c3n4.pc01.cls04.6713.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--sendqueue.population
+topology.cheyenne-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6706.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6709.-1.--ack-count.default
+topology.miami-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n2.pc01.cls04.6723.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c7n2.pc01.cls04.6724.-1.--emit-count.--system
+topology.harrison-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c5n4.pc01.cls04.6725.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6726.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c4n4.pc01.cls04.6723.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c3n2.pc01.cls04.6720.-1.--emit-count.--system
+topology.akron-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c6n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c4n3.pc01.cls04.6722.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n3.pc01.cls04.6709.-1.--receive.population
+topology.cheyenne-storm-1.kafka-spout.g1c3n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6702.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n4.pc01.cls04.6706.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c3n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6700.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c4n1.pc01.cls04.6712.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n1.pc01.cls04.6720.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c3n4.pc01.cls04.6709.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6706.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c5n1.pc01.cls04.6707.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.nashville-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c6n3.pc01.cls04.6706.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c5n1.pc01.cls04.6727.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c4n4.pc01.cls04.6710.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c5n4.pc01.cls04.6710.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6708.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g2c2n4.pc01.cls04.6725.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c7n1.pc01.cls04.6726.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c3n2.pc01.cls04.6712.-1.--emit-count.--system
+topology.fairfax-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c6n2.pc01.cls04.6710.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c6n4.pc01.cls04.6714.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c5n4.pc01.cls04.6711.-1.--complete-latency.default
+topology.cheyenne-storm-1.kafka-topic.cheyenne-storm-1.partition-34.latestEmittedOffset
+topology.burbank-storm-1.kafka-spout.g2c5n2.pc01.cls04.6702.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c1n2.pc01.cls04.6702.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c7n2.pc01.cls04.6712.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g1c3n4.pc01.cls04.6711.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c4n1.pc01.cls04.6701.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c7n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c2n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c3n3.pc01.cls04.6700.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n1.pc01.cls04.6721.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g2c5n3.pc01.cls04.6720.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c7n1.pc01.cls04.6727.-1.--receive.population
+topology.springfield-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c7n3.pc01.cls04.6708.-1.--complete-latency.default
+topology.nashville-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6725.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6723.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.washington-dc-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g2c6n1.pc01.cls04.6715.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c3n2.pc01.cls04.6725.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c6n3.pc01.cls04.6710.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c1n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c3n4.pc01.cls04.6712.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g1c1n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c7n1.pc01.cls04.6714.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g2c1n1.pc01.cls04.6701.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c6n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c5n4.pc01.cls04.6700.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c2n3.pc01.cls04.6713.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c4n2.pc01.cls04.6707.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c2n4.pc01.cls04.6714.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c3n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g1c5n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c6n1.pc01.cls04.6724.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c1n3.pc01.cls04.6716.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c2n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g1c4n3.pc01.cls04.6703.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c1n1.pc01.cls04.6715.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c5n4.pc01.cls04.6711.-1.--emit-count.--system
+topology.springfield-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c2n3.pc01.cls04.6701.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c1n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.tacoma-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c6n1.pc01.cls04.6720.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g1c7n2.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c4n1.pc01.cls04.6702.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c5n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c4n1.pc01.cls04.6711.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c4n4.pc01.cls04.6725.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c1n2.pc01.cls04.6700.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c3n3.pc01.cls04.6712.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c2n4.pc01.cls04.6726.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c3n2.pc01.cls04.6726.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c3n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g2c3n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c5n2.pc01.cls04.6723.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c1n2.pc01.cls04.6721.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c5n1.pc01.cls04.6727.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6726.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c5n1.pc01.cls04.6706.-1.--receive.population
+topology.nashville-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c6n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c7n3.pc01.cls04.6720.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.buffalo-storm-1.kafka-spout.g2c6n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c7n3.pc01.cls04.6702.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c6n2.pc01.cls04.6727.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g1c4n4.pc01.cls04.6723.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6713.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c6n2.pc01.cls04.6720.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c1n4.pc01.cls04.6726.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g1c2n1.pc01.cls04.6705.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n2.pc01.cls04.6700.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6701.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g1c5n3.pc01.cls04.6717.-1.--complete-latency.default
+topology.houston-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c7n3.pc01.cls04.6726.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c3n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c7n4.pc01.cls04.6702.-1.--emit-count.default
+topology.pearl-city-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c3n1.pc01.cls04.6700.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c7n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g1c7n4.pc01.cls04.6706.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c1n4.pc01.cls04.6721.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c7n4.pc01.cls04.6712.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c2n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c2n2.pc01.cls04.6727.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-spout.g2c2n1.pc01.cls04.6708.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c5n4.pc01.cls04.6708.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c5n2.pc01.cls04.6708.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6725.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c6n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c6n2.pc01.cls04.6701.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g2c1n2.pc01.cls04.6727.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g1c1n4.pc01.cls04.6720.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c4n2.pc01.cls04.6712.-1.--complete-latency.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c1n1.pc01.cls04.6706.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c3n1.pc01.cls04.6726.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c2n4.pc01.cls04.6710.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c1n3.pc01.cls04.6711.-1.--ack-count.default
+topology.tacoma-storm-1.kafka-spout.g2c3n1.pc01.cls04.6705.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c2n4.pc01.cls04.6713.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g2c1n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c6n1.pc01.cls04.6709.-1.--sendqueue.population
+topology.houston-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c2n3.pc01.cls04.6700.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g1c5n4.pc01.cls04.6704.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c5n1.pc01.cls04.6707.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c3n2.pc01.cls04.6725.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c2n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g1c6n1.pc01.cls04.6700.-1.--sendqueue.population
+topology.houston-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--ack-count.default
+topology.seattle-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c3n2.pc01.cls04.6704.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c1n3.pc01.cls04.6703.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c2n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g2c2n2.pc01.cls04.6711.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c5n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c7n3.pc01.cls04.6705.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c5n2.pc01.cls04.6724.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c7n4.pc01.cls04.6713.-1.--receive.population
+topology.pearl-city-storm-1.kafka-spout.g2c1n1.pc01.cls04.6713.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6708.-1.--ack-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c7n3.pc01.cls04.6727.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g1c1n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c4n1.pc01.cls04.6700.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6700.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6724.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c7n3.pc01.cls04.6727.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n3.pc01.cls04.6710.-1.--emit-count.default
+topology.puerto-rico-storm-1.kafka-spout.g2c6n1.pc01.cls04.6702.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c3n2.pc01.cls04.6705.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6723.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c2n3.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c3n4.pc01.cls04.6707.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c2n3.pc01.cls04.6712.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c1n1.pc01.cls04.6706.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c4n2.pc01.cls04.6709.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c7n1.pc01.cls04.6726.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-30.latestTimeOffset
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c5n2.pc01.cls04.6712.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c6n2.pc01.cls04.6709.-1.--emit-count.--system
+topology.springfield-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c2n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c2n1.pc01.cls04.6713.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c6n1.pc01.cls04.6705.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c2n3.pc01.cls04.6724.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g1c2n4.pc01.cls04.6720.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c2n4.pc01.cls04.6704.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c3n3.pc01.cls04.6727.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c1n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c2n1.pc01.cls04.6709.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c6n3.pc01.cls04.6711.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c4n1.pc01.cls04.6701.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6712.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g1c3n4.pc01.cls04.6722.-1.--receive.population
+topology.springfield-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g1c2n1.pc01.cls04.6713.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c3n2.pc01.cls04.6724.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n4.pc01.cls04.6706.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-35.latestTimeOffset
+topology.stockton-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c7n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c4n2.pc01.cls04.6710.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c3n1.pc01.cls04.6726.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n1.pc01.cls04.6705.-1.--ack-count.default
+topology.houston-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n3.pc01.cls04.6720.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c4n4.pc01.cls04.6725.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c1n4.pc01.cls04.6711.-1.--emit-count.--system
+topology.harrison-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c5n2.pc01.cls04.6702.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c2n3.pc01.cls04.6725.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c4n2.pc01.cls04.6708.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c7n3.pc01.cls04.6701.-1.--ack-count.default
+topology.harrison-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c3n1.pc01.cls04.6700.-1.--emit-count.--system
+topology.kansas-city-storm-1.kafka-spout.g2c6n2.pc01.cls04.6701.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c4n3.pc01.cls04.6722.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g1c3n1.pc01.cls04.6713.-1.--ack-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n1.pc01.cls04.6723.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n1.pc01.cls04.6709.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g2c6n1.pc01.cls04.6722.-1.--emit-count.--system
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6704.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g2c7n1.pc01.cls04.6720.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c6n1.pc01.cls04.6703.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g2c4n1.pc01.cls04.6724.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6701.-1.--sendqueue.population
+topology.akron-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c3n3.pc01.cls04.6702.-1.--complete-latency.default
+topology.nyc-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6710.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c5n1.pc01.cls04.6705.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c4n4.pc01.cls04.6712.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c6n3.pc01.cls04.6725.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c6n2.pc01.cls04.6714.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c4n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g1c1n4.pc01.cls04.6711.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c6n2.pc01.cls04.6714.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c1n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c4n2.pc01.cls04.6722.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c5n2.pc01.cls04.6712.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6726.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--complete-latency.default
+topology.miami-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c5n4.pc01.cls04.6720.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c6n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g2c2n4.pc01.cls04.6704.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c3n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c6n4.pc01.cls04.6701.-1.--emit-count.--system
+topology.fairfax-storm-1.kafka-spout.g1c2n2.pc01.cls04.6726.-1.--emit-count.default
+topology.houston-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c4n1.pc01.cls04.6725.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c7n3.pc01.cls04.6722.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c6n3.pc01.cls04.6725.-1.--ack-count.default
+topology.burbank-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.omaha-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c2n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g1c4n1.pc01.cls04.6709.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c5n4.pc01.cls04.6706.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.cheyenne-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c6n2.pc01.cls04.6713.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c5n2.pc01.cls04.6700.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n3.pc01.cls04.6703.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c2n1.pc01.cls04.6726.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c3n1.pc01.cls04.6709.-1.--emit-count.--system
+topology.akron-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n3.pc01.cls04.6713.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c3n3.pc01.cls04.6724.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c6n2.pc01.cls04.6714.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6702.-1.--ack-count.default
+topology.miami-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c2n3.pc01.cls04.6711.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c3n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6723.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6702.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n4.pc01.cls04.6712.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-spout.g2c7n1.pc01.cls04.6705.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c3n4.pc01.cls04.6712.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.omaha-storm-1.kafka-spout.g2c6n1.pc01.cls04.6715.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c2n4.pc01.cls04.6723.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c3n1.pc01.cls04.6724.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c3n1.pc01.cls04.6726.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g2c2n2.pc01.cls04.6720.-1.--emit-count.default
+topology.pearl-city-storm-1.kafka-topic.pearl-city-storm-1.partition-7.latestEmittedOffset
+topology.akron-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c4n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c5n2.pc01.cls04.6721.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c6n4.pc01.cls04.6721.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c7n1.pc01.cls04.6724.-1.--complete-latency.default
+topology.akron-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.buffalo-storm-1.kafka-spout.g1c5n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c5n2.pc01.cls04.6708.-1.--emit-count.--system
+topology.tacoma-storm-1.kafka-spout.g2c2n4.pc01.cls04.6713.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g1c1n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n4.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c1n3.pc01.cls04.6708.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c4n4.pc01.cls04.6705.-1.--ack-count.default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n1.pc01.cls04.6727.-1.--receive.population
+topology.tacoma-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c1n4.pc01.cls04.6702.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g1c7n3.pc01.cls04.6704.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.chicago-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c5n2.pc01.cls04.6721.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g2c4n2.pc01.cls04.6712.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c5n4.pc01.cls04.6708.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c6n3.pc01.cls04.6724.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c7n3.pc01.cls04.6707.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6704.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6720.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c2n2.pc01.cls04.6712.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c4n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c3n1.pc01.cls04.6723.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c2n1.pc01.cls04.6706.-1.--ack-count.default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.nashville-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c6n1.pc01.cls04.6707.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g2c5n2.pc01.cls04.6703.-1.--receive.population
+topology.nashville-storm-1.kafka-spout.g2c1n1.pc01.cls04.6712.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c4n2.pc01.cls04.6715.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c2n4.pc01.cls04.6704.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c5n2.pc01.cls04.6713.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c6n2.pc01.cls04.6702.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c4n1.pc01.cls04.6713.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n3.pc01.cls04.6724.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6723.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c4n1.pc01.cls04.6727.-1.--receive.population
+topology.nyc-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c2n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.buffalo-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c7n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c3n4.pc01.cls04.6711.-1.--emit-count.default
+topology.springfield-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6715.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6727.-1.--ack-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.houston-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g1c6n2.pc01.cls04.6707.-1.--sendqueue.population
+topology.buffalo-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c6n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c2n4.pc01.cls04.6722.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c7n4.pc01.cls04.6720.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g2c4n1.pc01.cls04.6713.-1.--ack-count.default
+topology.buffalo-storm-1.kafka-spout.g1c5n3.pc01.cls04.6707.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c4n4.pc01.cls04.6707.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c1n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c2n4.pc01.cls04.6701.-1.--receive.population
+topology.akron-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.omaha-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c4n4.pc01.cls04.6726.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c1n4.pc01.cls04.6700.-1.--emit-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c2n1.pc01.cls04.6719.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g1c6n1.pc01.cls04.6724.-1.--emit-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c5n1.pc01.cls04.6723.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g2c1n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c4n3.pc01.cls04.6722.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c1n2.pc01.cls04.6727.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g2c7n4.pc01.cls04.6725.-1.--ack-count.default
+topology.nyc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g2c7n4.pc01.cls04.6721.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c1n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g1c4n3.pc01.cls04.6711.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c7n2.pc01.cls04.6721.-1.--emit-count.--system
+topology.chicago-storm-1.kafka-spout.g1c4n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c3n4.pc01.cls04.6706.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g1c4n1.pc01.cls04.6701.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c1n4.pc01.cls04.6727.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g2c4n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c1n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c4n4.pc01.cls04.6712.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6725.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g2c1n4.pc01.cls04.6723.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c7n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c1n2.pc01.cls04.6700.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c4n2.pc01.cls04.6701.-1.--emit-count.default
+topology.puerto-rico-storm-1.kafka-spout.g2c2n4.pc01.cls04.6725.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n1.pc01.cls04.6705.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c6n3.pc01.cls04.6702.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c1n2.pc01.cls04.6701.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c1n2.pc01.cls04.6715.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g1c3n1.pc01.cls04.6726.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c7n4.pc01.cls04.6706.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c5n2.pc01.cls04.6726.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-0.spoutLag
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6720.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-spout.g1c4n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c3n2.pc01.cls04.6701.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c5n4.pc01.cls04.6726.-1.--emit-count.--system
+topology.omaha-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c6n3.pc01.cls04.6713.-1.--emit-count.default
+topology.pearl-city-storm-1.kafka-spout.g2c6n1.pc01.cls04.6721.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6707.-1.--ack-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n1.pc01.cls04.6702.-1.--emit-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c4n4.pc01.cls04.6727.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c1n4.pc01.cls04.6709.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g1c4n3.pc01.cls04.6712.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c1n1.pc01.cls04.6702.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n2.pc01.cls04.6723.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g1c4n4.pc01.cls04.6713.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c2n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6700.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c3n2.pc01.cls04.6707.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c4n3.pc01.cls04.6709.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g1c5n1.pc01.cls04.6721.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6705.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c5n3.pc01.cls04.6705.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c4n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c1n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c5n4.pc01.cls04.6721.-1.--ack-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c3n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6702.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6720.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g1c4n4.pc01.cls04.6720.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6725.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c7n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c4n2.pc01.cls04.6720.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c4n4.pc01.cls04.6726.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n2.pc01.cls04.6720.-1.--receive.population
+topology.buffalo-storm-1.kafka-spout.g2c4n3.pc01.cls04.6704.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c3n3.pc01.cls04.6709.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c6n2.pc01.cls04.6727.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c3n2.pc01.cls04.6709.-1.--ack-count.default
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g2c2n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c7n1.pc01.cls04.6720.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c5n4.pc01.cls04.6723.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g1c2n3.pc01.cls04.6708.-1.--emit-count.default
+topology.houston-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c6n1.pc01.cls04.6703.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6726.-1.--ack-count.default
+topology.springfield-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c2n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.miami-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c4n4.pc01.cls04.6708.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c4n4.pc01.cls04.6711.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c1n2.pc01.cls04.6702.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c4n4.pc01.cls04.6707.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n3.pc01.cls04.6712.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c4n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n3.pc01.cls04.6710.-1.--emit-count.default
+topology.omaha-storm-1.kafka-spout.g2c5n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c6n4.pc01.cls04.6724.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c4n4.pc01.cls04.6704.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6707.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c2n2.pc01.cls04.6701.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c5n1.pc01.cls04.6723.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c2n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g2c7n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c4n4.pc01.cls04.6702.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c3n3.pc01.cls04.6723.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g1c3n3.pc01.cls04.6712.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c6n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c2n3.pc01.cls04.6712.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c3n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.houston-storm-1.kafka-spout.g1c7n1.pc01.cls04.6704.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6713.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c4n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c5n3.pc01.cls04.6726.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c6n1.pc01.cls04.6722.-1.--receive.population
+topology.omaha-storm-1.kafka-spout.g2c4n2.pc01.cls04.6712.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-topic.fort-worth-storm-1.partition-20.earliestTimeOffset
+topology.chicago-storm-1.kafka-spout.g1c7n1.pc01.cls04.6704.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c5n4.pc01.cls04.6708.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c5n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c2n4.pc01.cls04.6709.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g2c6n2.pc01.cls04.6704.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g2c4n4.pc01.cls04.6701.-1.--complete-latency.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c3n3.pc01.cls04.6700.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6704.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c4n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c1n1.pc01.cls04.6703.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c2n1.pc01.cls04.6721.-1.--emit-count.default
+topology.tacoma-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c3n1.pc01.cls04.6723.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g2c1n4.pc01.cls04.6705.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c2n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c2n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.nashville-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c6n4.pc01.cls04.6702.-1.--emit-count.--system
+topology.omaha-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--receive.population
+topology.pearl-city-storm-1.kafka-spout.g1c3n4.pc01.cls04.6721.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c7n1.pc01.cls04.6727.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n2.pc01.cls04.6704.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c7n3.pc01.cls04.6712.-1.--receive.population
+topology.cheyenne-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c4n4.pc01.cls04.6724.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c3n1.pc01.cls04.6726.-1.--emit-count.--system
+topology.englewood-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c2n4.pc01.cls04.6706.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c2n4.pc01.cls04.6713.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c4n1.pc01.cls04.6707.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c2n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c7n1.pc01.cls04.6727.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c5n3.pc01.cls04.6704.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c3n4.pc01.cls04.6722.-1.--sendqueue.population
+topology.tacoma-storm-1.kafka-spout.g2c4n4.pc01.cls04.6712.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c2n3.pc01.cls04.6720.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c4n4.pc01.cls04.6727.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g1c2n1.pc01.cls04.6725.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c6n3.pc01.cls04.6706.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g1c7n3.pc01.cls04.6703.-1.--emit-count.--system
+topology.st-paul-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g2c5n4.pc01.cls04.6721.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g1c6n1.pc01.cls04.6720.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c4n1.pc01.cls04.6720.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c6n3.pc01.cls04.6720.-1.--emit-count.--system
+topology.burbank-storm-1.kafka-spout.g1c5n4.pc01.cls04.6711.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c4n4.pc01.cls04.6702.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c5n1.pc01.cls04.6721.-1.--ack-count.default
+topology.seattle-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n1.pc01.cls04.6720.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-topic.fort-worth-storm-1.partition-62.latestCompletedOffset
+topology.chicago-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n3.pc01.cls04.6710.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g1c7n4.pc01.cls04.6720.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g1c4n4.pc01.cls04.6714.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n3.pc01.cls04.6710.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g1c5n1.pc01.cls04.6705.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c7n4.pc01.cls04.6711.-1.--sendqueue.population
+topology.nashville-storm-1.kafka-spout.g2c6n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n4.pc01.cls04.6712.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6722.-1.--sendqueue.population
+topology.phoenix-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n4.pc01.cls04.6723.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c5n1.pc01.cls04.6710.-1.--ack-count.default
+topology.akron-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c1n3.pc01.cls04.6710.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c3n3.pc01.cls04.6700.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c7n4.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c6n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c2n4.pc01.cls04.6726.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c3n3.pc01.cls04.6701.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g2c3n3.pc01.cls04.6712.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c6n2.pc01.cls04.6708.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c2n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.tacoma-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c5n1.pc01.cls04.6701.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c5n1.pc01.cls04.6726.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g1c6n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g2c6n4.pc01.cls04.6703.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c7n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.cheyenne-storm-1.kafka-spout.g2c7n4.pc01.cls04.6726.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c1n2.pc01.cls04.6702.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c7n2.pc01.cls04.6703.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c4n3.pc01.cls04.6711.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c1n4.pc01.cls04.6723.-1.--sendqueue.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g2c3n1.pc01.cls04.6714.-1.--ack-count.default
+topology.omaha-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c4n4.pc01.cls04.6714.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c3n2.pc01.cls04.6707.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6702.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g2c6n1.pc01.cls04.6713.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c2n2.pc01.cls04.6711.-1.--ack-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.omaha-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c1n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c5n2.pc01.cls04.6710.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g1c6n4.pc01.cls04.6700.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6724.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-spout.g1c4n4.pc01.cls04.6710.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c1n3.pc01.cls04.6725.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c4n1.pc01.cls04.6700.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c1n4.pc01.cls04.6726.-1.--complete-latency.default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-29.earliestTimeOffset
+topology.seattle-storm-1.kafka-spout.g2c2n1.pc01.cls04.6714.-1.--sendqueue.population
+topology.chicago-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c7n1.pc01.cls04.6727.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c1n4.pc01.cls04.6715.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g2c1n3.pc01.cls04.6711.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6720.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c5n2.pc01.cls04.6720.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g1c6n1.pc01.cls04.6724.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6722.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c3n4.pc01.cls04.6713.-1.--ack-count.default
+topology.atlanta-storm-1.kafka-spout.g1c6n4.pc01.cls04.6704.-1.--emit-count.default
+topology.puerto-rico-storm-1.kafka-spout.g2c1n1.pc01.cls04.6708.-1.--emit-count.default
+topology.houston-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n2.pc01.cls04.6707.-1.--emit-count.default
+topology.cheyenne-storm-1.kafka-spout.g1c6n1.pc01.cls04.6704.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g1c5n4.pc01.cls04.6711.-1.--emit-count.default
+topology.tacoma-storm-1.kafka-spout.g2c6n1.pc01.cls04.6708.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6711.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c3n4.pc01.cls04.6726.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g2c4n4.pc01.cls04.6708.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g1c5n4.pc01.cls04.6710.-1.--ack-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c3n2.pc01.cls04.6701.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g2c7n1.pc01.cls04.6707.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g2c6n3.pc01.cls04.6709.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g1c2n4.pc01.cls04.6722.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6702.-1.--emit-count.default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c2n4.pc01.cls04.6722.-1.--receive.population
+topology.omaha-storm-1.kafka-spout.g2c4n2.pc01.cls04.6712.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c7n4.pc01.cls04.6706.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c7n1.pc01.cls04.6726.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c2n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6725.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c5n2.pc01.cls04.6706.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g1c5n2.pc01.cls04.6704.-1.--emit-count.--system
+topology.kansas-city-storm-1.kafka-spout.g1c3n3.pc01.cls04.6702.-1.--receive.population
+topology.nyc-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c6n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g2c7n1.pc01.cls04.6715.-1.--receive.population
+topology.kansas-city-storm-1.kafka-spout.g1c2n2.pc01.cls04.6710.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c5n2.pc01.cls04.6704.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-topic.fort-worth-storm-1.partition-2.latestEmittedOffset
+topology.orlando-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c6n2.pc01.cls04.6720.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g1c5n4.pc01.cls04.6703.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c6n1.pc01.cls04.6705.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6706.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c2n1.pc01.cls04.6724.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c3n4.pc01.cls04.6709.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g2c7n4.pc01.cls04.6725.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g1c4n2.pc01.cls04.6701.-1.--ack-count.default
+topology.tacoma-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6705.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c5n2.pc01.cls04.6724.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g2c6n1.pc01.cls04.6727.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g1c2n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g1c5n4.pc01.cls04.6711.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c2n3.pc01.cls04.6701.-1.--sendqueue.population
+topology.chicago-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c3n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.nashville-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c7n4.pc01.cls04.6722.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g1c7n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c3n2.pc01.cls04.6725.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g1c5n3.pc01.cls04.6711.-1.--ack-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c2n3.pc01.cls04.6704.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6706.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c6n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c5n2.pc01.cls04.6727.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c4n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--sendqueue.population
+topology.chicago-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n2.pc01.cls04.6711.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c1n2.pc01.cls04.6727.-1.--emit-count.default
+topology.buffalo-storm-1.kafka-spout.g2c4n4.pc01.cls04.6705.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c6n4.pc01.cls04.6701.-1.--receive.population
+topology.buffalo-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c7n1.pc01.cls04.6702.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c3n3.pc01.cls04.6724.-1.--receive.population
+topology.orlando-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.pearl-city-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n2.pc01.cls04.6720.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n3.pc01.cls04.6703.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--sendqueue.population
+topology.burbank-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c2n4.pc01.cls04.6711.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n4.pc01.cls04.6714.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n2.pc01.cls04.6706.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n2.pc01.cls04.6701.-1.--receive.population
+topology.omaha-storm-1.kafka-spout.g2c5n2.pc01.cls04.6701.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6722.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n1.pc01.cls04.6727.-1.--ack-count.default
+topology.cheyenne-storm-1.kafka-spout.g2c6n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c2n4.pc01.cls04.6711.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c4n3.pc01.cls04.6711.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c6n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c4n3.pc01.cls04.6720.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c3n1.pc01.cls04.6712.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6704.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c1n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g1c4n4.pc01.cls04.6708.-1.--receive.population
+topology.nyc-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.nashville-storm-1.kafka-spout.g1c3n1.pc01.cls04.6709.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c7n4.pc01.cls04.6708.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6706.-1.--ack-count.default
+topology.burbank-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n4.pc01.cls04.6711.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c2n2.pc01.cls04.6722.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g1c5n3.pc01.cls04.6712.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c3n4.pc01.cls04.6706.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g2c6n2.pc01.cls04.6700.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c6n3.pc01.cls04.6703.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c2n2.pc01.cls04.6727.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c3n2.pc01.cls04.6704.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n4.pc01.cls04.6711.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g2c3n2.pc01.cls04.6710.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g1c2n2.pc01.cls04.6727.-1.--sendqueue.population
+topology.tacoma-storm-1.kafka-spout.g1c3n4.pc01.cls04.6714.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.washington-dc-storm-1.kafka-topic.washington-dc-storm-1.partition-7.spoutLag
+topology.seattle-storm-1.kafka-spout.g2c4n2.pc01.cls04.6709.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c3n2.pc01.cls04.6725.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c3n1.pc01.cls04.6727.-1.--emit-count.--system
+topology.orlando-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g2c6n1.pc01.cls04.6707.-1.--complete-latency.default
+topology.buffalo-storm-1.kafka-spout.g2c4n3.pc01.cls04.6704.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g1c4n2.pc01.cls04.6708.-1.--receive.population
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6705.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6727.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c6n4.pc01.cls04.6722.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g1c7n4.pc01.cls04.6705.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g2c2n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c5n4.pc01.cls04.6723.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6701.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g1c2n3.pc01.cls04.6707.-1.--emit-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c6n3.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c5n2.pc01.cls04.6724.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c5n1.pc01.cls04.6708.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g1c2n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.omaha-storm-1.kafka-spout.g2c3n1.pc01.cls04.6714.-1.--receive.population
+topology.pearl-city-storm-1.kafka-spout.g2c2n2.pc01.cls04.6721.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c2n3.pc01.cls04.6700.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g2c5n3.pc01.cls04.6711.-1.--emit-count.--system
+topology.pearl-city-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c6n1.pc01.cls04.6700.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c2n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6706.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g1c1n4.pc01.cls04.6726.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g2c7n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g2c6n1.pc01.cls04.6704.-1.--ack-count.default
+topology.buffalo-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c4n1.pc01.cls04.6701.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g1c7n1.pc01.cls04.6706.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g1c6n2.pc01.cls04.6707.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g2c6n1.pc01.cls04.6705.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g2c6n4.pc01.cls04.6705.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c4n2.pc01.cls04.6726.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--sendqueue.population
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c1n1.pc01.cls04.6707.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c2n1.pc01.cls04.6725.-1.--emit-count.--system
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6709.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n2.pc01.cls04.6722.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c7n3.pc01.cls04.6704.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6720.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g1c3n1.pc01.cls04.6721.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c3n4.pc01.cls04.6722.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g1c6n1.pc01.cls04.6702.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c3n4.pc01.cls04.6700.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g1c2n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--emit-count.--system
+topology.puerto-rico-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c6n2.pc01.cls04.6714.-1.--receive.population
+topology.tacoma-storm-1.kafka-spout.g2c5n2.pc01.cls04.6715.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6703.-1.--complete-latency.default
+topology.tacoma-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.buffalo-storm-1.kafka-spout.g1c4n4.pc01.cls04.6714.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c1n2.pc01.cls04.6725.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c3n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n4.pc01.cls04.6705.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c7n1.pc01.cls04.6725.-1.--sendqueue.population
+topology.houston-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6709.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n3.pc01.cls04.6706.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g1c2n3.pc01.cls04.6714.-1.--emit-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c6n1.pc01.cls04.6720.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c1n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c5n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c1n3.pc01.cls04.6714.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c6n1.pc01.cls04.6710.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6702.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c1n4.pc01.cls04.6727.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c3n1.pc01.cls04.6722.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c6n3.pc01.cls04.6706.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g1c3n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c7n3.pc01.cls04.6725.-1.--ack-count.default
+topology.omaha-storm-1.kafka-spout.g2c6n1.pc01.cls04.6710.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g1c1n2.pc01.cls04.6702.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g1c1n4.pc01.cls04.6700.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c6n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c6n4.pc01.cls04.6724.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c7n3.pc01.cls04.6705.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6700.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c2n3.pc01.cls04.6725.-1.--receive.population
+topology.akron-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c4n1.pc01.cls04.6724.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g2c1n1.pc01.cls04.6715.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g1c5n2.pc01.cls04.6725.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c1n3.pc01.cls04.6701.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c1n1.pc01.cls04.6726.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c6n4.pc01.cls04.6701.-1.--emit-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c4n4.pc01.cls04.6720.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.pearl-city-storm-1.kafka-topic.pearl-city-storm-1.partition-7.earliestTimeOffset
+topology.san-jose-storm-1.kafka-spout.g2c3n3.pc01.cls04.6726.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g1c6n1.pc01.cls04.6713.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c4n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c2n4.pc01.cls04.6714.-1.--sendqueue.population
+topology.omaha-storm-1.kafka-spout.g2c1n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.buffalo-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g2c4n2.pc01.cls04.6724.-1.--emit-count.default
+topology.buffalo-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c4n4.pc01.cls04.6725.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c7n3.pc01.cls04.6709.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g1c5n3.pc01.cls04.6711.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c5n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c1n4.pc01.cls04.6721.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c7n4.pc01.cls04.6726.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c6n3.pc01.cls04.6700.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g1c2n4.pc01.cls04.6710.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c7n2.pc01.cls04.6700.-1.--ack-count.default
+topology.nyc-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c6n1.pc01.cls04.6725.-1.--ack-count.default
+topology.pearl-city-storm-1.kafka-spout.g1c5n2.pc01.cls04.6721.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c7n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c4n3.pc01.cls04.6706.-1.--sendqueue.population
+topology.springfield-storm-1.kafka-spout.g1c7n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c3n3.pc01.cls04.6709.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c5n1.pc01.cls04.6710.-1.--emit-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c6n2.pc01.cls04.6710.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g2c5n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c2n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6714.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g1c1n2.pc01.cls04.6706.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c6n4.pc01.cls04.6703.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6703.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c1n1.pc01.cls04.6710.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.nashville-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6714.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c4n2.pc01.cls04.6705.-1.--emit-count.default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6716.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c7n3.pc01.cls04.6708.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c1n1.pc01.cls04.6700.-1.--emit-count.--system
+topology.washington-dc-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n4.pc01.cls04.6712.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g1c4n2.pc01.cls04.6722.-1.--complete-latency.default
+topology.nashville-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c2n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6710.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c3n3.pc01.cls04.6702.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g2c4n3.pc01.cls04.6723.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c6n4.pc01.cls04.6721.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c4n2.pc01.cls04.6704.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g2c4n3.pc01.cls04.6727.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c5n1.pc01.cls04.6705.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6711.-1.--ack-count.default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g2c3n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g1c3n1.pc01.cls04.6720.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c4n2.pc01.cls04.6723.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c4n2.pc01.cls04.6713.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g1c1n4.pc01.cls04.6724.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n4.pc01.cls04.6707.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c5n3.pc01.cls04.6711.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c7n1.pc01.cls04.6726.-1.--sendqueue.population
+topology.fort-worth-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c6n2.pc01.cls04.6710.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c3n3.pc01.cls04.6725.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g2c7n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c2n2.pc01.cls04.6702.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6707.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c4n2.pc01.cls04.6703.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c7n2.pc01.cls04.6711.-1.--emit-count.--system
+topology.pearl-city-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n3.pc01.cls04.6707.-1.--emit-count.--system
+topology.phoenix-storm-1.kafka-spout.g1c4n2.pc01.cls04.6714.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.harrison-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c5n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c1n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.omaha-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c6n3.pc01.cls04.6705.-1.--complete-latency.default
+topology.cheyenne-storm-1.kafka-spout.g2c6n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g2c7n1.pc01.cls04.6711.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c3n1.pc01.cls04.6713.-1.--receive.population
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-18.latestEmittedOffset
+topology.burbank-storm-1.kafka-spout.g2c5n4.pc01.cls04.6707.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g1c4n3.pc01.cls04.6708.-1.--receive.population
+topology.pearl-city-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6715.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c1n2.pc01.cls04.6727.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c2n1.pc01.cls04.6727.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c3n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c2n3.pc01.cls04.6721.-1.--complete-latency.default
+topology.stockton-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c6n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n1.pc01.cls04.6715.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n2.pc01.cls04.6700.-1.--emit-count.--system
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c5n4.pc01.cls04.6711.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c6n3.pc01.cls04.6700.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c6n2.pc01.cls04.6704.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c5n3.pc01.cls04.6713.-1.--receive.population
+topology.tacoma-storm-1.kafka-spout.g1c3n4.pc01.cls04.6714.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c5n4.pc01.cls04.6723.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6714.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g2c5n1.pc01.cls04.6726.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c3n4.pc01.cls04.6713.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6703.-1.--emit-count.default
+topology.buffalo-storm-1.kafka-spout.g1c4n4.pc01.cls04.6706.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c4n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.buffalo-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g1c5n2.pc01.cls04.6708.-1.--emit-count.--system
+topology.springfield-storm-1.kafka-spout.g1c1n1.pc01.cls04.6701.-1.--receive.population
+topology.washington-dc-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c5n1.pc01.cls04.6718.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c6n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g2c1n2.pc01.cls04.6708.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c5n2.pc01.cls04.6727.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c4n4.pc01.cls04.6725.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g1c6n1.pc01.cls04.6726.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c6n4.pc01.cls04.6703.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--sendqueue.population
+topology.stockton-storm-1.kafka-spout.g1c4n1.pc01.cls04.6727.-1.--receive.population
+topology.harrison-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c1n3.pc01.cls04.6705.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g2c1n1.pc01.cls04.6701.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c6n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.pearl-city-storm-1.kafka-spout.g2c3n3.pc01.cls04.6721.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c3n3.pc01.cls04.6705.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c6n4.pc01.cls04.6709.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g1c5n1.pc01.cls04.6711.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--emit-count.--system
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6707.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n4.pc01.cls04.6720.-1.--complete-latency.default
+topology.tacoma-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c2n2.pc01.cls04.6713.-1.--complete-latency.default
+topology.nashville-storm-1.kafka-spout.g2c5n3.pc01.cls04.6727.-1.--sendqueue.population
+topology.phoenix-storm-1.kafka-spout.g2c7n3.pc01.cls04.6727.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c3n1.pc01.cls04.6706.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g2c7n3.pc01.cls04.6726.-1.--emit-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--sendqueue.population
+topology.englewood-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c4n4.pc01.cls04.6720.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c1n1.pc01.cls04.6703.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c3n3.pc01.cls04.6720.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c3n4.pc01.cls04.6700.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c4n4.pc01.cls04.6723.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c7n1.pc01.cls04.6724.-1.--ack-count.default
+topology.burbank-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c5n1.pc01.cls04.6714.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6726.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g1c5n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c3n4.pc01.cls04.6722.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c5n4.pc01.cls04.6700.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6713.-1.--receive.population
+topology.chicago-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c6n3.pc01.cls04.6720.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g1c2n4.pc01.cls04.6707.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c1n1.pc01.cls04.6724.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c7n2.pc01.cls04.6703.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c5n2.pc01.cls04.6723.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6725.-1.--ack-count.default
+topology.cheyenne-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c5n2.pc01.cls04.6710.-1.--complete-latency.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g2c3n1.pc01.cls04.6702.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--ack-count.default
+topology.pearl-city-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c3n2.pc01.cls04.6723.-1.--emit-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c4n1.pc01.cls04.6723.-1.--sendqueue.population
+topology.washington-dc-storm-1.kafka-spout.g1c5n2.pc01.cls04.6705.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g1c7n3.pc01.cls04.6709.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g2c1n1.pc01.cls04.6708.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c3n4.pc01.cls04.6714.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g2c4n2.pc01.cls04.6709.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6704.-1.--emit-count.default
+topology.tacoma-storm-1.kafka-spout.g2c5n2.pc01.cls04.6706.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c3n4.pc01.cls04.6710.-1.--emit-count.default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6725.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c1n3.pc01.cls04.6703.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c5n3.pc01.cls04.6724.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n4.pc01.cls04.6702.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g2c5n2.pc01.cls04.6707.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6709.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g1c3n1.pc01.cls04.6724.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6707.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c7n4.pc01.cls04.6722.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6709.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c7n4.pc01.cls04.6725.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c2n2.pc01.cls04.6712.-1.--emit-count.default
+topology.nashville-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n1.pc01.cls04.6713.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n4.pc01.cls04.6723.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6712.-1.--emit-count.--system
+topology.anaheim-storm-1.kafka-spout.g1c6n1.pc01.cls04.6703.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g1c6n4.pc01.cls04.6700.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c4n1.pc01.cls04.6707.-1.--emit-count.--system
+topology.miami-storm-1.kafka-spout.g1c1n4.pc01.cls04.6701.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c1n2.pc01.cls04.6705.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c4n3.pc01.cls04.6706.-1.--complete-latency.default
+topology.orlando-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c7n4.pc01.cls04.6702.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c3n2.pc01.cls04.6702.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c1n2.pc01.cls04.6727.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g1c1n3.pc01.cls04.6723.-1.--receive.population
+topology.chicago-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c4n1.pc01.cls04.6706.-1.--sendqueue.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c1n3.pc01.cls04.6722.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c5n2.pc01.cls04.6702.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c7n1.pc01.cls04.6708.-1.--complete-latency.default
+topology.omaha-storm-1.kafka-spout.g2c5n2.pc01.cls04.6701.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c7n1.pc01.cls04.6700.-1.--sendqueue.population
+topology.buffalo-storm-1.kafka-spout.g2c4n3.pc01.cls04.6704.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n3.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c2n3.pc01.cls04.6723.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6721.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c2n3.pc01.cls04.6700.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c7n1.pc01.cls04.6711.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c5n3.pc01.cls04.6710.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c3n1.pc01.cls04.6726.-1.--sendqueue.population
+topology.puerto-rico-storm-1.kafka-spout.g1c3n3.pc01.cls04.6702.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6709.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c3n4.pc01.cls04.6724.-1.--emit-count.--system
+topology.englewood-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c7n2.pc01.cls04.6708.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c1n3.pc01.cls04.6727.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c7n3.pc01.cls04.6712.-1.--emit-count.default
+topology.harrison-storm-1.kafka-spout.g2c5n3.pc01.cls04.6703.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c5n1.pc01.cls04.6713.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6707.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6707.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c5n3.pc01.cls04.6723.-1.--complete-latency.default
+topology.nyc-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6711.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c6n2.pc01.cls04.6710.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6715.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c1n3.pc01.cls04.6712.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g1c4n4.pc01.cls04.6714.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c5n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c6n4.pc01.cls04.6705.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g2c4n2.pc01.cls04.6708.-1.--receive.population
+topology.anaheim-storm-1.kafka-spout.g1c1n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g2c7n1.pc01.cls04.6700.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c1n3.pc01.cls04.6712.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g2c4n4.pc01.cls04.6722.-1.--receive.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--sendqueue.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c2n2.pc01.cls04.6712.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c4n1.pc01.cls04.6711.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.omaha-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g1c4n2.pc01.cls04.6711.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c6n2.pc01.cls04.6708.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c2n3.pc01.cls04.6725.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6712.-1.--complete-latency.default
+topology.houston-storm-1.kafka-spout.g2c4n3.pc01.cls04.6711.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c2n1.pc01.cls04.6727.-1.--emit-count.default
+topology.springfield-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g2c4n1.pc01.cls04.6714.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g2c6n3.pc01.cls04.6710.-1.--sendqueue.population
+topology.buffalo-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g2c3n1.pc01.cls04.6702.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g1c1n2.pc01.cls04.6724.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g2c1n1.pc01.cls04.6711.-1.--receive.population
+topology.phoenix-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n4.pc01.cls04.6724.-1.--ack-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g1c4n2.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6710.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g1c3n3.pc01.cls04.6710.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g1c2n1.pc01.cls04.6715.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g1c6n4.pc01.cls04.6704.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c5n2.pc01.cls04.6720.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c1n1.pc01.cls04.6723.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c5n4.pc01.cls04.6700.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c1n2.pc01.cls04.6702.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c5n3.pc01.cls04.6707.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c5n1.pc01.cls04.6714.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c3n2.pc01.cls04.6712.-1.--emit-count.default
+topology.nyc-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c5n1.pc01.cls04.6705.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g2c1n2.pc01.cls04.6711.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6726.-1.--sendqueue.population
+topology.washington-dc-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c4n4.pc01.cls04.6702.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c7n2.pc01.cls04.6711.-1.--complete-latency.default
+topology.atlanta-storm-1.kafka-spout.g1c1n1.pc01.cls04.6701.-1.--ack-count.default
+topology.omaha-storm-1.kafka-spout.g2c5n2.pc01.cls04.6701.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c7n3.pc01.cls04.6702.-1.--complete-latency.default
+topology.phoenix-storm-1.kafka-spout.g2c1n1.pc01.cls04.6711.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c7n3.pc01.cls04.6708.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g1c1n2.pc01.cls04.6727.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c1n1.pc01.cls04.6720.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g1c5n3.pc01.cls04.6712.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c5n3.pc01.cls04.6727.-1.--sendqueue.population
+topology.burbank-storm-1.derivedfields-bolt.g2c1n3.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c6n1.pc01.cls04.6705.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c2n2.pc01.cls04.6726.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c6n4.pc01.cls04.6727.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c4n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g2c5n1.pc01.cls04.6703.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c3n2.pc01.cls04.6704.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c6n2.pc01.cls04.6713.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c5n1.pc01.cls04.6705.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g1c4n4.pc01.cls04.6703.-1.--receive.population
+topology.miami-storm-1.kafka-spout.g1c5n2.pc01.cls04.6704.-1.--receive.population
+topology.nyc-storm-1.kafka-spout.g2c2n4.pc01.cls04.6714.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g1c5n3.pc01.cls04.6725.-1.--ack-count.default
+topology.seattle-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c4n3.pc01.cls04.6704.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c7n3.pc01.cls04.6701.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g1c6n4.pc01.cls04.6704.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c4n3.pc01.cls04.6711.-1.--ack-count.default
+topology.puerto-rico-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c2n2.pc01.cls04.6724.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6720.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c6n4.pc01.cls04.6721.-1.--receive.population
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c4n1.pc01.cls04.6702.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c4n3.pc01.cls04.6709.-1.--ack-count.default
+topology.chicago-storm-1.kafka-spout.g2c2n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n1.pc01.cls04.6721.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g1c4n4.pc01.cls04.6712.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g1c1n4.pc01.cls04.6711.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c7n2.pc01.cls04.6704.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c2n4.pc01.cls04.6713.-1.--receive.population
+topology.akron-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6713.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g2c4n2.pc01.cls04.6720.-1.--ack-count.default
+topology.buffalo-storm-1.kafka-spout.g2c4n3.pc01.cls04.6704.-1.--emit-count.default
+topology.nashville-storm-1.kafka-spout.g2c6n2.pc01.cls04.6705.-1.--ack-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6703.-1.--ack-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c6n3.pc01.cls04.6723.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6710.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g2c5n3.pc01.cls04.6701.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g1c7n4.pc01.cls04.6706.-1.--sendqueue.population
+topology.houston-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--receive.population
+topology.fort-worth-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c3n3.pc01.cls04.6727.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c7n3.pc01.cls04.6708.-1.--ack-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c7n2.pc01.cls04.6727.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c5n1.pc01.cls04.6707.-1.--ack-count.default
+topology.harrison-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g2c5n1.pc01.cls04.6727.-1.--receive.population
+topology.burbank-storm-1.derivedfields-bolt.g2c4n1.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c4n1.pc01.cls04.6704.-1.--ack-count.default
+topology.springfield-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c3n1.pc01.cls04.6706.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c6n3.pc01.cls04.6709.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g2c3n1.pc01.cls04.6711.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c2n2.pc01.cls04.6721.-1.--receive.population
+topology.houston-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6718.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c6n3.pc01.cls04.6723.-1.--ack-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c2n4.pc01.cls04.6702.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c5n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.fort-worth-storm-1.kafka-spout.g1c3n2.pc01.cls04.6721.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c2n2.pc01.cls04.6706.-1.--complete-latency.default
+topology.cheyenne-storm-1.kafka-spout.g1c3n2.pc01.cls04.6720.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c6n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c2n3.pc01.cls04.6707.-1.--emit-count.default
+topology.houston-storm-1.kafka-spout.g2c4n2.pc01.cls04.6720.-1.--receive.population
+topology.nashville-storm-1.kafka-spout.g1c2n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g1c6n3.pc01.cls04.6725.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g2c7n4.pc01.cls04.6713.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g2c6n2.pc01.cls04.6712.-1.--emit-count.default
+topology.seattle-storm-1.kafka-spout.g2c7n2.pc01.cls04.6707.-1.--ack-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c2n3.pc01.cls04.6724.-1.--sendqueue.population
+topology.kansas-city-storm-1.kafka-spout.g1c2n2.pc01.cls04.6710.-1.--complete-latency.default
+topology.omaha-storm-1.kafka-spout.g2c5n2.pc01.cls04.6710.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g2c2n2.pc01.cls04.6714.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6703.-1.--ack-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c1n4.pc01.cls04.6709.-1.--emit-count.--system
+topology.burbank-storm-1.kafka-spout.g1c5n1.pc01.cls04.6727.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c6n3.pc01.cls04.6700.-1.--sendqueue.population
+topology.puerto-rico-storm-1.kafka-spout.g1c5n1.pc01.cls04.6724.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c2n3.pc01.cls04.6708.-1.--emit-count.--system
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-6.earliestTimeOffset
+topology.chicago-storm-1.kafka-spout.g1c2n1.pc01.cls04.6725.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g1c5n3.pc01.cls04.6712.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c2n4.pc01.cls04.6721.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c2n3.pc01.cls04.6700.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c7n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c5n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c6n2.pc01.cls04.6702.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g2c1n4.pc01.cls04.6724.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c2n4.pc01.cls04.6703.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g1c2n1.pc01.cls04.6722.-1.--receive.population
+topology.tacoma-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.puerto-rico-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c7n1.pc01.cls04.6724.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c5n4.pc01.cls04.6700.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c5n3.pc01.cls04.6704.-1.--receive.population
+topology.akron-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g1c6n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.kansas-city-storm-1.kafka-spout.g2c3n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.springfield-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c3n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c7n1.pc01.cls04.6723.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6706.-1.--sendqueue.population
+topology.elkridge-storm-1.derivedfields-bolt.g1c3n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c6n3.pc01.cls04.6724.-1.--complete-latency.default
+topology.akron-storm-1.kafka-spout.g2c2n2.pc01.cls04.6724.-1.--ack-count.default
+topology.springfield-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.cheyenne-storm-1.kafka-spout.g1c5n1.pc01.cls04.6703.-1.--receive.population
+topology.harrison-storm-1.kafka-spout.g2c1n3.pc01.cls04.6708.-1.--emit-count.default
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6705.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c1n2.pc01.cls04.6715.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g2c2n4.pc01.cls04.6702.-1.--emit-count.default
+topology.phoenix-storm-1.kafka-spout.g2c5n1.pc01.cls04.6709.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c6n4.pc01.cls04.6711.-1.--complete-latency.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6708.-1.--execute-count.kafka-spout:default
+topology.pearl-city-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c4n3.pc01.cls04.6710.-1.--receive.population
+topology.kansas-city-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c2n1.pc01.cls04.6710.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c1n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c4n1.pc01.cls04.6724.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c1n1.pc01.cls04.6701.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c5n2.pc01.cls04.6704.-1.--sendqueue.population
+topology.englewood-storm-1.kafka-spout.g1c7n3.pc01.cls04.6703.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c6n2.pc01.cls04.6714.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c2n3.pc01.cls04.6702.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g2c5n3.pc01.cls04.6727.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c4n2.pc01.cls04.6707.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c2n4.pc01.cls04.6726.-1.--ack-count.default
+topology.seattle-storm-1.kafka-spout.g2c2n3.pc01.cls04.6712.-1.--ack-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g2c3n3.pc01.cls04.6721.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c1n4.pc01.cls04.6702.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c4n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.stockton-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n1.pc01.cls04.6709.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g2c5n1.pc01.cls04.6723.-1.--sendqueue.population
+topology.buffalo-storm-1.kafka-spout.g1c5n3.pc01.cls04.6707.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c2n4.pc01.cls04.6704.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6716.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c6n4.pc01.cls04.6703.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c1n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c7n3.pc01.cls04.6706.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c4n3.pc01.cls04.6710.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c5n2.pc01.cls04.6709.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c5n3.pc01.cls04.6704.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c2n4.pc01.cls04.6721.-1.--emit-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g1c5n2.pc01.cls04.6723.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c2n2.pc01.cls04.6714.-1.--emit-count.default
+topology.akron-storm-1.kafka-spout.g2c2n4.pc01.cls04.6712.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g2c7n3.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6720.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6711.-1.--process-latency.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g1c3n1.pc01.cls04.6722.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g1c7n3.pc01.cls04.6726.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6705.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c7n2.pc01.cls04.6721.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6707.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g2c6n1.pc01.cls04.6707.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c4n4.pc01.cls04.6710.-1.--complete-latency.default
+topology.cheyenne-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6703.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c3n3.pc01.cls04.6712.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c7n2.pc01.cls04.6724.-1.--ack-count.default
+topology.fort-worth-storm-1.kafka-spout.g1c4n2.pc01.cls04.6726.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--complete-latency.default
+topology.chicago-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c1n4.pc01.cls04.6727.-1.--complete-latency.default
+topology.englewood-storm-1.derivedfields-bolt.g2c2n1.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6709.-1.--complete-latency.default
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6722.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c3n4.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6724.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c2n3.pc01.cls04.6725.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c7n3.pc01.cls04.6723.-1.--ack-count.default
+topology.englewood-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6723.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g2c6n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c6n4.pc01.cls04.6705.-1.--ack-count.default
+topology.akron-storm-1.kafka-spout.g1c4n2.pc01.cls04.6708.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g2c2n3.pc01.cls04.6702.-1.--receive.population
+topology.nashville-storm-1.derivedfields-bolt.g2c1n1.pc01.cls04.6712.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c5n3.pc01.cls04.6713.-1.--ack-count.default
+topology.st-paul-storm-1.kafka-spout.g2c1n4.pc01.cls04.6702.-1.--emit-count.default
+topology.houston-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g1c4n4.pc01.cls04.6708.-1.--ack-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g1c6n4.pc01.cls04.6704.-1.--receive.population
+topology.fort-worth-storm-1.kafka-spout.g2c6n3.pc01.cls04.6703.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g1c5n3.pc01.cls04.6723.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g2c6n1.pc01.cls04.6708.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--ack-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.tacoma-storm-1.kafka-spout.g1c4n4.pc01.cls04.6705.-1.--receive.population
+topology.chicago-storm-1.kafka-spout.g1c2n1.pc01.cls04.6725.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c7n2.pc01.cls04.6706.-1.--emit-count.default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6711.-1.--receive.population
+topology.englewood-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6721.-1.--ack-count.kafka-spout:default
+topology.chicago-storm-1.kafka-spout.g2c3n4.pc01.cls04.6726.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6703.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c2n3.pc01.cls04.6725.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g2c7n1.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6727.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c2n1.pc01.cls04.6713.-1.--emit-count.default
+topology.chicago-storm-1.kafka-spout.g1c4n1.pc01.cls04.6724.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c3n2.pc01.cls04.6712.-1.--receive.population
+topology.puerto-rico-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c1n2.pc01.cls04.6701.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c2n1.pc01.cls04.6719.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c4n1.pc01.cls04.6724.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c4n3.pc01.cls04.6703.-1.--receive.population
+topology.akron-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c7n4.pc01.cls04.6702.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c7n1.pc01.cls04.6703.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g1c6n1.pc01.cls04.6702.-1.--sendqueue.population
+topology.fort-worth-storm-1.kafka-spout.g1c7n1.pc01.cls04.6709.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g1c3n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.san-jose-storm-1.kafka-spout.g2c3n4.pc01.cls04.6707.-1.--complete-latency.default
+topology.chicago-storm-1.kafka-spout.g1c3n2.pc01.cls04.6722.-1.--complete-latency.default
+topology.miami-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6700.-1.--ack-count.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6710.-1.--ack-count.kafka-spout:default
+topology.omaha-storm-1.kafka-spout.g2c6n1.pc01.cls04.6710.-1.--ack-count.default
+topology.nashville-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c4n2.pc01.cls04.6722.-1.--emit-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c5n4.pc01.cls04.6709.-1.--emit-count.default
+topology.fort-worth-storm-1.kafka-topic.fort-worth-storm-1.partition-62.latestEmittedOffset
+topology.st-paul-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.buffalo-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c6n1.pc01.cls04.6720.-1.--sendqueue.population
+topology.harrison-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.fairfax-storm-1.kafka-spout.g1c3n3.pc01.cls04.6725.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g2c3n1.pc01.cls04.6704.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g1c2n4.pc01.cls04.6712.-1.--emit-count.default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n3.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c6n4.pc01.cls04.6714.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6721.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.harrison-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.chicago-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6721.-1.--process-latency.kafka-spout:default
+topology.cheyenne-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g2c7n2.pc01.cls04.6723.-1.--complete-latency.default
+topology.nyc-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g2c2n1.pc01.cls04.6708.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c2n1.pc01.cls04.6704.-1.--receive.population
+topology.seattle-storm-1.kafka-spout.g2c2n1.pc01.cls04.6714.-1.--ack-count.default
+topology.phoenix-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6711.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c7n2.pc01.cls04.6721.-1.--ack-count.default
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6722.-1.--ack-count.default
+topology.akron-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c6n4.pc01.cls04.6707.-1.--ack-count.default
+topology.stockton-storm-1.kafka-spout.g2c7n4.pc01.cls04.6724.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c2n4.pc01.cls04.6701.-1.--complete-latency.default
+topology.fort-worth-storm-1.kafka-spout.g1c5n1.pc01.cls04.6722.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g2c3n1.pc01.cls04.6721.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6723.-1.--process-latency.kafka-spout:default
+topology.washington-dc-storm-1.kafka-spout.g1c5n1.pc01.cls04.6715.-1.--complete-latency.default
+topology.atlanta-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c2n4.pc01.cls04.6722.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c6n3.pc01.cls04.6701.-1.--receive.population
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6706.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c3n3.pc01.cls04.6711.-1.--ack-count.default
+topology.san-jose-storm-1.kafka-spout.g2c5n3.pc01.cls04.6722.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c6n1.pc01.cls04.6707.-1.--ack-count.default
+topology.springfield-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--receive.population
+topology.washington-dc-storm-1.kafka-spout.g1c7n3.pc01.cls04.6715.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--emit-count.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c6n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g1c1n3.pc01.cls04.6719.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c6n3.pc01.cls04.6703.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c1n4.pc01.cls04.6702.-1.--complete-latency.default
+topology.harrison-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g1c1n1.pc01.cls04.6700.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g1c3n3.pc01.cls04.6710.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g2c6n2.pc01.cls04.6714.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g2c1n4.pc01.cls04.6726.-1.--emit-count.default
+topology.miami-storm-1.kafka-spout.g1c6n4.pc01.cls04.6715.-1.--receive.population
+topology.cheyenne-storm-1.kafka-spout.g1c5n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.buffalo-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g1c1n4.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.atlanta-storm-1.kafka-spout.g2c6n2.pc01.cls04.6726.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--receive.population
+topology.elkridge-storm-1.kafka-spout.g1c2n2.pc01.cls04.6700.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.pearl-city-storm-1.kafka-spout.g2c1n1.pc01.cls04.6713.-1.--complete-latency.default
+topology.nashville-storm-1.kafka-spout.g2c1n1.pc01.cls04.6712.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g2c2n3.pc01.cls04.6725.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c5n2.pc01.cls04.6708.-1.--ack-count.default
+topology.buffalo-storm-1.kafka-spout.g2c5n2.pc01.cls04.6705.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c5n1.pc01.cls04.6727.-1.--sendqueue.population
+topology.orlando-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n1.pc01.cls04.6711.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c2n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.miami-storm-1.kafka-spout.g2c3n4.pc01.cls04.6706.-1.--complete-latency.default
+topology.anaheim-storm-1.kafka-spout.g1c7n3.pc01.cls04.6701.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c2n2.pc01.cls04.6711.-1.--emit-count.--system
+topology.burbank-storm-1.derivedfields-bolt.g1c1n3.pc01.cls04.6712.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c5n3.pc01.cls04.6705.-1.--complete-latency.default
+topology.seattle-storm-1.kafka-spout.g2c7n1.pc01.cls04.6722.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c7n2.pc01.cls04.6726.-1.--ack-count.default
+topology.phoenix-storm-1.kafka-spout.g2c2n2.pc01.cls04.6712.-1.--sendqueue.population
+topology.kansas-city-storm-1.derivedfields-bolt.g1c3n1.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g1c3n2.pc01.cls04.6707.-1.--ack-count.default
+topology.burbank-storm-1.kafka-spout.g2c6n1.pc01.cls04.6704.-1.--sendqueue.population
+topology.st-paul-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6720.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c7n4.pc01.cls04.6723.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c3n2.pc01.cls04.6707.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c4n4.pc01.cls04.6721.-1.--sendqueue.population
+topology.nashville-storm-1.kafka-spout.g2c1n3.pc01.cls04.6725.-1.--emit-count.--system
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6710.-1.--execute-count.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c4n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g1c7n4.pc01.cls04.6706.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c5n3.pc01.cls04.6725.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6710.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c6n1.pc01.cls04.6707.-1.--emit-count.default
+topology.springfield-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c4n2.pc01.cls04.6701.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c2n2.pc01.cls04.6707.-1.--receive.population
+topology.san-jose-storm-1.derivedfields-bolt.g2c1n2.pc01.cls04.6707.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c6n2.pc01.cls04.6721.-1.--ack-count.default
+topology.englewood-storm-1.kafka-spout.g1c3n3.pc01.cls04.6704.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g2c4n2.pc01.cls04.6706.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g1c3n1.pc01.cls04.6726.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c4n3.pc01.cls04.6703.-1.--emit-count.--system
+topology.englewood-storm-1.kafka-spout.g1c1n4.pc01.cls04.6703.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c4n3.pc01.cls04.6709.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g1c6n2.pc01.cls04.6727.-1.--complete-latency.default
+topology.cheyenne-storm-1.kafka-spout.g2c7n4.pc01.cls04.6726.-1.--emit-count.default
+topology.burbank-storm-1.kafka-spout.g1c2n3.pc01.cls04.6714.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g2c5n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.orlando-storm-1.kafka-spout.g1c2n3.pc01.cls04.6708.-1.--emit-count.default
+topology.miami-storm-1.derivedfields-bolt.g1c5n2.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g1c7n3.pc01.cls04.6720.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c6n1.pc01.cls04.6723.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g2c3n3.pc01.cls04.6723.-1.--receive.population
+topology.akron-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6714.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6726.-1.--execute-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c1n1.pc01.cls04.6722.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g2c3n1.pc01.cls04.6720.-1.--receive.population
+topology.fairfax-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6706.-1.--process-latency.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6709.-1.--complete-latency.default
+topology.miami-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6700.-1.--process-latency.kafka-spout:default
+topology.fairfax-storm-1.kafka-spout.g2c7n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.seattle-storm-1.derivedfields-bolt.g2c7n2.pc01.cls04.6707.-1.--process-latency.kafka-spout:default
+topology.seattle-storm-1.derivedfields-bolt.g2c5n1.pc01.cls04.6708.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c7n4.pc01.cls04.6727.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g1c5n3.pc01.cls04.6724.-1.--complete-latency.default
+topology.englewood-storm-1.kafka-spout.g2c6n4.pc01.cls04.6726.-1.--complete-latency.default
+topology.orlando-storm-1.kafka-spout.g1c7n1.pc01.cls04.6726.-1.--emit-count.--system
+topology.englewood-storm-1.kafka-spout.g2c4n1.pc01.cls04.6724.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g1c7n2.pc01.cls04.6711.-1.--sendqueue.population
+topology.miami-storm-1.kafka-spout.g2c3n4.pc01.cls04.6710.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g1c7n4.pc01.cls04.6726.-1.--receive.population
+topology.springfield-storm-1.kafka-spout.g1c6n3.pc01.cls04.6705.-1.--emit-count.default
+topology.nashville-storm-1.kafka-spout.g2c6n1.pc01.cls04.6720.-1.--ack-count.default
+topology.miami-storm-1.kafka-spout.g1c1n3.pc01.cls04.6702.-1.--receive.population
+topology.phoenix-storm-1.kafka-spout.g1c4n2.pc01.cls04.6714.-1.--receive.population
+topology.seattle-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c1n1.pc01.cls04.6703.-1.--emit-count.--system
+topology.puerto-rico-storm-1.kafka-topic.puerto-rico-storm-1.partition-5.latestTimeOffset
+topology.anaheim-storm-1.derivedfields-bolt.g1c5n4.pc01.cls04.6725.-1.--execute-count.kafka-spout:default
+topology.elkridge-storm-1.kafka-spout.g2c1n4.pc01.cls04.6708.-1.--receive.population
+topology.burbank-storm-1.kafka-spout.g2c2n3.pc01.cls04.6727.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g1c4n1.pc01.cls04.6723.-1.--complete-latency.default
+topology.elkridge-storm-1.kafka-spout.g1c4n3.pc01.cls04.6701.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c6n2.pc01.cls04.6710.-1.--receive.population
+topology.miami-storm-1.derivedfields-bolt.g2c5n4.pc01.cls04.6708.-1.--ack-count.kafka-spout:default
+topology.nyc-storm-1.kafka-spout.g2c3n1.pc01.cls04.6701.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c3n3.pc01.cls04.6724.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g1c4n2.pc01.cls04.6707.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6712.-1.--receive.population
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n1.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.buffalo-storm-1.kafka-spout.g1c5n3.pc01.cls04.6707.-1.--complete-latency.default
+topology.stockton-storm-1.kafka-spout.g1c6n3.pc01.cls04.6709.-1.--sendqueue.population
+topology.nyc-storm-1.kafka-spout.g2c4n2.pc01.cls04.6710.-1.--receive.population
+topology.tacoma-storm-1.kafka-spout.g1c6n2.pc01.cls04.6713.-1.--emit-count.default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6702.-1.--emit-count.default
+topology.springfield-storm-1.kafka-spout.g2c2n1.pc01.cls04.6709.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g2c7n1.pc01.cls04.6725.-1.--emit-count.--system
+topology.akron-storm-1.derivedfields-bolt.g1c6n3.pc01.cls04.6711.-1.--execute-count.kafka-spout:default
+topology.fort-worth-storm-1.derivedfields-bolt.g2c2n3.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.fairfax-storm-1.derivedfields-bolt.g2c3n1.pc01.cls04.6707.-1.--execute-count.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c5n3.pc01.cls04.6707.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n2.pc01.cls04.6725.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c4n1.pc01.cls04.6709.-1.--complete-latency.default
+topology.st-paul-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6709.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c1n1.pc01.cls04.6701.-1.--receive.population
+topology.atlanta-storm-1.kafka-spout.g1c3n2.pc01.cls04.6708.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g1c6n3.pc01.cls04.6724.-1.--emit-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g2c7n4.pc01.cls04.6704.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g1c7n3.pc01.cls04.6722.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g1c5n2.pc01.cls04.6711.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c3n1.pc01.cls04.6723.-1.--ack-count.default
+topology.seattle-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6709.-1.--execute-count.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c3n3.pc01.cls04.6708.-1.--emit-count.default
+topology.atlanta-storm-1.kafka-spout.g1c5n2.pc01.cls04.6707.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g2c6n1.pc01.cls04.6703.-1.--complete-latency.default
+topology.harrison-storm-1.kafka-spout.g2c2n2.pc01.cls04.6706.-1.--receive.population
+topology.washington-dc-storm-1.kafka-spout.g2c6n2.pc01.cls04.6702.-1.--ack-count.default
+topology.washington-dc-storm-1.kafka-spout.g2c6n3.pc01.cls04.6714.-1.--emit-count.default
+topology.orlando-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6722.-1.--ack-count.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c2n3.pc01.cls04.6711.-1.--receive.population
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6726.-1.--complete-latency.default
+topology.cheyenne-storm-1.kafka-spout.g1c6n3.pc01.cls04.6712.-1.--emit-count.default
+topology.nyc-storm-1.kafka-spout.g2c4n3.pc01.cls04.6726.-1.--complete-latency.default
+topology.tacoma-storm-1.kafka-spout.g2c4n4.pc01.cls04.6712.-1.--complete-latency.default
+topology.burbank-storm-1.kafka-spout.g1c2n1.pc01.cls04.6720.-1.--emit-count.default
+topology.orlando-storm-1.kafka-spout.g2c4n2.pc01.cls04.6727.-1.--sendqueue.population
+topology.nyc-storm-1.derivedfields-bolt.g1c2n4.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.miami-storm-1.kafka-spout.g1c2n4.pc01.cls04.6723.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c1n4.pc01.cls04.6725.-1.--complete-latency.default
+topology.orlando-storm-1.derivedfields-bolt.g1c3n4.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.houston-storm-1.kafka-spout.g2c4n4.pc01.cls04.6701.-1.--receive.population
+topology.englewood-storm-1.kafka-spout.g1c3n2.pc01.cls04.6709.-1.--sendqueue.population
+topology.chicago-storm-1.kafka-spout.g1c7n1.pc01.cls04.6723.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c1n2.pc01.cls04.6701.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g1c4n1.pc01.cls04.6712.-1.--sendqueue.population
+topology.seattle-storm-1.kafka-spout.g2c5n2.pc01.cls04.6724.-1.--receive.population
+topology.washington-dc-storm-1.kafka-spout.g2c6n4.pc01.cls04.6707.-1.--emit-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6727.-1.--ack-count.kafka-spout:default
+topology.miami-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c1n3.pc01.cls04.6726.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c3n3.pc01.cls04.6725.-1.--sendqueue.population
+topology.elkridge-storm-1.kafka-spout.g1c5n4.pc01.cls04.6710.-1.--receive.population
+topology.fairfax-storm-1.kafka-spout.g2c1n4.pc01.cls04.6700.-1.--complete-latency.default
+topology.elkridge-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c5n2.pc01.cls04.6706.-1.--emit-count.default
+topology.stockton-storm-1.derivedfields-bolt.g2c5n2.pc01.cls04.6702.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n2.pc01.cls04.6701.-1.--ack-count.default
+topology.kansas-city-storm-1.kafka-spout.g2c6n2.pc01.cls04.6711.-1.--emit-count.default
+topology.buffalo-storm-1.kafka-spout.g1c4n4.pc01.cls04.6706.-1.--emit-count.default
+topology.kansas-city-storm-1.derivedfields-bolt.g1c4n1.pc01.cls04.6712.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g2c5n1.pc01.cls04.6721.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g2c4n4.pc01.cls04.6715.-1.--receive.population
+topology.st-paul-storm-1.kafka-spout.g1c2n1.pc01.cls04.6725.-1.--receive.population
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6710.-1.--sendqueue.population
+topology.burbank-storm-1.kafka-spout.g1c7n1.pc01.cls04.6709.-1.--emit-count.default
+topology.springfield-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6701.-1.--ack-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c7n2.pc01.cls04.6700.-1.--sendqueue.population
+topology.harrison-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.kafka-spout.g2c5n1.pc01.cls04.6722.-1.--ack-count.default
+topology.omaha-storm-1.kafka-spout.g1c5n3.pc01.cls04.6712.-1.--sendqueue.population
+topology.akron-storm-1.kafka-spout.g1c5n3.pc01.cls04.6703.-1.--ack-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c4n3.pc01.cls04.6714.-1.--ack-count.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c3n2.pc01.cls04.6722.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c2n2.pc01.cls04.6700.-1.--sendqueue.population
+topology.atlanta-storm-1.kafka-spout.g1c5n3.pc01.cls04.6723.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g2c2n3.pc01.cls04.6725.-1.--complete-latency.default
+topology.tacoma-storm-1.derivedfields-bolt.g1c6n2.pc01.cls04.6713.-1.--process-latency.kafka-spout:default
+topology.elkridge-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6704.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.derivedfields-bolt.g1c2n2.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.stockton-storm-1.derivedfields-bolt.g2c6n2.pc01.cls04.6702.-1.--ack-count.kafka-spout:default
+topology.springfield-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6722.-1.--process-latency.kafka-spout:default
+topology.kansas-city-storm-1.kafka-spout.g1c4n4.pc01.cls04.6715.-1.--emit-count.default
+topology.anaheim-storm-1.kafka-spout.g1c7n3.pc01.cls04.6701.-1.--emit-count.default
+topology.st-paul-storm-1.kafka-spout.g1c1n2.pc01.cls04.6706.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g2c2n4.pc01.cls04.6705.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.derivedfields-bolt.g1c4n4.pc01.cls04.6713.-1.--ack-count.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c7n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g2c4n4.pc01.cls04.6700.-1.--execute-count.kafka-spout:default
+topology.springfield-storm-1.kafka-spout.g2c3n2.pc01.cls04.6705.-1.--sendqueue.population
+topology.washington-dc-storm-1.derivedfields-bolt.g2c6n1.pc01.cls04.6723.-1.--ack-count.kafka-spout:default
+topology.englewood-storm-1.derivedfields-bolt.g2c4n2.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.orlando-storm-1.kafka-spout.g2c7n2.pc01.cls04.6701.-1.--emit-count.default
+topology.atlanta-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6724.-1.--execute-count.kafka-spout:default
+topology.puerto-rico-storm-1.kafka-spout.g2c2n4.pc01.cls04.6725.-1.--sendqueue.population
+topology.fairfax-storm-1.kafka-spout.g1c5n3.pc01.cls04.6727.-1.--emit-count.default
+topology.washington-dc-storm-1.kafka-spout.g1c4n1.pc01.cls04.6709.-1.--receive.population
+topology.stockton-storm-1.kafka-spout.g2c4n2.pc01.cls04.6712.-1.--emit-count.default
+topology.englewood-storm-1.derivedfields-bolt.g1c7n3.pc01.cls04.6704.-1.--execute-count.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c3n4.pc01.cls04.6722.-1.--ack-count.default
+topology.fairfax-storm-1.kafka-spout.g1c3n4.pc01.cls04.6726.-1.--receive.population
+topology.atlanta-storm-1.derivedfields-bolt.g1c2n1.pc01.cls04.6722.-1.--execute-count.kafka-spout:default
+topology.san-jose-storm-1.kafka-spout.g1c5n4.pc01.cls04.6727.-1.--sendqueue.population
+topology.san-jose-storm-1.kafka-spout.g2c3n1.pc01.cls04.6700.-1.--sendqueue.population
+topology.miami-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6713.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g2c2n2.pc01.cls04.6704.-1.--emit-count.default
+topology.fairfax-storm-1.kafka-spout.g1c1n4.pc01.cls04.6727.-1.--receive.population
+topology.houston-storm-1.kafka-spout.g2c2n3.pc01.cls04.6700.-1.--ack-count.default
+topology.houston-storm-1.kafka-spout.g2c4n2.pc01.cls04.6720.-1.--sendqueue.population
+topology.anaheim-storm-1.kafka-spout.g2c5n3.pc01.cls04.6720.-1.--ack-count.default
+topology.nyc-storm-1.kafka-spout.g2c4n4.pc01.cls04.6711.-1.--ack-count.default
+topology.orlando-storm-1.kafka-spout.g1c1n4.pc01.cls04.6707.-1.--emit-count.default
+topology.st-paul-storm-1.derivedfields-bolt.g2c6n4.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.burbank-storm-1.kafka-spout.g1c5n4.pc01.cls04.6711.-1.--sendqueue.population
+topology.st-paul-storm-1.kafka-spout.g2c4n2.pc01.cls04.6725.-1.--sendqueue.population
+topology.puerto-rico-storm-1.kafka-spout.g1c3n4.pc01.cls04.6711.-1.--ack-count.default
+topology.anaheim-storm-1.derivedfields-bolt.g1c7n2.pc01.cls04.6701.-1.--process-latency.kafka-spout:default
+topology.phoenix-storm-1.kafka-spout.g1c7n1.pc01.cls04.6709.-1.--emit-count.default
+topology.chicago-storm-1.derivedfields-bolt.g2c3n2.pc01.cls04.6726.-1.--process-latency.kafka-spout:default
+topology.akron-storm-1.kafka-spout.g2c7n3.pc01.cls04.6706.-1.--complete-latency.default
+topology.st-paul-storm-1.kafka-spout.g2c2n2.pc01.cls04.6710.-1.--emit-count.default
+topology.san-jose-storm-1.kafka-spout.g1c4n1.pc01.cls04.6702.-1.--complete-latency.default
+topology.nyc-storm-1.kafka-spout.g2c5n2.pc01.cls04.6713.-1.--sendqueue.population
+topology.cheyenne-storm-1.kafka-spout.g1c6n3.pc01.cls04.6712.-1.--ack-count.default
+topology.elkridge-storm-1.kafka-spout.g2c6n3.pc01.cls04.6705.-1.--emit-count.default
+topology.englewood-storm-1.kafka-spout.g1c7n4.pc01.cls04.6701.-1.--ack-count.default
+topology.fairfax-storm-1.derivedfields-bolt.g2c1n4.pc01.cls04.6724.-1.--process-latency.kafka-spout:default
+topology.stockton-storm-1.kafka-spout.g2c2n4.pc01.cls04.6720.-1.--sendqueue.population
+topology.buffalo-storm-1.kafka-spout.g2c4n2.pc01.cls04.6703.-1.--sendqueue.population
+topology.springfield-storm-1.derivedfields-bolt.g2c2n2.pc01.cls04.6705.-1.--process-latency.kafka-spout:default
+topology.st-paul-storm-1.kafka-spout.g2c3n3.pc01.cls04.6712.-1.--complete-latency.default
+topology.san-jose-storm-1.derivedfields-bolt.g2c3n3.pc01.cls04.6726.-1.--ack-count.kafka-spout:default
+topology.washington-dc-storm-1.derivedfields-bolt.g1c6n1.pc01.cls04.6720.-1.--process-latency.kafka-spout:default
+topology.anaheim-storm-1.kafka-spout.g1c7n3.pc01.cls04.6701.-1.--emit-count.--system
+topology.atlanta-storm-1.kafka-spout.g2c3n2.pc01.cls04.6724.-1.--sendqueue.population
+topology.seattle-storm-1.derivedfields-bolt.g1c1n1.pc01.cls04.6702.-1.--execute-count.kafka-spout:default
+topology.seattle-storm-1.kafka-spout.g2c7n1.pc01.cls04.6722.-1.--ack-count.default
+topology.cheyenne-storm-1.kafka-spout.g1c5n4.pc01.cls04.6703.-1.--complete-latency.default
... 150251 lines suppressed ...

-- 
To stop receiving notification emails like this one, please contact
avijayan@apache.org.