You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by ka...@apache.org on 2018/01/22 21:23:40 UTC

[1/7] storm git commit: STORM-2887: store metrics into RocksDB

Repository: storm
Updated Branches:
  refs/heads/master 879cb5b8f -> e6a423dd8


http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbStoreTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbStoreTest.java b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbStoreTest.java
new file mode 100644
index 0000000..63df80a
--- /dev/null
+++ b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbStoreTest.java
@@ -0,0 +1,328 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.metricstore.AggLevel;
+import org.apache.storm.metricstore.FilterOptions;
+import org.apache.storm.metricstore.Metric;
+import org.apache.storm.metricstore.MetricException;
+import org.apache.storm.metricstore.MetricStore;
+import org.apache.storm.metricstore.MetricStoreConfig;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class RocksDbStoreTest {
+    private final static Logger LOG = LoggerFactory.getLogger(RocksDbStoreTest.class);
+    static MetricStore store;
+    static Path tempDirForTest;
+
+    @BeforeClass
+    public static void setUp() throws MetricException, IOException {
+        // remove any previously created cache instance
+        StringMetadataCache.cleanUp();
+        tempDirForTest = Files.createTempDirectory("RocksDbStoreTest");
+        Map<String, Object> conf = new HashMap<>();
+        conf.put(DaemonConfig.STORM_METRIC_STORE_CLASS, "org.apache.storm.metricstore.rocksdb.RocksDbStore");
+        conf.put(DaemonConfig.STORM_ROCKSDB_LOCATION, tempDirForTest.toString());
+        conf.put(DaemonConfig.STORM_ROCKSDB_CREATE_IF_MISSING, true);
+        conf.put(DaemonConfig.STORM_ROCKSDB_METADATA_STRING_CACHE_CAPACITY, 4000);
+        conf.put(DaemonConfig.STORM_ROCKSDB_METRIC_RETENTION_HOURS, 240);
+        store = MetricStoreConfig.configure(conf);
+    }
+
+    @AfterClass
+    public static void tearDown() throws IOException {
+        if (store != null) {
+            store.close();
+        }
+        StringMetadataCache.cleanUp();
+        FileUtils.deleteDirectory(tempDirForTest.toFile());
+    }
+
+    @Test
+    public void testAggregation() throws Exception {
+        double sum0 = 0.0;
+        double sum1 = 0.0;
+        double sum10 = 0.0;
+        double sum60 = 0.0;
+        Metric toPopulate = null;
+        for (int i=0; i<20; i++) {
+            double value = 5 + i;
+            long timestamp = 1L + i*60*1000;
+            Metric m = new Metric("cpu", timestamp, "myTopologyId123", value,
+                    "componentId1", "executorId1", "hostname1", "streamid1", 7777, AggLevel.AGG_LEVEL_NONE);
+            toPopulate = new Metric(m);
+            store.insert(m);
+
+            if (timestamp < 60*1000) {
+                sum0 += value;
+                sum1 += value;
+                sum10 += value;
+                sum60 += value;
+            } else if (timestamp < 600*1000) {
+                sum10 += value;
+                sum60 += value;
+            } else {
+                sum60 += value;
+            }
+        }
+
+        waitForInsertFinish(toPopulate);
+
+        toPopulate.setTimestamp(1L);
+        toPopulate.setAggLevel(AggLevel.AGG_LEVEL_NONE);
+        boolean res = store.populateValue(toPopulate);
+        Assert.assertEquals(true, res);
+        Assert.assertEquals(sum0, toPopulate.getSum(), 0.001);
+        Assert.assertEquals(sum0, toPopulate.getValue(), 0.001);
+        Assert.assertEquals(5.0, toPopulate.getMin(), 0.001);
+        Assert.assertEquals(5.0, toPopulate.getMax(), 0.001);
+        Assert.assertEquals(1, toPopulate.getCount());
+
+        toPopulate.setTimestamp(0L);
+        toPopulate.setAggLevel(AggLevel.AGG_LEVEL_1_MIN);
+        res = store.populateValue(toPopulate);
+        Assert.assertEquals(true, res);
+        Assert.assertEquals(sum1, toPopulate.getSum(), 0.001);
+        Assert.assertEquals(sum1, toPopulate.getValue(), 0.001);
+        Assert.assertEquals(5.0, toPopulate.getMin(), 0.001);
+        Assert.assertEquals(5.0, toPopulate.getMax(), 0.001);
+        Assert.assertEquals(1, toPopulate.getCount());
+
+        toPopulate.setTimestamp(0L);
+        toPopulate.setAggLevel(AggLevel.AGG_LEVEL_10_MIN);
+        res = store.populateValue(toPopulate);
+        Assert.assertEquals(true, res);
+        Assert.assertEquals(sum10, toPopulate.getSum(), 0.001);
+        Assert.assertEquals(sum10/10.0, toPopulate.getValue(), 0.001);
+        Assert.assertEquals(5.0, toPopulate.getMin(), 0.001);
+        Assert.assertEquals(14.0, toPopulate.getMax(), 0.001);
+        Assert.assertEquals(10, toPopulate.getCount());
+
+        toPopulate.setTimestamp(0L);
+        toPopulate.setAggLevel(AggLevel.AGG_LEVEL_60_MIN);
+        res = store.populateValue(toPopulate);
+        Assert.assertEquals(true, res);
+        Assert.assertEquals(sum60, toPopulate.getSum(), 0.001);
+        Assert.assertEquals(sum60/20.0, toPopulate.getValue(), 0.001);
+        Assert.assertEquals(5.0, toPopulate.getMin(), 0.001);
+        Assert.assertEquals(24.0, toPopulate.getMax(), 0.001);
+        Assert.assertEquals(20, toPopulate.getCount());
+    }
+
+    @Test
+    public void testPopulateFailure() throws Exception {
+        Metric m = new Metric("cpu", 3000L, "myTopologyId456", 1.0,
+                "componentId2", "executorId2", "hostname2", "streamid2", 7778, AggLevel.AGG_LEVEL_NONE);
+        store.insert(m);
+        waitForInsertFinish(m);
+        Metric toFind = new Metric(m);
+        toFind.setTopologyId("somethingBogus");
+        boolean res = store.populateValue(toFind);
+        Assert.assertEquals(false, res);
+    }
+
+    private List<Metric> getMetricsFromScan(FilterOptions filter) throws MetricException {
+        List<Metric> list = new ArrayList<>();
+        store.scan(filter, (Metric m) -> {
+            list.add(m);
+        });
+        return list;
+    }
+
+    @Test
+    public void testScan() throws Exception {
+        FilterOptions filter;
+        List<Metric> list;
+
+        Metric m1 = new Metric("metricType1", 50000000L, "Topo-m1", 1.0,
+                "component-1", "executor-2", "hostname-1", "stream-1", 1, AggLevel.AGG_LEVEL_NONE);
+        Metric m2 = new Metric("metricType2", 50030000L, "Topo-m1", 1.0,
+                "component-1", "executor-1", "hostname-2", "stream-2", 1, AggLevel.AGG_LEVEL_NONE);
+        Metric m3 = new Metric("metricType3", 50200000L, "Topo-m1", 1.0,
+                "component-2", "executor-1", "hostname-1", "stream-3", 1, AggLevel.AGG_LEVEL_NONE);
+        Metric m4 = new Metric("metricType4", 50200000L, "Topo-m2", 1.0,
+                "component-2", "executor-1", "hostname-2", "stream-4", 2, AggLevel.AGG_LEVEL_NONE);
+        store.insert(m1);
+        store.insert(m2);
+        store.insert(m3);
+        store.insert(m4);
+        waitForInsertFinish(m4);
+
+        // validate search by time
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setStartTime(50000000L);
+        filter.setEndTime(50130000L);
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(2, list.size());
+        Assert.assertTrue(list.contains(m1));
+        Assert.assertTrue(list.contains(m2));
+
+        // validate search by topology id
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setTopologyId("Topo-m2");
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(1, list.size());
+        Assert.assertTrue(list.contains(m4));
+
+        // validate search by metric id
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setMetricName("metricType2");
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(1, list.size());
+        Assert.assertTrue(list.contains(m2));
+
+        // validate search by component id
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setComponentId("component-2");
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(2, list.size());
+        Assert.assertTrue(list.contains(m3));
+        Assert.assertTrue(list.contains(m4));
+
+        // validate search by executor id
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setExecutorId("executor-1");
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(3, list.size());
+        Assert.assertTrue(list.contains(m2));
+        Assert.assertTrue(list.contains(m3));
+        Assert.assertTrue(list.contains(m4));
+
+        // validate search by executor id
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setExecutorId("executor-1");
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(3, list.size());
+        Assert.assertTrue(list.contains(m2));
+        Assert.assertTrue(list.contains(m3));
+        Assert.assertTrue(list.contains(m4));
+
+        // validate search by host id
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setHostId("hostname-2");
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(2, list.size());
+        Assert.assertTrue(list.contains(m2));
+        Assert.assertTrue(list.contains(m4));
+
+        // validate search by port
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setPort(1);
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(3, list.size());
+        Assert.assertTrue(list.contains(m1));
+        Assert.assertTrue(list.contains(m2));
+        Assert.assertTrue(list.contains(m3));
+
+        // validate search by stream id
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setStreamId("stream-4");
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(1, list.size());
+        Assert.assertTrue(list.contains(m4));
+
+        // validate 4 metrics (aggregations) found for m4 for all agglevels when searching by port
+        filter = new FilterOptions();
+        filter.setPort(2);
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(4, list.size());
+        Assert.assertTrue(list.contains(m4));
+        Assert.assertFalse(list.contains(m1));
+        Assert.assertFalse(list.contains(m2));
+        Assert.assertFalse(list.contains(m3));
+
+        // validate search by topology id and executor id
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        filter.setTopologyId("Topo-m1");
+        filter.setExecutorId("executor-1");
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(2, list.size());
+        Assert.assertTrue(list.contains(m2));
+        Assert.assertTrue(list.contains(m3));
+    }
+
+    @Test
+    public void testMetricCleanup() throws Exception {
+        FilterOptions filter;
+        List<Metric> list;
+
+        // Share some common metadata strings to validate they do not get deleted
+        String commonTopologyId = "topology-cleanup-2";
+        String commonStreamId = "stream-cleanup-5";
+        String defaultS = "default";
+        Metric m1 = new Metric(defaultS, 40000000L, commonTopologyId, 1.0,
+                "component-1", defaultS, "hostname-1", commonStreamId, 1, AggLevel.AGG_LEVEL_NONE);
+        Metric m2 = new Metric(defaultS, System.currentTimeMillis(), commonTopologyId, 1.0,
+                "component-1", "executor-1", defaultS, commonStreamId, 1, AggLevel.AGG_LEVEL_NONE);
+
+        store.insert(m1);
+        store.insert(m2);
+        waitForInsertFinish(m2);
+
+        // validate at least two agg level none metrics exist
+        filter = new FilterOptions();
+        filter.addAggLevel(AggLevel.AGG_LEVEL_NONE);
+        list = getMetricsFromScan(filter);
+        Assert.assertTrue(list.size() >= 2);
+
+        // delete anything older than an hour
+        MetricsCleaner cleaner = new MetricsCleaner((RocksDbStore)store, 1, 1, null);
+        cleaner.purgeMetrics();
+        list = getMetricsFromScan(filter);
+        Assert.assertEquals(1, list.size());
+        Assert.assertTrue(list.contains(m2));
+    }
+
+    private void waitForInsertFinish(Metric m) throws Exception {
+        Metric last = new Metric(m);
+        int attempts = 0;
+        do {
+            Thread.sleep(1);
+            attempts++;
+            if (attempts > 5000) {
+                throw new Exception("Insertion timing out");
+            }
+        } while (!store.populateValue(last));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbValueTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbValueTest.java b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbValueTest.java
new file mode 100644
index 0000000..fb05796
--- /dev/null
+++ b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbValueTest.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import org.apache.storm.metricstore.AggLevel;
+import org.apache.storm.metricstore.Metric;
+import org.apache.storm.metricstore.MetricException;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class RocksDbValueTest {
+
+    @Test
+    public void testMetadataConstructor() {
+        long timestamp = System.currentTimeMillis();
+        String s = "MyTopology123";
+        RocksDbValue value = new RocksDbValue(timestamp, s);
+        Assert.assertEquals(timestamp, value.getLastTimestamp());
+        Assert.assertEquals(s, value.getMetdataString());
+
+        RocksDbValue value2 = new RocksDbValue(value.getRaw());
+        Assert.assertEquals(timestamp, value2.getLastTimestamp());
+        Assert.assertEquals(s, value2.getMetdataString());
+
+        int stringId = 0x509;
+        RocksDbKey key = new RocksDbKey(KeyType.EXEC_ID_STRING, stringId);
+        StringMetadata metadata = value2.getStringMetadata(key);
+        Assert.assertEquals(stringId, metadata.getStringId());
+        Assert.assertEquals(timestamp, metadata.getLastTimestamp());
+        Assert.assertEquals(1, metadata.getMetadataTypes().size());
+        Assert.assertEquals(KeyType.EXEC_ID_STRING, metadata.getMetadataTypes().get(0));
+    }
+
+    @Test
+    public void testMetricConstructor() throws MetricException {
+        Metric m = new Metric("cpu", 1L,"myTopologyId123", 1,
+                "componentId1", "executorId1", "hostname1", "streamid1",
+                7777, AggLevel.AGG_LEVEL_NONE);
+        Metric m2 = new Metric(m);
+        Metric m3 = new Metric(m);
+
+        m.addValue(238);
+
+        RocksDbValue value = new RocksDbValue(m);
+        value.populateMetric(m2);
+        Assert.assertEquals(m.getValue(), m2.getValue(), 0x001);
+        Assert.assertEquals(m.getCount(), m2.getCount(), 0x001);
+        Assert.assertEquals(m.getSum(), m2.getSum(), 0x001);
+        Assert.assertEquals(m.getMin(), m2.getMin(), 0x001);
+        Assert.assertEquals(m.getMax(), m2.getMax(), 0x001);
+
+        RocksDbValue value2 = new RocksDbValue(value.getRaw());
+        value2.populateMetric(m3);
+        Assert.assertEquals(m.getValue(), m3.getValue(), 0x001);
+        Assert.assertEquals(m.getCount(), m3.getCount(), 0x001);
+        Assert.assertEquals(m.getSum(), m3.getSum(), 0x001);
+        Assert.assertEquals(m.getMin(), m3.getMin(), 0x001);
+        Assert.assertEquals(m.getMax(), m3.getMax(), 0x001);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/StringMetadataCacheTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/StringMetadataCacheTest.java b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/StringMetadataCacheTest.java
new file mode 100644
index 0000000..6ab5991
--- /dev/null
+++ b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/StringMetadataCacheTest.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import org.apache.storm.metricstore.MetricException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.rocksdb.RocksDB;
+
+public class StringMetadataCacheTest {
+
+    @Before
+    public void setUp() {
+        // remove any previously created cache instance
+        StringMetadataCache.cleanUp();
+        RocksDB.loadLibrary();
+    }
+
+    private class TestDbWriter extends RocksDbMetricsWriter {
+        boolean evictCalled = false;
+
+        TestDbWriter() {
+            super(null, null, null);
+        }
+
+        @Override
+        void handleEvictedMetadata(RocksDbKey key, RocksDbValue val) {
+            evictCalled = true;
+        }
+    }
+
+    @After
+    public void tearDown() {
+        StringMetadataCache.cleanUp();
+    }
+
+    @Test
+    public void validateEviction() throws MetricException {
+        TestDbWriter writer = new TestDbWriter();
+        StringMetadataCache.init(writer, 2);
+        WritableStringMetadataCache wCache = StringMetadataCache.getWritableStringMetadataCache();
+        ReadOnlyStringMetadataCache rCache = StringMetadataCache.getReadOnlyStringMetadataCache();
+
+        String s1 = "string1";
+        Integer s1Id = 1;
+        long s1Timestamp = 1L;
+        StringMetadata metadata1 = new StringMetadata(KeyType.STREAM_ID_STRING, s1Id, s1Timestamp);
+        wCache.put(s1, metadata1, false);
+        Assert.assertEquals(metadata1, rCache.get(s1));
+        Assert.assertTrue(rCache.contains(s1Id));
+        Assert.assertEquals(s1, rCache.getMetadataString(s1Id));
+
+        String s2 = "string2";
+        Integer s2Id = 2;
+        long s2Timestamp = 2L;
+        StringMetadata metadata2 = new StringMetadata(KeyType.EXEC_ID_STRING, s2Id, s2Timestamp);
+        wCache.put(s2, metadata2, false);
+        Assert.assertEquals(metadata2, rCache.get(s2));
+        Assert.assertTrue(rCache.contains(s2Id));
+        Assert.assertEquals(s2, rCache.getMetadataString(s2Id));
+
+        Assert.assertEquals(false, writer.evictCalled);
+
+        // read s1 last....  This should cause s2 to be evicted on next put
+        rCache.get(s1);
+
+        String s3 = "string3";
+        Integer s3Id = 3;
+        long s3Timestamp = 3L;
+        StringMetadata metadata3 = new StringMetadata(KeyType.TOPOLOGY_STRING, s3Id, s3Timestamp);
+        wCache.put(s3, metadata3, false);
+
+        Assert.assertEquals(true, writer.evictCalled);
+        Assert.assertEquals(metadata3, rCache.get(s3));
+        Assert.assertTrue(rCache.contains(s3Id));
+        Assert.assertEquals(s3, rCache.getMetadataString(s3Id));
+
+        // since s2 read last, it should be evicted, s1 and s3 should exist
+        Assert.assertEquals(null, rCache.get(s2));
+        Assert.assertFalse(rCache.contains(s2Id));
+        Assert.assertEquals(metadata1, rCache.get(s1));
+        Assert.assertTrue(rCache.contains(s1Id));
+        Assert.assertEquals(s1, rCache.getMetadataString(s1Id));
+
+        StringMetadataCache.cleanUp();
+    }
+
+    @Test
+    public void validateMultipleKeyTypes() throws MetricException {
+        TestDbWriter writer = new TestDbWriter();
+        StringMetadataCache.init(writer, 2);
+        WritableStringMetadataCache wCache = StringMetadataCache.getWritableStringMetadataCache();
+
+        StringMetadata metadata = new StringMetadata(KeyType.STREAM_ID_STRING, 1, 1L);
+        wCache.put("default", metadata, false);
+
+        metadata = wCache.get("default");
+        metadata.update(3L, KeyType.COMPONENT_STRING);
+
+        metadata = wCache.get("default");
+        metadata.update(2L, KeyType.STREAM_ID_STRING);
+
+        metadata = wCache.get("default");
+        Assert.assertEquals(2, metadata.getMetadataTypes().size());
+        Assert.assertTrue(metadata.getMetadataTypes().contains(KeyType.STREAM_ID_STRING));
+        Assert.assertTrue(metadata.getMetadataTypes().contains(KeyType.COMPONENT_STRING));
+        Assert.assertEquals(3L, metadata.getLastTimestamp());
+
+        StringMetadataCache.cleanUp();
+    }
+}


[7/7] storm git commit: Merge branch 'agresch_rocksdb' of https://github.com/agresch/storm into STORM-2887-merge

Posted by ka...@apache.org.
Merge branch 'agresch_rocksdb' of https://github.com/agresch/storm into STORM-2887-merge


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

Branch: refs/heads/master
Commit: e6a423dd8cada12997e2a9448a9944050cf23a1f
Parents: 879cb5b 48e23a9
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Tue Jan 23 06:23:11 2018 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Tue Jan 23 06:23:11 2018 +0900

----------------------------------------------------------------------
 conf/defaults.yaml                              |   6 +
 docs/index.md                                   |   1 +
 docs/storm-metricstore.md                       |  83 ++
 pom.xml                                         |   9 +
 .../org/apache/storm/generated/Assignment.java  | 288 +++---
 .../storm/generated/ClusterWorkerHeartbeat.java |  52 +-
 .../jvm/org/apache/storm/generated/HBNodes.java |  32 +-
 .../org/apache/storm/generated/HBRecords.java   |  36 +-
 .../storm/generated/LSApprovedWorkers.java      |  44 +-
 .../generated/LSSupervisorAssignments.java      |  48 +-
 .../apache/storm/generated/LSTopoHistory.java   |  64 +-
 .../storm/generated/LSTopoHistoryList.java      |  36 +-
 .../storm/generated/LSWorkerHeartbeat.java      |  36 +-
 .../apache/storm/generated/LocalAssignment.java |  36 +-
 .../apache/storm/generated/LocalStateData.java  |  48 +-
 .../org/apache/storm/generated/LogConfig.java   |  48 +-
 .../jvm/org/apache/storm/generated/Nimbus.java  | 809 ++++++++++++++++-
 .../org/apache/storm/generated/StormBase.java   |  92 +-
 .../storm/generated/TopologyHistoryInfo.java    |  32 +-
 .../storm/generated/WorkerMetricList.java       | 466 ++++++++++
 .../storm/generated/WorkerMetricPoint.java      | 903 +++++++++++++++++++
 .../apache/storm/generated/WorkerMetrics.java   | 712 +++++++++++++++
 .../apache/storm/generated/WorkerResources.java |  88 +-
 .../auth/authorizer/SimpleACLAuthorizer.java    |  24 +-
 .../jvm/org/apache/storm/utils/ConfigUtils.java |   4 +-
 .../src/jvm/org/apache/storm/utils/Utils.java   |  14 +-
 storm-client/src/py/storm/Nimbus-remote         |   7 +
 storm-client/src/py/storm/Nimbus.py             | 196 +++-
 storm-client/src/py/storm/ttypes.py             | 715 +++++++++++----
 storm-client/src/storm.thrift                   |  23 +
 storm-server/pom.xml                            |   4 +
 .../java/org/apache/storm/DaemonConfig.java     |  59 +-
 .../java/org/apache/storm/LocalCluster.java     |   8 +-
 .../org/apache/storm/daemon/nimbus/Nimbus.java  |  66 +-
 .../storm/daemon/supervisor/Container.java      |  61 +-
 .../apache/storm/daemon/supervisor/Slot.java    |   3 +
 .../storm/metric/StormMetricsRegistry.java      |   5 +
 .../org/apache/storm/metricstore/AggLevel.java  |  40 +
 .../apache/storm/metricstore/FilterOptions.java | 154 ++++
 .../org/apache/storm/metricstore/Metric.java    | 270 ++++++
 .../storm/metricstore/MetricException.java      |  32 +
 .../apache/storm/metricstore/MetricStore.java   |  74 ++
 .../storm/metricstore/MetricStoreConfig.java    |  45 +
 .../storm/metricstore/rocksdb/KeyType.java      |  70 ++
 .../metricstore/rocksdb/MetricsCleaner.java     |  98 ++
 .../rocksdb/ReadOnlyStringMetadataCache.java    |  52 ++
 .../storm/metricstore/rocksdb/RocksDbKey.java   | 228 +++++
 .../rocksdb/RocksDbMetricsWriter.java           | 320 +++++++
 .../storm/metricstore/rocksdb/RocksDbStore.java | 639 +++++++++++++
 .../storm/metricstore/rocksdb/RocksDbValue.java | 144 +++
 .../metricstore/rocksdb/StringMetadata.java     |  78 ++
 .../rocksdb/StringMetadataCache.java            | 202 +++++
 .../rocksdb/WritableStringMetadataCache.java    |  54 ++
 .../java/org/apache/storm/utils/LruMap.java     |  56 ++
 .../metricstore/rocksdb/RocksDbKeyTest.java     |  74 ++
 .../metricstore/rocksdb/RocksDbStoreTest.java   | 328 +++++++
 .../metricstore/rocksdb/RocksDbValueTest.java   |  76 ++
 .../rocksdb/StringMetadataCacheTest.java        | 129 +++
 58 files changed, 7571 insertions(+), 750 deletions(-)
----------------------------------------------------------------------



[4/7] storm git commit: STORM-2887: store metrics into RocksDB

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricPoint.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricPoint.java b/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricPoint.java
new file mode 100644
index 0000000..719277a
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricPoint.java
@@ -0,0 +1,903 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class WorkerMetricPoint implements org.apache.thrift.TBase<WorkerMetricPoint, WorkerMetricPoint._Fields>, java.io.Serializable, Cloneable, Comparable<WorkerMetricPoint> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WorkerMetricPoint");
+
+  private static final org.apache.thrift.protocol.TField METRIC_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("metricName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("timestamp", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField METRIC_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("metricValue", org.apache.thrift.protocol.TType.DOUBLE, (short)3);
+  private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("componentId", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField EXECUTOR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("executorId", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField STREAM_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("streamId", org.apache.thrift.protocol.TType.STRING, (short)6);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WorkerMetricPointStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WorkerMetricPointTupleSchemeFactory());
+  }
+
+  private String metricName; // required
+  private long timestamp; // required
+  private double metricValue; // required
+  private String componentId; // required
+  private String executorId; // required
+  private String streamId; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    METRIC_NAME((short)1, "metricName"),
+    TIMESTAMP((short)2, "timestamp"),
+    METRIC_VALUE((short)3, "metricValue"),
+    COMPONENT_ID((short)4, "componentId"),
+    EXECUTOR_ID((short)5, "executorId"),
+    STREAM_ID((short)6, "streamId");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // METRIC_NAME
+          return METRIC_NAME;
+        case 2: // TIMESTAMP
+          return TIMESTAMP;
+        case 3: // METRIC_VALUE
+          return METRIC_VALUE;
+        case 4: // COMPONENT_ID
+          return COMPONENT_ID;
+        case 5: // EXECUTOR_ID
+          return EXECUTOR_ID;
+        case 6: // STREAM_ID
+          return STREAM_ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIMESTAMP_ISSET_ID = 0;
+  private static final int __METRICVALUE_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.METRIC_NAME, new org.apache.thrift.meta_data.FieldMetaData("metricName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("timestamp", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.METRIC_VALUE, new org.apache.thrift.meta_data.FieldMetaData("metricValue", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("componentId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.EXECUTOR_ID, new org.apache.thrift.meta_data.FieldMetaData("executorId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.STREAM_ID, new org.apache.thrift.meta_data.FieldMetaData("streamId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WorkerMetricPoint.class, metaDataMap);
+  }
+
+  public WorkerMetricPoint() {
+  }
+
+  public WorkerMetricPoint(
+    String metricName,
+    long timestamp,
+    double metricValue,
+    String componentId,
+    String executorId,
+    String streamId)
+  {
+    this();
+    this.metricName = metricName;
+    this.timestamp = timestamp;
+    set_timestamp_isSet(true);
+    this.metricValue = metricValue;
+    set_metricValue_isSet(true);
+    this.componentId = componentId;
+    this.executorId = executorId;
+    this.streamId = streamId;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WorkerMetricPoint(WorkerMetricPoint other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_metricName()) {
+      this.metricName = other.metricName;
+    }
+    this.timestamp = other.timestamp;
+    this.metricValue = other.metricValue;
+    if (other.is_set_componentId()) {
+      this.componentId = other.componentId;
+    }
+    if (other.is_set_executorId()) {
+      this.executorId = other.executorId;
+    }
+    if (other.is_set_streamId()) {
+      this.streamId = other.streamId;
+    }
+  }
+
+  public WorkerMetricPoint deepCopy() {
+    return new WorkerMetricPoint(this);
+  }
+
+  @Override
+  public void clear() {
+    this.metricName = null;
+    set_timestamp_isSet(false);
+    this.timestamp = 0;
+    set_metricValue_isSet(false);
+    this.metricValue = 0.0;
+    this.componentId = null;
+    this.executorId = null;
+    this.streamId = null;
+  }
+
+  public String get_metricName() {
+    return this.metricName;
+  }
+
+  public void set_metricName(String metricName) {
+    this.metricName = metricName;
+  }
+
+  public void unset_metricName() {
+    this.metricName = null;
+  }
+
+  /** Returns true if field metricName is set (has been assigned a value) and false otherwise */
+  public boolean is_set_metricName() {
+    return this.metricName != null;
+  }
+
+  public void set_metricName_isSet(boolean value) {
+    if (!value) {
+      this.metricName = null;
+    }
+  }
+
+  public long get_timestamp() {
+    return this.timestamp;
+  }
+
+  public void set_timestamp(long timestamp) {
+    this.timestamp = timestamp;
+    set_timestamp_isSet(true);
+  }
+
+  public void unset_timestamp() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID);
+  }
+
+  /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */
+  public boolean is_set_timestamp() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID);
+  }
+
+  public void set_timestamp_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value);
+  }
+
+  public double get_metricValue() {
+    return this.metricValue;
+  }
+
+  public void set_metricValue(double metricValue) {
+    this.metricValue = metricValue;
+    set_metricValue_isSet(true);
+  }
+
+  public void unset_metricValue() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __METRICVALUE_ISSET_ID);
+  }
+
+  /** Returns true if field metricValue is set (has been assigned a value) and false otherwise */
+  public boolean is_set_metricValue() {
+    return EncodingUtils.testBit(__isset_bitfield, __METRICVALUE_ISSET_ID);
+  }
+
+  public void set_metricValue_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __METRICVALUE_ISSET_ID, value);
+  }
+
+  public String get_componentId() {
+    return this.componentId;
+  }
+
+  public void set_componentId(String componentId) {
+    this.componentId = componentId;
+  }
+
+  public void unset_componentId() {
+    this.componentId = null;
+  }
+
+  /** Returns true if field componentId is set (has been assigned a value) and false otherwise */
+  public boolean is_set_componentId() {
+    return this.componentId != null;
+  }
+
+  public void set_componentId_isSet(boolean value) {
+    if (!value) {
+      this.componentId = null;
+    }
+  }
+
+  public String get_executorId() {
+    return this.executorId;
+  }
+
+  public void set_executorId(String executorId) {
+    this.executorId = executorId;
+  }
+
+  public void unset_executorId() {
+    this.executorId = null;
+  }
+
+  /** Returns true if field executorId is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executorId() {
+    return this.executorId != null;
+  }
+
+  public void set_executorId_isSet(boolean value) {
+    if (!value) {
+      this.executorId = null;
+    }
+  }
+
+  public String get_streamId() {
+    return this.streamId;
+  }
+
+  public void set_streamId(String streamId) {
+    this.streamId = streamId;
+  }
+
+  public void unset_streamId() {
+    this.streamId = null;
+  }
+
+  /** Returns true if field streamId is set (has been assigned a value) and false otherwise */
+  public boolean is_set_streamId() {
+    return this.streamId != null;
+  }
+
+  public void set_streamId_isSet(boolean value) {
+    if (!value) {
+      this.streamId = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case METRIC_NAME:
+      if (value == null) {
+        unset_metricName();
+      } else {
+        set_metricName((String)value);
+      }
+      break;
+
+    case TIMESTAMP:
+      if (value == null) {
+        unset_timestamp();
+      } else {
+        set_timestamp((Long)value);
+      }
+      break;
+
+    case METRIC_VALUE:
+      if (value == null) {
+        unset_metricValue();
+      } else {
+        set_metricValue((Double)value);
+      }
+      break;
+
+    case COMPONENT_ID:
+      if (value == null) {
+        unset_componentId();
+      } else {
+        set_componentId((String)value);
+      }
+      break;
+
+    case EXECUTOR_ID:
+      if (value == null) {
+        unset_executorId();
+      } else {
+        set_executorId((String)value);
+      }
+      break;
+
+    case STREAM_ID:
+      if (value == null) {
+        unset_streamId();
+      } else {
+        set_streamId((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case METRIC_NAME:
+      return get_metricName();
+
+    case TIMESTAMP:
+      return get_timestamp();
+
+    case METRIC_VALUE:
+      return get_metricValue();
+
+    case COMPONENT_ID:
+      return get_componentId();
+
+    case EXECUTOR_ID:
+      return get_executorId();
+
+    case STREAM_ID:
+      return get_streamId();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case METRIC_NAME:
+      return is_set_metricName();
+    case TIMESTAMP:
+      return is_set_timestamp();
+    case METRIC_VALUE:
+      return is_set_metricValue();
+    case COMPONENT_ID:
+      return is_set_componentId();
+    case EXECUTOR_ID:
+      return is_set_executorId();
+    case STREAM_ID:
+      return is_set_streamId();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WorkerMetricPoint)
+      return this.equals((WorkerMetricPoint)that);
+    return false;
+  }
+
+  public boolean equals(WorkerMetricPoint that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_metricName = true && this.is_set_metricName();
+    boolean that_present_metricName = true && that.is_set_metricName();
+    if (this_present_metricName || that_present_metricName) {
+      if (!(this_present_metricName && that_present_metricName))
+        return false;
+      if (!this.metricName.equals(that.metricName))
+        return false;
+    }
+
+    boolean this_present_timestamp = true;
+    boolean that_present_timestamp = true;
+    if (this_present_timestamp || that_present_timestamp) {
+      if (!(this_present_timestamp && that_present_timestamp))
+        return false;
+      if (this.timestamp != that.timestamp)
+        return false;
+    }
+
+    boolean this_present_metricValue = true;
+    boolean that_present_metricValue = true;
+    if (this_present_metricValue || that_present_metricValue) {
+      if (!(this_present_metricValue && that_present_metricValue))
+        return false;
+      if (this.metricValue != that.metricValue)
+        return false;
+    }
+
+    boolean this_present_componentId = true && this.is_set_componentId();
+    boolean that_present_componentId = true && that.is_set_componentId();
+    if (this_present_componentId || that_present_componentId) {
+      if (!(this_present_componentId && that_present_componentId))
+        return false;
+      if (!this.componentId.equals(that.componentId))
+        return false;
+    }
+
+    boolean this_present_executorId = true && this.is_set_executorId();
+    boolean that_present_executorId = true && that.is_set_executorId();
+    if (this_present_executorId || that_present_executorId) {
+      if (!(this_present_executorId && that_present_executorId))
+        return false;
+      if (!this.executorId.equals(that.executorId))
+        return false;
+    }
+
+    boolean this_present_streamId = true && this.is_set_streamId();
+    boolean that_present_streamId = true && that.is_set_streamId();
+    if (this_present_streamId || that_present_streamId) {
+      if (!(this_present_streamId && that_present_streamId))
+        return false;
+      if (!this.streamId.equals(that.streamId))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_metricName = true && (is_set_metricName());
+    list.add(present_metricName);
+    if (present_metricName)
+      list.add(metricName);
+
+    boolean present_timestamp = true;
+    list.add(present_timestamp);
+    if (present_timestamp)
+      list.add(timestamp);
+
+    boolean present_metricValue = true;
+    list.add(present_metricValue);
+    if (present_metricValue)
+      list.add(metricValue);
+
+    boolean present_componentId = true && (is_set_componentId());
+    list.add(present_componentId);
+    if (present_componentId)
+      list.add(componentId);
+
+    boolean present_executorId = true && (is_set_executorId());
+    list.add(present_executorId);
+    if (present_executorId)
+      list.add(executorId);
+
+    boolean present_streamId = true && (is_set_streamId());
+    list.add(present_streamId);
+    if (present_streamId)
+      list.add(streamId);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WorkerMetricPoint other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_metricName()).compareTo(other.is_set_metricName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_metricName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metricName, other.metricName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_timestamp()).compareTo(other.is_set_timestamp());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_timestamp()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_metricValue()).compareTo(other.is_set_metricValue());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_metricValue()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metricValue, other.metricValue);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_componentId()).compareTo(other.is_set_componentId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_componentId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.componentId, other.componentId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executorId()).compareTo(other.is_set_executorId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executorId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executorId, other.executorId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_streamId()).compareTo(other.is_set_streamId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_streamId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.streamId, other.streamId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WorkerMetricPoint(");
+    boolean first = true;
+
+    sb.append("metricName:");
+    if (this.metricName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.metricName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("timestamp:");
+    sb.append(this.timestamp);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("metricValue:");
+    sb.append(this.metricValue);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("componentId:");
+    if (this.componentId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.componentId);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("executorId:");
+    if (this.executorId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.executorId);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("streamId:");
+    if (this.streamId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.streamId);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_metricName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'metricName' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_timestamp()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'timestamp' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_metricValue()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'metricValue' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_componentId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'componentId' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_executorId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'executorId' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_streamId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'streamId' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WorkerMetricPointStandardSchemeFactory implements SchemeFactory {
+    public WorkerMetricPointStandardScheme getScheme() {
+      return new WorkerMetricPointStandardScheme();
+    }
+  }
+
+  private static class WorkerMetricPointStandardScheme extends StandardScheme<WorkerMetricPoint> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WorkerMetricPoint struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // METRIC_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.metricName = iprot.readString();
+              struct.set_metricName_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TIMESTAMP
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.timestamp = iprot.readI64();
+              struct.set_timestamp_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // METRIC_VALUE
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.metricValue = iprot.readDouble();
+              struct.set_metricValue_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // COMPONENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.componentId = iprot.readString();
+              struct.set_componentId_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // EXECUTOR_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.executorId = iprot.readString();
+              struct.set_executorId_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // STREAM_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.streamId = iprot.readString();
+              struct.set_streamId_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WorkerMetricPoint struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.metricName != null) {
+        oprot.writeFieldBegin(METRIC_NAME_FIELD_DESC);
+        oprot.writeString(struct.metricName);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
+      oprot.writeI64(struct.timestamp);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(METRIC_VALUE_FIELD_DESC);
+      oprot.writeDouble(struct.metricValue);
+      oprot.writeFieldEnd();
+      if (struct.componentId != null) {
+        oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC);
+        oprot.writeString(struct.componentId);
+        oprot.writeFieldEnd();
+      }
+      if (struct.executorId != null) {
+        oprot.writeFieldBegin(EXECUTOR_ID_FIELD_DESC);
+        oprot.writeString(struct.executorId);
+        oprot.writeFieldEnd();
+      }
+      if (struct.streamId != null) {
+        oprot.writeFieldBegin(STREAM_ID_FIELD_DESC);
+        oprot.writeString(struct.streamId);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WorkerMetricPointTupleSchemeFactory implements SchemeFactory {
+    public WorkerMetricPointTupleScheme getScheme() {
+      return new WorkerMetricPointTupleScheme();
+    }
+  }
+
+  private static class WorkerMetricPointTupleScheme extends TupleScheme<WorkerMetricPoint> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WorkerMetricPoint struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.metricName);
+      oprot.writeI64(struct.timestamp);
+      oprot.writeDouble(struct.metricValue);
+      oprot.writeString(struct.componentId);
+      oprot.writeString(struct.executorId);
+      oprot.writeString(struct.streamId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WorkerMetricPoint struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.metricName = iprot.readString();
+      struct.set_metricName_isSet(true);
+      struct.timestamp = iprot.readI64();
+      struct.set_timestamp_isSet(true);
+      struct.metricValue = iprot.readDouble();
+      struct.set_metricValue_isSet(true);
+      struct.componentId = iprot.readString();
+      struct.set_componentId_isSet(true);
+      struct.executorId = iprot.readString();
+      struct.set_executorId_isSet(true);
+      struct.streamId = iprot.readString();
+      struct.set_streamId_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/WorkerMetrics.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/WorkerMetrics.java b/storm-client/src/jvm/org/apache/storm/generated/WorkerMetrics.java
new file mode 100644
index 0000000..f7813fd
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/WorkerMetrics.java
@@ -0,0 +1,712 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class WorkerMetrics implements org.apache.thrift.TBase<WorkerMetrics, WorkerMetrics._Fields>, java.io.Serializable, Cloneable, Comparable<WorkerMetrics> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WorkerMetrics");
+
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topologyId", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField HOSTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("hostname", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField METRIC_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("metricList", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WorkerMetricsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WorkerMetricsTupleSchemeFactory());
+  }
+
+  private String topologyId; // required
+  private int port; // required
+  private String hostname; // required
+  private WorkerMetricList metricList; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TOPOLOGY_ID((short)1, "topologyId"),
+    PORT((short)2, "port"),
+    HOSTNAME((short)3, "hostname"),
+    METRIC_LIST((short)4, "metricList");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TOPOLOGY_ID
+          return TOPOLOGY_ID;
+        case 2: // PORT
+          return PORT;
+        case 3: // HOSTNAME
+          return HOSTNAME;
+        case 4: // METRIC_LIST
+          return METRIC_LIST;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __PORT_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topologyId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.METRIC_LIST, new org.apache.thrift.meta_data.FieldMetaData("metricList", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerMetricList.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WorkerMetrics.class, metaDataMap);
+  }
+
+  public WorkerMetrics() {
+  }
+
+  public WorkerMetrics(
+    String topologyId,
+    int port,
+    String hostname,
+    WorkerMetricList metricList)
+  {
+    this();
+    this.topologyId = topologyId;
+    this.port = port;
+    set_port_isSet(true);
+    this.hostname = hostname;
+    this.metricList = metricList;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WorkerMetrics(WorkerMetrics other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_topologyId()) {
+      this.topologyId = other.topologyId;
+    }
+    this.port = other.port;
+    if (other.is_set_hostname()) {
+      this.hostname = other.hostname;
+    }
+    if (other.is_set_metricList()) {
+      this.metricList = new WorkerMetricList(other.metricList);
+    }
+  }
+
+  public WorkerMetrics deepCopy() {
+    return new WorkerMetrics(this);
+  }
+
+  @Override
+  public void clear() {
+    this.topologyId = null;
+    set_port_isSet(false);
+    this.port = 0;
+    this.hostname = null;
+    this.metricList = null;
+  }
+
+  public String get_topologyId() {
+    return this.topologyId;
+  }
+
+  public void set_topologyId(String topologyId) {
+    this.topologyId = topologyId;
+  }
+
+  public void unset_topologyId() {
+    this.topologyId = null;
+  }
+
+  /** Returns true if field topologyId is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topologyId() {
+    return this.topologyId != null;
+  }
+
+  public void set_topologyId_isSet(boolean value) {
+    if (!value) {
+      this.topologyId = null;
+    }
+  }
+
+  public int get_port() {
+    return this.port;
+  }
+
+  public void set_port(int port) {
+    this.port = port;
+    set_port_isSet(true);
+  }
+
+  public void unset_port() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  /** Returns true if field port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID);
+  }
+
+  public void set_port_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value);
+  }
+
+  public String get_hostname() {
+    return this.hostname;
+  }
+
+  public void set_hostname(String hostname) {
+    this.hostname = hostname;
+  }
+
+  public void unset_hostname() {
+    this.hostname = null;
+  }
+
+  /** Returns true if field hostname is set (has been assigned a value) and false otherwise */
+  public boolean is_set_hostname() {
+    return this.hostname != null;
+  }
+
+  public void set_hostname_isSet(boolean value) {
+    if (!value) {
+      this.hostname = null;
+    }
+  }
+
+  public WorkerMetricList get_metricList() {
+    return this.metricList;
+  }
+
+  public void set_metricList(WorkerMetricList metricList) {
+    this.metricList = metricList;
+  }
+
+  public void unset_metricList() {
+    this.metricList = null;
+  }
+
+  /** Returns true if field metricList is set (has been assigned a value) and false otherwise */
+  public boolean is_set_metricList() {
+    return this.metricList != null;
+  }
+
+  public void set_metricList_isSet(boolean value) {
+    if (!value) {
+      this.metricList = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TOPOLOGY_ID:
+      if (value == null) {
+        unset_topologyId();
+      } else {
+        set_topologyId((String)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Integer)value);
+      }
+      break;
+
+    case HOSTNAME:
+      if (value == null) {
+        unset_hostname();
+      } else {
+        set_hostname((String)value);
+      }
+      break;
+
+    case METRIC_LIST:
+      if (value == null) {
+        unset_metricList();
+      } else {
+        set_metricList((WorkerMetricList)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TOPOLOGY_ID:
+      return get_topologyId();
+
+    case PORT:
+      return get_port();
+
+    case HOSTNAME:
+      return get_hostname();
+
+    case METRIC_LIST:
+      return get_metricList();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TOPOLOGY_ID:
+      return is_set_topologyId();
+    case PORT:
+      return is_set_port();
+    case HOSTNAME:
+      return is_set_hostname();
+    case METRIC_LIST:
+      return is_set_metricList();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WorkerMetrics)
+      return this.equals((WorkerMetrics)that);
+    return false;
+  }
+
+  public boolean equals(WorkerMetrics that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_topologyId = true && this.is_set_topologyId();
+    boolean that_present_topologyId = true && that.is_set_topologyId();
+    if (this_present_topologyId || that_present_topologyId) {
+      if (!(this_present_topologyId && that_present_topologyId))
+        return false;
+      if (!this.topologyId.equals(that.topologyId))
+        return false;
+    }
+
+    boolean this_present_port = true;
+    boolean that_present_port = true;
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (this.port != that.port)
+        return false;
+    }
+
+    boolean this_present_hostname = true && this.is_set_hostname();
+    boolean that_present_hostname = true && that.is_set_hostname();
+    if (this_present_hostname || that_present_hostname) {
+      if (!(this_present_hostname && that_present_hostname))
+        return false;
+      if (!this.hostname.equals(that.hostname))
+        return false;
+    }
+
+    boolean this_present_metricList = true && this.is_set_metricList();
+    boolean that_present_metricList = true && that.is_set_metricList();
+    if (this_present_metricList || that_present_metricList) {
+      if (!(this_present_metricList && that_present_metricList))
+        return false;
+      if (!this.metricList.equals(that.metricList))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_topologyId = true && (is_set_topologyId());
+    list.add(present_topologyId);
+    if (present_topologyId)
+      list.add(topologyId);
+
+    boolean present_port = true;
+    list.add(present_port);
+    if (present_port)
+      list.add(port);
+
+    boolean present_hostname = true && (is_set_hostname());
+    list.add(present_hostname);
+    if (present_hostname)
+      list.add(hostname);
+
+    boolean present_metricList = true && (is_set_metricList());
+    list.add(present_metricList);
+    if (present_metricList)
+      list.add(metricList);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WorkerMetrics other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_topologyId()).compareTo(other.is_set_topologyId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topologyId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologyId, other.topologyId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_hostname()).compareTo(other.is_set_hostname());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_hostname()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, other.hostname);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_metricList()).compareTo(other.is_set_metricList());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_metricList()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metricList, other.metricList);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WorkerMetrics(");
+    boolean first = true;
+
+    sb.append("topologyId:");
+    if (this.topologyId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topologyId);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("port:");
+    sb.append(this.port);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("hostname:");
+    if (this.hostname == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.hostname);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("metricList:");
+    if (this.metricList == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.metricList);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_topologyId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'topologyId' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_port()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_hostname()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'hostname' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_metricList()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'metricList' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (metricList != null) {
+      metricList.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WorkerMetricsStandardSchemeFactory implements SchemeFactory {
+    public WorkerMetricsStandardScheme getScheme() {
+      return new WorkerMetricsStandardScheme();
+    }
+  }
+
+  private static class WorkerMetricsStandardScheme extends StandardScheme<WorkerMetrics> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WorkerMetrics struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TOPOLOGY_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topologyId = iprot.readString();
+              struct.set_topologyId_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.port = iprot.readI32();
+              struct.set_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // HOSTNAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.hostname = iprot.readString();
+              struct.set_hostname_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // METRIC_LIST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.metricList = new WorkerMetricList();
+              struct.metricList.read(iprot);
+              struct.set_metricList_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WorkerMetrics struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.topologyId != null) {
+        oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC);
+        oprot.writeString(struct.topologyId);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(PORT_FIELD_DESC);
+      oprot.writeI32(struct.port);
+      oprot.writeFieldEnd();
+      if (struct.hostname != null) {
+        oprot.writeFieldBegin(HOSTNAME_FIELD_DESC);
+        oprot.writeString(struct.hostname);
+        oprot.writeFieldEnd();
+      }
+      if (struct.metricList != null) {
+        oprot.writeFieldBegin(METRIC_LIST_FIELD_DESC);
+        struct.metricList.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WorkerMetricsTupleSchemeFactory implements SchemeFactory {
+    public WorkerMetricsTupleScheme getScheme() {
+      return new WorkerMetricsTupleScheme();
+    }
+  }
+
+  private static class WorkerMetricsTupleScheme extends TupleScheme<WorkerMetrics> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WorkerMetrics struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.topologyId);
+      oprot.writeI32(struct.port);
+      oprot.writeString(struct.hostname);
+      struct.metricList.write(oprot);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WorkerMetrics struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.topologyId = iprot.readString();
+      struct.set_topologyId_isSet(true);
+      struct.port = iprot.readI32();
+      struct.set_port_isSet(true);
+      struct.hostname = iprot.readString();
+      struct.set_hostname_isSet(true);
+      struct.metricList = new WorkerMetricList();
+      struct.metricList.read(iprot);
+      struct.set_metricList_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/WorkerResources.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/WorkerResources.java b/storm-client/src/jvm/org/apache/storm/generated/WorkerResources.java
index 56e2348..d25b5b0 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/WorkerResources.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/WorkerResources.java
@@ -876,15 +876,15 @@ public class WorkerResources implements org.apache.thrift.TBase<WorkerResources,
           case 6: // RESOURCES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map646 = iprot.readMapBegin();
-                struct.resources = new HashMap<String,Double>(2*_map646.size);
-                String _key647;
-                double _val648;
-                for (int _i649 = 0; _i649 < _map646.size; ++_i649)
+                org.apache.thrift.protocol.TMap _map666 = iprot.readMapBegin();
+                struct.resources = new HashMap<String,Double>(2*_map666.size);
+                String _key667;
+                double _val668;
+                for (int _i669 = 0; _i669 < _map666.size; ++_i669)
                 {
-                  _key647 = iprot.readString();
-                  _val648 = iprot.readDouble();
-                  struct.resources.put(_key647, _val648);
+                  _key667 = iprot.readString();
+                  _val668 = iprot.readDouble();
+                  struct.resources.put(_key667, _val668);
                 }
                 iprot.readMapEnd();
               }
@@ -896,15 +896,15 @@ public class WorkerResources implements org.apache.thrift.TBase<WorkerResources,
           case 7: // SHARED_RESOURCES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map650 = iprot.readMapBegin();
-                struct.shared_resources = new HashMap<String,Double>(2*_map650.size);
-                String _key651;
-                double _val652;
-                for (int _i653 = 0; _i653 < _map650.size; ++_i653)
+                org.apache.thrift.protocol.TMap _map670 = iprot.readMapBegin();
+                struct.shared_resources = new HashMap<String,Double>(2*_map670.size);
+                String _key671;
+                double _val672;
+                for (int _i673 = 0; _i673 < _map670.size; ++_i673)
                 {
-                  _key651 = iprot.readString();
-                  _val652 = iprot.readDouble();
-                  struct.shared_resources.put(_key651, _val652);
+                  _key671 = iprot.readString();
+                  _val672 = iprot.readDouble();
+                  struct.shared_resources.put(_key671, _val672);
                 }
                 iprot.readMapEnd();
               }
@@ -956,10 +956,10 @@ public class WorkerResources implements org.apache.thrift.TBase<WorkerResources,
           oprot.writeFieldBegin(RESOURCES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.resources.size()));
-            for (Map.Entry<String, Double> _iter654 : struct.resources.entrySet())
+            for (Map.Entry<String, Double> _iter674 : struct.resources.entrySet())
             {
-              oprot.writeString(_iter654.getKey());
-              oprot.writeDouble(_iter654.getValue());
+              oprot.writeString(_iter674.getKey());
+              oprot.writeDouble(_iter674.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -971,10 +971,10 @@ public class WorkerResources implements org.apache.thrift.TBase<WorkerResources,
           oprot.writeFieldBegin(SHARED_RESOURCES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.shared_resources.size()));
-            for (Map.Entry<String, Double> _iter655 : struct.shared_resources.entrySet())
+            for (Map.Entry<String, Double> _iter675 : struct.shared_resources.entrySet())
             {
-              oprot.writeString(_iter655.getKey());
-              oprot.writeDouble(_iter655.getValue());
+              oprot.writeString(_iter675.getKey());
+              oprot.writeDouble(_iter675.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1039,20 +1039,20 @@ public class WorkerResources implements org.apache.thrift.TBase<WorkerResources,
       if (struct.is_set_resources()) {
         {
           oprot.writeI32(struct.resources.size());
-          for (Map.Entry<String, Double> _iter656 : struct.resources.entrySet())
+          for (Map.Entry<String, Double> _iter676 : struct.resources.entrySet())
           {
-            oprot.writeString(_iter656.getKey());
-            oprot.writeDouble(_iter656.getValue());
+            oprot.writeString(_iter676.getKey());
+            oprot.writeDouble(_iter676.getValue());
           }
         }
       }
       if (struct.is_set_shared_resources()) {
         {
           oprot.writeI32(struct.shared_resources.size());
-          for (Map.Entry<String, Double> _iter657 : struct.shared_resources.entrySet())
+          for (Map.Entry<String, Double> _iter677 : struct.shared_resources.entrySet())
           {
-            oprot.writeString(_iter657.getKey());
-            oprot.writeDouble(_iter657.getValue());
+            oprot.writeString(_iter677.getKey());
+            oprot.writeDouble(_iter677.getValue());
           }
         }
       }
@@ -1084,30 +1084,30 @@ public class WorkerResources implements org.apache.thrift.TBase<WorkerResources,
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TMap _map658 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
-          struct.resources = new HashMap<String,Double>(2*_map658.size);
-          String _key659;
-          double _val660;
-          for (int _i661 = 0; _i661 < _map658.size; ++_i661)
+          org.apache.thrift.protocol.TMap _map678 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.resources = new HashMap<String,Double>(2*_map678.size);
+          String _key679;
+          double _val680;
+          for (int _i681 = 0; _i681 < _map678.size; ++_i681)
           {
-            _key659 = iprot.readString();
-            _val660 = iprot.readDouble();
-            struct.resources.put(_key659, _val660);
+            _key679 = iprot.readString();
+            _val680 = iprot.readDouble();
+            struct.resources.put(_key679, _val680);
           }
         }
         struct.set_resources_isSet(true);
       }
       if (incoming.get(6)) {
         {
-          org.apache.thrift.protocol.TMap _map662 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
-          struct.shared_resources = new HashMap<String,Double>(2*_map662.size);
-          String _key663;
-          double _val664;
-          for (int _i665 = 0; _i665 < _map662.size; ++_i665)
+          org.apache.thrift.protocol.TMap _map682 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.shared_resources = new HashMap<String,Double>(2*_map682.size);
+          String _key683;
+          double _val684;
+          for (int _i685 = 0; _i685 < _map682.size; ++_i685)
           {
-            _key663 = iprot.readString();
-            _val664 = iprot.readDouble();
-            struct.shared_resources.put(_key663, _val664);
+            _key683 = iprot.readString();
+            _val684 = iprot.readDouble();
+            struct.shared_resources.put(_key683, _val684);
           }
         }
         struct.set_shared_resources_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java
index dcc4854..05247f9 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java
@@ -18,12 +18,12 @@
 
 package org.apache.storm.security.auth.authorizer;
 
+import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
-import java.util.HashSet;
-import java.util.Collection;
-import java.io.IOException;
 
 import org.apache.storm.Config;
 import org.apache.storm.security.auth.IAuthorizer;
@@ -49,7 +49,9 @@ public class SimpleACLAuthorizer implements IAuthorizer {
             "getClusterInfo",
             "getSupervisorPageInfo",
             "getOwnerResourceSummaries"));
-    protected Set<String> supervisorCommands = new HashSet<>(Arrays.asList("fileDownload"));
+    protected Set<String> supervisorCommands = new HashSet<>(Arrays.asList(
+            "fileDownload",
+            "processWorkerMetrics"));
     protected Set<String> topoReadOnlyCommands = new HashSet<>(Arrays.asList(
             "getTopologyConf",
             "getTopology",
@@ -86,8 +88,9 @@ public class SimpleACLAuthorizer implements IAuthorizer {
     protected Set<String> nimbusGroups;
     protected IPrincipalToLocal ptol;
     protected IGroupMappingServiceProvider groupMappingServiceProvider;
+
     /**
-     * Invoked once immediately after construction
+     * Invoked once immediately after construction.
      * @param conf Storm configuration
      */
     @Override
@@ -123,7 +126,7 @@ public class SimpleACLAuthorizer implements IAuthorizer {
     }
 
     /**
-     * permit() method is invoked for each incoming Thrift request
+     * permit() method is invoked for each incoming Thrift request.
      * @param context request context includes info about
      * @param operation operation name
      * @param topoConf configuration of targeted topology
@@ -138,7 +141,7 @@ public class SimpleACLAuthorizer implements IAuthorizer {
         if (groupMappingServiceProvider != null) {
             try {
                 userGroups = groupMappingServiceProvider.getGroups(user);
-            } catch(IOException e) {
+            } catch (IOException e) {
                 LOG.warn("Error while trying to fetch user groups",e);
             }
         }
@@ -169,7 +172,7 @@ public class SimpleACLAuthorizer implements IAuthorizer {
     }
 
     private Boolean checkTopoPermission(String principal, String user, Set<String> userGroups,
-                                        Map<String, Object> topoConf, String userConfigKey, String groupConfigKey){
+                                        Map<String, Object> topoConf, String userConfigKey, String groupConfigKey) {
         Set<String> configuredUsers = new HashSet<>();
 
         if (topoConf.containsKey(userConfigKey)) {
@@ -189,10 +192,11 @@ public class SimpleACLAuthorizer implements IAuthorizer {
     }
 
     private Boolean checkUserGroupAllowed(Set<String> userGroups, Set<String> configuredGroups) {
-        if(userGroups.size() > 0 && configuredGroups.size() > 0) {
+        if (userGroups.size() > 0 && configuredGroups.size() > 0) {
             for (String tgroup : configuredGroups) {
-                if(userGroups.contains(tgroup))
+                if (userGroups.contains(tgroup)) {
                     return true;
+                }
             }
         }
         return false;

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
index 1c8015a..1bc94ac 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
@@ -217,7 +217,6 @@ public class ConfigUtils {
     }
 
     public static String absoluteStormBlobStoreDir(Map<String, Object> conf) {
-        String stormHome = System.getProperty("storm.home");
         String blobStoreDir = (String) conf.get(Config.BLOBSTORE_DIR);
         if (blobStoreDir == null) {
             return ConfigUtils.absoluteStormLocalDir(conf);
@@ -225,6 +224,7 @@ public class ConfigUtils {
             if (new File(blobStoreDir).isAbsolute()) {
                 return blobStoreDir;
             } else {
+                String stormHome = System.getProperty("storm.home");
                 return (stormHome + FILE_SEPARATOR + blobStoreDir);
             }
         }
@@ -273,7 +273,7 @@ public class ConfigUtils {
     public static Map overrideLoginConfigWithSystemProperty(Map<String, Object> conf) { // note that we delete the return value
         String loginConfFile = System.getProperty("java.security.auth.login.config");
         if (loginConfFile != null) {
-             conf.put("java.security.auth.login.config", loginConfFile);
+            conf.put("java.security.auth.login.config", loginConfFile);
         }
         return conf;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/utils/Utils.java b/storm-client/src/jvm/org/apache/storm/utils/Utils.java
index 4ad2ee2..73cfc81 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/Utils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/Utils.java
@@ -291,12 +291,22 @@ public class Utils {
      * runtime to avoid any zombie process in case cleanup function hangs.
      */
     public static void addShutdownHookWithForceKillIn1Sec (Runnable func) {
+        addShutdownHookWithDelayedForceKill(func, 1);
+    }
+
+    /**
+     * Adds the user supplied function as a shutdown hook for cleanup.
+     * Also adds a function that sleeps for numSecs and then halts the
+     * runtime to avoid any zombie process in case cleanup function hangs.
+     */
+    public static void addShutdownHookWithDelayedForceKill (Runnable func, int numSecs) {
         Runnable sleepKill = new Runnable() {
             @Override
             public void run() {
                 try {
-                    Time.sleepSecs(1);
-                    LOG.warn("Forceing Halt...");
+                    LOG.info("Halting after {} seconds", numSecs);
+                    Time.sleepSecs(numSecs);
+                    LOG.warn("Forcing Halt...");
                     Runtime.getRuntime().halt(20);
                 } catch (Exception e) {
                     LOG.warn("Exception in the ShutDownHook", e);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/py/storm/Nimbus-remote
----------------------------------------------------------------------
diff --git a/storm-client/src/py/storm/Nimbus-remote b/storm-client/src/py/storm/Nimbus-remote
index 7b080cf..1ce91e4 100644
--- a/storm-client/src/py/storm/Nimbus-remote
+++ b/storm-client/src/py/storm/Nimbus-remote
@@ -88,6 +88,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  StormTopology getUserTopology(string id)')
   print('  TopologyHistoryInfo getTopologyHistory(string user)')
   print('   getOwnerResourceSummaries(string owner)')
+  print('  void processWorkerMetrics(WorkerMetrics metrics)')
   print('')
   sys.exit(0)
 
@@ -420,6 +421,12 @@ elif cmd == 'getOwnerResourceSummaries':
     sys.exit(1)
   pp.pprint(client.getOwnerResourceSummaries(args[0],))
 
+elif cmd == 'processWorkerMetrics':
+  if len(args) != 1:
+    print('processWorkerMetrics requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.processWorkerMetrics(eval(args[0]),))
+
 else:
   print('Unrecognized method %s' % cmd)
   sys.exit(1)

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/py/storm/Nimbus.py
----------------------------------------------------------------------
diff --git a/storm-client/src/py/storm/Nimbus.py b/storm-client/src/py/storm/Nimbus.py
index 522921b..5f9f324 100644
--- a/storm-client/src/py/storm/Nimbus.py
+++ b/storm-client/src/py/storm/Nimbus.py
@@ -383,6 +383,13 @@ class Iface:
     """
     pass
 
+  def processWorkerMetrics(self, metrics):
+    """
+    Parameters:
+     - metrics
+    """
+    pass
+
 
 class Client(Iface):
   def __init__(self, iprot, oprot=None):
@@ -1967,6 +1974,35 @@ class Client(Iface):
       raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "getOwnerResourceSummaries failed: unknown result")
 
+  def processWorkerMetrics(self, metrics):
+    """
+    Parameters:
+     - metrics
+    """
+    self.send_processWorkerMetrics(metrics)
+    self.recv_processWorkerMetrics()
+
+  def send_processWorkerMetrics(self, metrics):
+    self._oprot.writeMessageBegin('processWorkerMetrics', TMessageType.CALL, self._seqid)
+    args = processWorkerMetrics_args()
+    args.metrics = metrics
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_processWorkerMetrics(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = processWorkerMetrics_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    return
+
 
 class Processor(Iface, TProcessor):
   def __init__(self, handler):
@@ -2018,6 +2054,7 @@ class Processor(Iface, TProcessor):
     self._processMap["getUserTopology"] = Processor.process_getUserTopology
     self._processMap["getTopologyHistory"] = Processor.process_getTopologyHistory
     self._processMap["getOwnerResourceSummaries"] = Processor.process_getOwnerResourceSummaries
+    self._processMap["processWorkerMetrics"] = Processor.process_processWorkerMetrics
 
   def process(self, iprot, oprot):
     (name, type, seqid) = iprot.readMessageBegin()
@@ -3115,6 +3152,25 @@ class Processor(Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_processWorkerMetrics(self, seqid, iprot, oprot):
+    args = processWorkerMetrics_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = processWorkerMetrics_result()
+    try:
+      self._handler.processWorkerMetrics(args.metrics)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("processWorkerMetrics", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
 
 # HELPER FUNCTIONS AND STRUCTURES
 
@@ -4977,11 +5033,11 @@ class getComponentPendingProfileActions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype776, _size773) = iprot.readListBegin()
-          for _i777 in xrange(_size773):
-            _elem778 = ProfileRequest()
-            _elem778.read(iprot)
-            self.success.append(_elem778)
+          (_etype801, _size798) = iprot.readListBegin()
+          for _i802 in xrange(_size798):
+            _elem803 = ProfileRequest()
+            _elem803.read(iprot)
+            self.success.append(_elem803)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -4998,8 +5054,8 @@ class getComponentPendingProfileActions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter779 in self.success:
-        iter779.write(oprot)
+      for iter804 in self.success:
+        iter804.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10070,11 +10126,11 @@ class getOwnerResourceSummaries_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype783, _size780) = iprot.readListBegin()
-          for _i784 in xrange(_size780):
-            _elem785 = OwnerResourceSummary()
-            _elem785.read(iprot)
-            self.success.append(_elem785)
+          (_etype808, _size805) = iprot.readListBegin()
+          for _i809 in xrange(_size805):
+            _elem810 = OwnerResourceSummary()
+            _elem810.read(iprot)
+            self.success.append(_elem810)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10097,8 +10153,8 @@ class getOwnerResourceSummaries_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter786 in self.success:
-        iter786.write(oprot)
+      for iter811 in self.success:
+        iter811.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.aze is not None:
@@ -10128,3 +10184,115 @@ class getOwnerResourceSummaries_result:
 
   def __ne__(self, other):
     return not (self == other)
+
+class processWorkerMetrics_args:
+  """
+  Attributes:
+   - metrics
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'metrics', (WorkerMetrics, WorkerMetrics.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, metrics=None,):
+    self.metrics = metrics
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.metrics = WorkerMetrics()
+          self.metrics.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('processWorkerMetrics_args')
+    if self.metrics is not None:
+      oprot.writeFieldBegin('metrics', TType.STRUCT, 1)
+      self.metrics.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.metrics)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class processWorkerMetrics_result:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('processWorkerMetrics_result')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)


[6/7] storm git commit: STORM-2887: store metrics into RocksDB

Posted by ka...@apache.org.
STORM-2887: store metrics into RocksDB


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

Branch: refs/heads/master
Commit: 48e23a99bf90c50f5d4864dbd449b0079ab2d5e2
Parents: 879cb5b
Author: Aaron Gresch <ag...@yahoo-inc.com>
Authored: Thu Dec 7 11:36:03 2017 -0600
Committer: Aaron Gresch <ag...@yahoo-inc.com>
Committed: Mon Jan 22 09:25:57 2018 -0600

----------------------------------------------------------------------
 conf/defaults.yaml                              |   6 +
 docs/index.md                                   |   1 +
 docs/storm-metricstore.md                       |  83 ++
 pom.xml                                         |   9 +
 .../org/apache/storm/generated/Assignment.java  | 288 +++---
 .../storm/generated/ClusterWorkerHeartbeat.java |  52 +-
 .../jvm/org/apache/storm/generated/HBNodes.java |  32 +-
 .../org/apache/storm/generated/HBRecords.java   |  36 +-
 .../storm/generated/LSApprovedWorkers.java      |  44 +-
 .../generated/LSSupervisorAssignments.java      |  48 +-
 .../apache/storm/generated/LSTopoHistory.java   |  64 +-
 .../storm/generated/LSTopoHistoryList.java      |  36 +-
 .../storm/generated/LSWorkerHeartbeat.java      |  36 +-
 .../apache/storm/generated/LocalAssignment.java |  36 +-
 .../apache/storm/generated/LocalStateData.java  |  48 +-
 .../org/apache/storm/generated/LogConfig.java   |  48 +-
 .../jvm/org/apache/storm/generated/Nimbus.java  | 809 ++++++++++++++++-
 .../org/apache/storm/generated/StormBase.java   |  92 +-
 .../storm/generated/TopologyHistoryInfo.java    |  32 +-
 .../storm/generated/WorkerMetricList.java       | 466 ++++++++++
 .../storm/generated/WorkerMetricPoint.java      | 903 +++++++++++++++++++
 .../apache/storm/generated/WorkerMetrics.java   | 712 +++++++++++++++
 .../apache/storm/generated/WorkerResources.java |  88 +-
 .../auth/authorizer/SimpleACLAuthorizer.java    |  24 +-
 .../jvm/org/apache/storm/utils/ConfigUtils.java |   4 +-
 .../src/jvm/org/apache/storm/utils/Utils.java   |  14 +-
 storm-client/src/py/storm/Nimbus-remote         |   7 +
 storm-client/src/py/storm/Nimbus.py             | 196 +++-
 storm-client/src/py/storm/ttypes.py             | 715 +++++++++++----
 storm-client/src/storm.thrift                   |  23 +
 storm-server/pom.xml                            |   4 +
 .../java/org/apache/storm/DaemonConfig.java     |  59 +-
 .../java/org/apache/storm/LocalCluster.java     |   8 +-
 .../org/apache/storm/daemon/nimbus/Nimbus.java  |  66 +-
 .../storm/daemon/supervisor/Container.java      |  61 +-
 .../apache/storm/daemon/supervisor/Slot.java    |   3 +
 .../storm/metric/StormMetricsRegistry.java      |   5 +
 .../org/apache/storm/metricstore/AggLevel.java  |  40 +
 .../apache/storm/metricstore/FilterOptions.java | 154 ++++
 .../org/apache/storm/metricstore/Metric.java    | 270 ++++++
 .../storm/metricstore/MetricException.java      |  32 +
 .../apache/storm/metricstore/MetricStore.java   |  74 ++
 .../storm/metricstore/MetricStoreConfig.java    |  45 +
 .../storm/metricstore/rocksdb/KeyType.java      |  70 ++
 .../metricstore/rocksdb/MetricsCleaner.java     |  98 ++
 .../rocksdb/ReadOnlyStringMetadataCache.java    |  52 ++
 .../storm/metricstore/rocksdb/RocksDbKey.java   | 228 +++++
 .../rocksdb/RocksDbMetricsWriter.java           | 320 +++++++
 .../storm/metricstore/rocksdb/RocksDbStore.java | 639 +++++++++++++
 .../storm/metricstore/rocksdb/RocksDbValue.java | 144 +++
 .../metricstore/rocksdb/StringMetadata.java     |  78 ++
 .../rocksdb/StringMetadataCache.java            | 202 +++++
 .../rocksdb/WritableStringMetadataCache.java    |  54 ++
 .../java/org/apache/storm/utils/LruMap.java     |  56 ++
 .../metricstore/rocksdb/RocksDbKeyTest.java     |  74 ++
 .../metricstore/rocksdb/RocksDbStoreTest.java   | 328 +++++++
 .../metricstore/rocksdb/RocksDbValueTest.java   |  76 ++
 .../rocksdb/StringMetadataCacheTest.java        | 129 +++
 58 files changed, 7571 insertions(+), 750 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 2260a0b..e15a265 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -295,6 +295,12 @@ pacemaker.thrift.message.size.max: 10485760
 storm.daemon.metrics.reporter.plugins:
      - "org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter"
 
+storm.metricstore.class: "org.apache.storm.metricstore.rocksdb.RocksDbStore"
+storm.metricstore.rocksdb.location: "storm_rocks"
+storm.metricstore.rocksdb.create_if_missing: true
+storm.metricstore.rocksdb.metadata_string_cache_capacity: 4000
+storm.metricstore.rocksdb.retention_hours: 240
+
 # configuration of cluster metrics consumer
 storm.cluster.metrics.consumer.publish.interval.secs: 60
 

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index a0ee350..af7c182 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -122,4 +122,5 @@ But small change will not affect the user experience. We will notify the user wh
 * [Defining a non-JVM language DSL for Storm](Defining-a-non-jvm-language-dsl-for-storm.html)
 * [Multilang protocol](Multilang-protocol.html) (how to provide support for another language)
 * [Implementation docs](Implementation-docs.html)
+* [Storm Metricstore](storm-metricstore.html)
 

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/docs/storm-metricstore.md
----------------------------------------------------------------------
diff --git a/docs/storm-metricstore.md b/docs/storm-metricstore.md
new file mode 100644
index 0000000..4111322
--- /dev/null
+++ b/docs/storm-metricstore.md
@@ -0,0 +1,83 @@
+---
+title: Storm Metricstore
+layout: documentation
+documentation: true
+---
+A metric store ([`MetricStore`]({{page.git-blob-base}}/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java)) interface was added 
+to Nimbus to allow storing metric information ([`Metric`]({{page.git-blob-base}}/storm-server/src/main/java/org/apache/storm/metricstore/Metric.java)) 
+to a database.  The default implementation 
+([`RocksDbStore`]({{page.git-blob-base}}/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java)) is using RocksDB, 
+a key-value store.
+
+As metrics are stored in RocksDB, their string values (for topology ID and executor ID, etc.) are converted to unique integer IDs, and these strings 
+are also stored to the database as metadata indexed by the integer ID. When a metric is stored, it is also aggregated with any existing metric 
+within the same 1, 10, and 60 minute timeframe.  
+
+The [`FilterOptions`]({{page.git-blob-base}}/storm-server/src/main/java/org/apache/storm/metricstore/FilterOptions.java) class provides an interface
+to select which options can be used to scan the metrics.
+
+
+### Configuration
+
+The following configuation options exist:
+
+```yaml
+storm.metricstore.class: "org.apache.storm.metricstore.rocksdb.RocksDbStore"
+storm.metricstore.rocksdb.location: "storm_rocks"
+storm.metricstore.rocksdb.create_if_missing: true
+storm.metricstore.rocksdb.metadata_string_cache_capacity: 4000
+storm.metricstore.rocksdb.retention_hours: 240
+```
+
+* storm.metricstore.class is the class that implements the 
+([`MetricStore`]({{page.git-blob-base}}/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java)).
+* storm.metricstore.rocksdb.location provides to location of the RocksDB database on Nimbus
+* storm.metricstore.rocksdb.create_if_missing permits creating a RocksDB database if missing
+* storm.metricstore.rocksdb.metadata_string_cache_capacity controls the number of metadata strings cached in memory.
+* storm.metricstore.rocksdb.retention_hours sets the length of time metrics will remain active.
+
+
+### RocksDB Schema
+
+The RocksDB Key (represented by [`RocksDbKey`]({{page.git-blob-base}}/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbKey.java))
+fields are as follows:
+
+
+| Field             | Size | Offset | Description                                                                                                  |
+|-------------------|------|--------|--------------------------------------------------------------------------------------------------------------|
+| Type              | 1    | 0      | The type maps to the KeyType enum, specifying a metric or various types of metadata strings                  |
+| Aggregation Level | 1    | 1      | The aggregation level for a metric (see AggLevel enum). Set to 0 for metadata.                               |
+| Topology Id       | 4    | 2      | The metadata string Id representing a topologyId for a metric, or the unique string Id for a metadata string |
+| Timestamp         | 8    | 6      | The timestamp for a metric, unused for metadata                                                              |
+| Metric Id         | 4    | 14     | The metadata string Id for the metric name                                                                   |
+| Component Id      | 4    | 18     | The metadata string Id for the component Id                                                                  |
+| Executor Id       | 4    | 22     | The metadata string Id for the executor Id                                                                   |
+| Host Id           | 4    | 26     | The metadata string Id for the host Id                                                                       |
+| Port              | 4    | 30     | The port number                                                                                              |
+| Stream Id         | 4    | 34     | The metadata string Id for the stream Id                                                                     |
+
+
+The RocksDB Value fields for metadata strings (represented by 
+[`RocksDbValue`]({{page.git-blob-base}}/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbValue.java)) are as follows:
+
+
+| Field           | Size | Offset | Description                                                                            |
+|-----------------|------|--------|----------------------------------------------------------------------------------------|
+| Version         | 1    | 0      | The current metadata version - allows migrating if the format changes in the future    |
+| Timestamp       | 8    | 1      | The time when the metadata was last used by a metric. Allows deleting of old metadata. |
+| Metadata String | any  | 9      | The metadata string                                                                    |
+
+
+RocksDB Value fields for metric data are as follows:
+
+| Field   | Size | Offset | Description                                                                       |
+|---------|------|--------|-----------------------------------------------------------------------------------|
+| Version | 1    | 0      | The current metric version - allows migrating if the format changes in the future |
+| Value   | 8    | 1      | The metric value                                                                  |
+| Count   | 8    | 9      | The metric count                                                                  |
+| Min     | 8    | 17     | The minimum metric value                                                          |
+| Max     | 8    | 25     | The maximum metric value                                                          |
+| Sum     | 8    | 33     | The sum of the metric values                                                      |
+
+
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3c924e3..ccace42 100644
--- a/pom.xml
+++ b/pom.xml
@@ -324,6 +324,7 @@
         <jool.version>0.9.12</jool.version>
         <caffeine.version>2.3.5</caffeine.version>
         <jaxb-version>2.3.0</jaxb-version>
+        <rocksdb-version>5.8.6</rocksdb-version>
 
         <!-- see intellij profile below... This fixes an annoyance with intellij -->
         <provided.scope>provided</provided.scope>
@@ -1048,6 +1049,11 @@
                 <artifactId>jaxb-api</artifactId>
                 <version>${jaxb-version}</version>
             </dependency>
+            <dependency>
+                <groupId>org.rocksdb</groupId>
+                <artifactId>rocksdbjni</artifactId>
+                <version>${rocksdb-version}</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 
@@ -1091,6 +1097,9 @@
                         <trimStackTrace>false</trimStackTrace>
                         <forkCount>1.0C</forkCount>
                         <reuseForks>true</reuseForks>
+                        <systemPropertyVariables>
+                            <storm.home>${project.basedir}/target/testhome</storm.home>
+                        </systemPropertyVariables>
                     </configuration>
                 </plugin>
                 <plugin>

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/Assignment.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/Assignment.java b/storm-client/src/jvm/org/apache/storm/generated/Assignment.java
index c3436d5..e8741df 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/Assignment.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/Assignment.java
@@ -966,15 +966,15 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 2: // NODE_HOST
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map666 = iprot.readMapBegin();
-                struct.node_host = new HashMap<String,String>(2*_map666.size);
-                String _key667;
-                String _val668;
-                for (int _i669 = 0; _i669 < _map666.size; ++_i669)
+                org.apache.thrift.protocol.TMap _map686 = iprot.readMapBegin();
+                struct.node_host = new HashMap<String,String>(2*_map686.size);
+                String _key687;
+                String _val688;
+                for (int _i689 = 0; _i689 < _map686.size; ++_i689)
                 {
-                  _key667 = iprot.readString();
-                  _val668 = iprot.readString();
-                  struct.node_host.put(_key667, _val668);
+                  _key687 = iprot.readString();
+                  _val688 = iprot.readString();
+                  struct.node_host.put(_key687, _val688);
                 }
                 iprot.readMapEnd();
               }
@@ -986,26 +986,26 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 3: // EXECUTOR_NODE_PORT
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map670 = iprot.readMapBegin();
-                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map670.size);
-                List<Long> _key671;
-                NodeInfo _val672;
-                for (int _i673 = 0; _i673 < _map670.size; ++_i673)
+                org.apache.thrift.protocol.TMap _map690 = iprot.readMapBegin();
+                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map690.size);
+                List<Long> _key691;
+                NodeInfo _val692;
+                for (int _i693 = 0; _i693 < _map690.size; ++_i693)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
-                    _key671 = new ArrayList<Long>(_list674.size);
-                    long _elem675;
-                    for (int _i676 = 0; _i676 < _list674.size; ++_i676)
+                    org.apache.thrift.protocol.TList _list694 = iprot.readListBegin();
+                    _key691 = new ArrayList<Long>(_list694.size);
+                    long _elem695;
+                    for (int _i696 = 0; _i696 < _list694.size; ++_i696)
                     {
-                      _elem675 = iprot.readI64();
-                      _key671.add(_elem675);
+                      _elem695 = iprot.readI64();
+                      _key691.add(_elem695);
                     }
                     iprot.readListEnd();
                   }
-                  _val672 = new NodeInfo();
-                  _val672.read(iprot);
-                  struct.executor_node_port.put(_key671, _val672);
+                  _val692 = new NodeInfo();
+                  _val692.read(iprot);
+                  struct.executor_node_port.put(_key691, _val692);
                 }
                 iprot.readMapEnd();
               }
@@ -1017,25 +1017,25 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 4: // EXECUTOR_START_TIME_SECS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map677 = iprot.readMapBegin();
-                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map677.size);
-                List<Long> _key678;
-                long _val679;
-                for (int _i680 = 0; _i680 < _map677.size; ++_i680)
+                org.apache.thrift.protocol.TMap _map697 = iprot.readMapBegin();
+                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map697.size);
+                List<Long> _key698;
+                long _val699;
+                for (int _i700 = 0; _i700 < _map697.size; ++_i700)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list681 = iprot.readListBegin();
-                    _key678 = new ArrayList<Long>(_list681.size);
-                    long _elem682;
-                    for (int _i683 = 0; _i683 < _list681.size; ++_i683)
+                    org.apache.thrift.protocol.TList _list701 = iprot.readListBegin();
+                    _key698 = new ArrayList<Long>(_list701.size);
+                    long _elem702;
+                    for (int _i703 = 0; _i703 < _list701.size; ++_i703)
                     {
-                      _elem682 = iprot.readI64();
-                      _key678.add(_elem682);
+                      _elem702 = iprot.readI64();
+                      _key698.add(_elem702);
                     }
                     iprot.readListEnd();
                   }
-                  _val679 = iprot.readI64();
-                  struct.executor_start_time_secs.put(_key678, _val679);
+                  _val699 = iprot.readI64();
+                  struct.executor_start_time_secs.put(_key698, _val699);
                 }
                 iprot.readMapEnd();
               }
@@ -1047,17 +1047,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 5: // WORKER_RESOURCES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map684 = iprot.readMapBegin();
-                struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map684.size);
-                NodeInfo _key685;
-                WorkerResources _val686;
-                for (int _i687 = 0; _i687 < _map684.size; ++_i687)
+                org.apache.thrift.protocol.TMap _map704 = iprot.readMapBegin();
+                struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map704.size);
+                NodeInfo _key705;
+                WorkerResources _val706;
+                for (int _i707 = 0; _i707 < _map704.size; ++_i707)
                 {
-                  _key685 = new NodeInfo();
-                  _key685.read(iprot);
-                  _val686 = new WorkerResources();
-                  _val686.read(iprot);
-                  struct.worker_resources.put(_key685, _val686);
+                  _key705 = new NodeInfo();
+                  _key705.read(iprot);
+                  _val706 = new WorkerResources();
+                  _val706.read(iprot);
+                  struct.worker_resources.put(_key705, _val706);
                 }
                 iprot.readMapEnd();
               }
@@ -1069,15 +1069,15 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 6: // TOTAL_SHARED_OFF_HEAP
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map688 = iprot.readMapBegin();
-                struct.total_shared_off_heap = new HashMap<String,Double>(2*_map688.size);
-                String _key689;
-                double _val690;
-                for (int _i691 = 0; _i691 < _map688.size; ++_i691)
+                org.apache.thrift.protocol.TMap _map708 = iprot.readMapBegin();
+                struct.total_shared_off_heap = new HashMap<String,Double>(2*_map708.size);
+                String _key709;
+                double _val710;
+                for (int _i711 = 0; _i711 < _map708.size; ++_i711)
                 {
-                  _key689 = iprot.readString();
-                  _val690 = iprot.readDouble();
-                  struct.total_shared_off_heap.put(_key689, _val690);
+                  _key709 = iprot.readString();
+                  _val710 = iprot.readDouble();
+                  struct.total_shared_off_heap.put(_key709, _val710);
                 }
                 iprot.readMapEnd();
               }
@@ -1117,10 +1117,10 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(NODE_HOST_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.node_host.size()));
-            for (Map.Entry<String, String> _iter692 : struct.node_host.entrySet())
+            for (Map.Entry<String, String> _iter712 : struct.node_host.entrySet())
             {
-              oprot.writeString(_iter692.getKey());
-              oprot.writeString(_iter692.getValue());
+              oprot.writeString(_iter712.getKey());
+              oprot.writeString(_iter712.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1132,17 +1132,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(EXECUTOR_NODE_PORT_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, struct.executor_node_port.size()));
-            for (Map.Entry<List<Long>, NodeInfo> _iter693 : struct.executor_node_port.entrySet())
+            for (Map.Entry<List<Long>, NodeInfo> _iter713 : struct.executor_node_port.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter693.getKey().size()));
-                for (long _iter694 : _iter693.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter713.getKey().size()));
+                for (long _iter714 : _iter713.getKey())
                 {
-                  oprot.writeI64(_iter694);
+                  oprot.writeI64(_iter714);
                 }
                 oprot.writeListEnd();
               }
-              _iter693.getValue().write(oprot);
+              _iter713.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1154,17 +1154,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(EXECUTOR_START_TIME_SECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, struct.executor_start_time_secs.size()));
-            for (Map.Entry<List<Long>, Long> _iter695 : struct.executor_start_time_secs.entrySet())
+            for (Map.Entry<List<Long>, Long> _iter715 : struct.executor_start_time_secs.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter695.getKey().size()));
-                for (long _iter696 : _iter695.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter715.getKey().size()));
+                for (long _iter716 : _iter715.getKey())
                 {
-                  oprot.writeI64(_iter696);
+                  oprot.writeI64(_iter716);
                 }
                 oprot.writeListEnd();
               }
-              oprot.writeI64(_iter695.getValue());
+              oprot.writeI64(_iter715.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1176,10 +1176,10 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(WORKER_RESOURCES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.worker_resources.size()));
-            for (Map.Entry<NodeInfo, WorkerResources> _iter697 : struct.worker_resources.entrySet())
+            for (Map.Entry<NodeInfo, WorkerResources> _iter717 : struct.worker_resources.entrySet())
             {
-              _iter697.getKey().write(oprot);
-              _iter697.getValue().write(oprot);
+              _iter717.getKey().write(oprot);
+              _iter717.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1191,10 +1191,10 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(TOTAL_SHARED_OFF_HEAP_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.total_shared_off_heap.size()));
-            for (Map.Entry<String, Double> _iter698 : struct.total_shared_off_heap.entrySet())
+            for (Map.Entry<String, Double> _iter718 : struct.total_shared_off_heap.entrySet())
             {
-              oprot.writeString(_iter698.getKey());
-              oprot.writeDouble(_iter698.getValue());
+              oprot.writeString(_iter718.getKey());
+              oprot.writeDouble(_iter718.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1249,62 +1249,62 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       if (struct.is_set_node_host()) {
         {
           oprot.writeI32(struct.node_host.size());
-          for (Map.Entry<String, String> _iter699 : struct.node_host.entrySet())
+          for (Map.Entry<String, String> _iter719 : struct.node_host.entrySet())
           {
-            oprot.writeString(_iter699.getKey());
-            oprot.writeString(_iter699.getValue());
+            oprot.writeString(_iter719.getKey());
+            oprot.writeString(_iter719.getValue());
           }
         }
       }
       if (struct.is_set_executor_node_port()) {
         {
           oprot.writeI32(struct.executor_node_port.size());
-          for (Map.Entry<List<Long>, NodeInfo> _iter700 : struct.executor_node_port.entrySet())
+          for (Map.Entry<List<Long>, NodeInfo> _iter720 : struct.executor_node_port.entrySet())
           {
             {
-              oprot.writeI32(_iter700.getKey().size());
-              for (long _iter701 : _iter700.getKey())
+              oprot.writeI32(_iter720.getKey().size());
+              for (long _iter721 : _iter720.getKey())
               {
-                oprot.writeI64(_iter701);
+                oprot.writeI64(_iter721);
               }
             }
-            _iter700.getValue().write(oprot);
+            _iter720.getValue().write(oprot);
           }
         }
       }
       if (struct.is_set_executor_start_time_secs()) {
         {
           oprot.writeI32(struct.executor_start_time_secs.size());
-          for (Map.Entry<List<Long>, Long> _iter702 : struct.executor_start_time_secs.entrySet())
+          for (Map.Entry<List<Long>, Long> _iter722 : struct.executor_start_time_secs.entrySet())
           {
             {
-              oprot.writeI32(_iter702.getKey().size());
-              for (long _iter703 : _iter702.getKey())
+              oprot.writeI32(_iter722.getKey().size());
+              for (long _iter723 : _iter722.getKey())
               {
-                oprot.writeI64(_iter703);
+                oprot.writeI64(_iter723);
               }
             }
-            oprot.writeI64(_iter702.getValue());
+            oprot.writeI64(_iter722.getValue());
           }
         }
       }
       if (struct.is_set_worker_resources()) {
         {
           oprot.writeI32(struct.worker_resources.size());
-          for (Map.Entry<NodeInfo, WorkerResources> _iter704 : struct.worker_resources.entrySet())
+          for (Map.Entry<NodeInfo, WorkerResources> _iter724 : struct.worker_resources.entrySet())
           {
-            _iter704.getKey().write(oprot);
-            _iter704.getValue().write(oprot);
+            _iter724.getKey().write(oprot);
+            _iter724.getValue().write(oprot);
           }
         }
       }
       if (struct.is_set_total_shared_off_heap()) {
         {
           oprot.writeI32(struct.total_shared_off_heap.size());
-          for (Map.Entry<String, Double> _iter705 : struct.total_shared_off_heap.entrySet())
+          for (Map.Entry<String, Double> _iter725 : struct.total_shared_off_heap.entrySet())
           {
-            oprot.writeString(_iter705.getKey());
-            oprot.writeDouble(_iter705.getValue());
+            oprot.writeString(_iter725.getKey());
+            oprot.writeDouble(_iter725.getValue());
           }
         }
       }
@@ -1321,96 +1321,96 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map706 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.node_host = new HashMap<String,String>(2*_map706.size);
-          String _key707;
-          String _val708;
-          for (int _i709 = 0; _i709 < _map706.size; ++_i709)
+          org.apache.thrift.protocol.TMap _map726 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.node_host = new HashMap<String,String>(2*_map726.size);
+          String _key727;
+          String _val728;
+          for (int _i729 = 0; _i729 < _map726.size; ++_i729)
           {
-            _key707 = iprot.readString();
-            _val708 = iprot.readString();
-            struct.node_host.put(_key707, _val708);
+            _key727 = iprot.readString();
+            _val728 = iprot.readString();
+            struct.node_host.put(_key727, _val728);
           }
         }
         struct.set_node_host_isSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map710 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map710.size);
-          List<Long> _key711;
-          NodeInfo _val712;
-          for (int _i713 = 0; _i713 < _map710.size; ++_i713)
+          org.apache.thrift.protocol.TMap _map730 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map730.size);
+          List<Long> _key731;
+          NodeInfo _val732;
+          for (int _i733 = 0; _i733 < _map730.size; ++_i733)
           {
             {
-              org.apache.thrift.protocol.TList _list714 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key711 = new ArrayList<Long>(_list714.size);
-              long _elem715;
-              for (int _i716 = 0; _i716 < _list714.size; ++_i716)
+              org.apache.thrift.protocol.TList _list734 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key731 = new ArrayList<Long>(_list734.size);
+              long _elem735;
+              for (int _i736 = 0; _i736 < _list734.size; ++_i736)
               {
-                _elem715 = iprot.readI64();
-                _key711.add(_elem715);
+                _elem735 = iprot.readI64();
+                _key731.add(_elem735);
               }
             }
-            _val712 = new NodeInfo();
-            _val712.read(iprot);
-            struct.executor_node_port.put(_key711, _val712);
+            _val732 = new NodeInfo();
+            _val732.read(iprot);
+            struct.executor_node_port.put(_key731, _val732);
           }
         }
         struct.set_executor_node_port_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map717 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map717.size);
-          List<Long> _key718;
-          long _val719;
-          for (int _i720 = 0; _i720 < _map717.size; ++_i720)
+          org.apache.thrift.protocol.TMap _map737 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map737.size);
+          List<Long> _key738;
+          long _val739;
+          for (int _i740 = 0; _i740 < _map737.size; ++_i740)
           {
             {
-              org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key718 = new ArrayList<Long>(_list721.size);
-              long _elem722;
-              for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+              org.apache.thrift.protocol.TList _list741 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key738 = new ArrayList<Long>(_list741.size);
+              long _elem742;
+              for (int _i743 = 0; _i743 < _list741.size; ++_i743)
               {
-                _elem722 = iprot.readI64();
-                _key718.add(_elem722);
+                _elem742 = iprot.readI64();
+                _key738.add(_elem742);
               }
             }
-            _val719 = iprot.readI64();
-            struct.executor_start_time_secs.put(_key718, _val719);
+            _val739 = iprot.readI64();
+            struct.executor_start_time_secs.put(_key738, _val739);
           }
         }
         struct.set_executor_start_time_secs_isSet(true);
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map724 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map724.size);
-          NodeInfo _key725;
-          WorkerResources _val726;
-          for (int _i727 = 0; _i727 < _map724.size; ++_i727)
+          org.apache.thrift.protocol.TMap _map744 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map744.size);
+          NodeInfo _key745;
+          WorkerResources _val746;
+          for (int _i747 = 0; _i747 < _map744.size; ++_i747)
           {
-            _key725 = new NodeInfo();
-            _key725.read(iprot);
-            _val726 = new WorkerResources();
-            _val726.read(iprot);
-            struct.worker_resources.put(_key725, _val726);
+            _key745 = new NodeInfo();
+            _key745.read(iprot);
+            _val746 = new WorkerResources();
+            _val746.read(iprot);
+            struct.worker_resources.put(_key745, _val746);
           }
         }
         struct.set_worker_resources_isSet(true);
       }
       if (incoming.get(4)) {
         {
-          org.apache.thrift.protocol.TMap _map728 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
-          struct.total_shared_off_heap = new HashMap<String,Double>(2*_map728.size);
-          String _key729;
-          double _val730;
-          for (int _i731 = 0; _i731 < _map728.size; ++_i731)
+          org.apache.thrift.protocol.TMap _map748 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.total_shared_off_heap = new HashMap<String,Double>(2*_map748.size);
+          String _key749;
+          double _val750;
+          for (int _i751 = 0; _i751 < _map748.size; ++_i751)
           {
-            _key729 = iprot.readString();
-            _val730 = iprot.readDouble();
-            struct.total_shared_off_heap.put(_key729, _val730);
+            _key749 = iprot.readString();
+            _val750 = iprot.readDouble();
+            struct.total_shared_off_heap.put(_key749, _val750);
           }
         }
         struct.set_total_shared_off_heap_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/ClusterWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/ClusterWorkerHeartbeat.java b/storm-client/src/jvm/org/apache/storm/generated/ClusterWorkerHeartbeat.java
index 1613778..a2651cd 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/ClusterWorkerHeartbeat.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/ClusterWorkerHeartbeat.java
@@ -635,17 +635,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
           case 2: // EXECUTOR_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map752 = iprot.readMapBegin();
-                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map752.size);
-                ExecutorInfo _key753;
-                ExecutorStats _val754;
-                for (int _i755 = 0; _i755 < _map752.size; ++_i755)
+                org.apache.thrift.protocol.TMap _map772 = iprot.readMapBegin();
+                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map772.size);
+                ExecutorInfo _key773;
+                ExecutorStats _val774;
+                for (int _i775 = 0; _i775 < _map772.size; ++_i775)
                 {
-                  _key753 = new ExecutorInfo();
-                  _key753.read(iprot);
-                  _val754 = new ExecutorStats();
-                  _val754.read(iprot);
-                  struct.executor_stats.put(_key753, _val754);
+                  _key773 = new ExecutorInfo();
+                  _key773.read(iprot);
+                  _val774 = new ExecutorStats();
+                  _val774.read(iprot);
+                  struct.executor_stats.put(_key773, _val774);
                 }
                 iprot.readMapEnd();
               }
@@ -692,10 +692,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
         oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.executor_stats.size()));
-          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter756 : struct.executor_stats.entrySet())
+          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter776 : struct.executor_stats.entrySet())
           {
-            _iter756.getKey().write(oprot);
-            _iter756.getValue().write(oprot);
+            _iter776.getKey().write(oprot);
+            _iter776.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -727,10 +727,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       oprot.writeString(struct.storm_id);
       {
         oprot.writeI32(struct.executor_stats.size());
-        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter757 : struct.executor_stats.entrySet())
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter777 : struct.executor_stats.entrySet())
         {
-          _iter757.getKey().write(oprot);
-          _iter757.getValue().write(oprot);
+          _iter777.getKey().write(oprot);
+          _iter777.getValue().write(oprot);
         }
       }
       oprot.writeI32(struct.time_secs);
@@ -743,17 +743,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       struct.storm_id = iprot.readString();
       struct.set_storm_id_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map758 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map758.size);
-        ExecutorInfo _key759;
-        ExecutorStats _val760;
-        for (int _i761 = 0; _i761 < _map758.size; ++_i761)
+        org.apache.thrift.protocol.TMap _map778 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map778.size);
+        ExecutorInfo _key779;
+        ExecutorStats _val780;
+        for (int _i781 = 0; _i781 < _map778.size; ++_i781)
         {
-          _key759 = new ExecutorInfo();
-          _key759.read(iprot);
-          _val760 = new ExecutorStats();
-          _val760.read(iprot);
-          struct.executor_stats.put(_key759, _val760);
+          _key779 = new ExecutorInfo();
+          _key779.read(iprot);
+          _val780 = new ExecutorStats();
+          _val780.read(iprot);
+          struct.executor_stats.put(_key779, _val780);
         }
       }
       struct.set_executor_stats_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/HBNodes.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBNodes.java b/storm-client/src/jvm/org/apache/storm/generated/HBNodes.java
index 75c5c6d..887dbc0 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/HBNodes.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBNodes.java
@@ -364,13 +364,13 @@ public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields
           case 1: // PULSE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list858 = iprot.readListBegin();
-                struct.pulseIds = new ArrayList<String>(_list858.size);
-                String _elem859;
-                for (int _i860 = 0; _i860 < _list858.size; ++_i860)
+                org.apache.thrift.protocol.TList _list886 = iprot.readListBegin();
+                struct.pulseIds = new ArrayList<String>(_list886.size);
+                String _elem887;
+                for (int _i888 = 0; _i888 < _list886.size; ++_i888)
                 {
-                  _elem859 = iprot.readString();
-                  struct.pulseIds.add(_elem859);
+                  _elem887 = iprot.readString();
+                  struct.pulseIds.add(_elem887);
                 }
                 iprot.readListEnd();
               }
@@ -396,9 +396,9 @@ public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields
         oprot.writeFieldBegin(PULSE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.pulseIds.size()));
-          for (String _iter861 : struct.pulseIds)
+          for (String _iter889 : struct.pulseIds)
           {
-            oprot.writeString(_iter861);
+            oprot.writeString(_iter889);
           }
           oprot.writeListEnd();
         }
@@ -429,9 +429,9 @@ public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields
       if (struct.is_set_pulseIds()) {
         {
           oprot.writeI32(struct.pulseIds.size());
-          for (String _iter862 : struct.pulseIds)
+          for (String _iter890 : struct.pulseIds)
           {
-            oprot.writeString(_iter862);
+            oprot.writeString(_iter890);
           }
         }
       }
@@ -443,13 +443,13 @@ public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list863 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.pulseIds = new ArrayList<String>(_list863.size);
-          String _elem864;
-          for (int _i865 = 0; _i865 < _list863.size; ++_i865)
+          org.apache.thrift.protocol.TList _list891 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.pulseIds = new ArrayList<String>(_list891.size);
+          String _elem892;
+          for (int _i893 = 0; _i893 < _list891.size; ++_i893)
           {
-            _elem864 = iprot.readString();
-            struct.pulseIds.add(_elem864);
+            _elem892 = iprot.readString();
+            struct.pulseIds.add(_elem892);
           }
         }
         struct.set_pulseIds_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/HBRecords.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/HBRecords.java b/storm-client/src/jvm/org/apache/storm/generated/HBRecords.java
index f726e5c..cfed785 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/HBRecords.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/HBRecords.java
@@ -367,14 +367,14 @@ public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._
           case 1: // PULSES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list850 = iprot.readListBegin();
-                struct.pulses = new ArrayList<HBPulse>(_list850.size);
-                HBPulse _elem851;
-                for (int _i852 = 0; _i852 < _list850.size; ++_i852)
+                org.apache.thrift.protocol.TList _list878 = iprot.readListBegin();
+                struct.pulses = new ArrayList<HBPulse>(_list878.size);
+                HBPulse _elem879;
+                for (int _i880 = 0; _i880 < _list878.size; ++_i880)
                 {
-                  _elem851 = new HBPulse();
-                  _elem851.read(iprot);
-                  struct.pulses.add(_elem851);
+                  _elem879 = new HBPulse();
+                  _elem879.read(iprot);
+                  struct.pulses.add(_elem879);
                 }
                 iprot.readListEnd();
               }
@@ -400,9 +400,9 @@ public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._
         oprot.writeFieldBegin(PULSES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pulses.size()));
-          for (HBPulse _iter853 : struct.pulses)
+          for (HBPulse _iter881 : struct.pulses)
           {
-            _iter853.write(oprot);
+            _iter881.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -433,9 +433,9 @@ public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._
       if (struct.is_set_pulses()) {
         {
           oprot.writeI32(struct.pulses.size());
-          for (HBPulse _iter854 : struct.pulses)
+          for (HBPulse _iter882 : struct.pulses)
           {
-            _iter854.write(oprot);
+            _iter882.write(oprot);
           }
         }
       }
@@ -447,14 +447,14 @@ public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list855 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.pulses = new ArrayList<HBPulse>(_list855.size);
-          HBPulse _elem856;
-          for (int _i857 = 0; _i857 < _list855.size; ++_i857)
+          org.apache.thrift.protocol.TList _list883 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.pulses = new ArrayList<HBPulse>(_list883.size);
+          HBPulse _elem884;
+          for (int _i885 = 0; _i885 < _list883.size; ++_i885)
           {
-            _elem856 = new HBPulse();
-            _elem856.read(iprot);
-            struct.pulses.add(_elem856);
+            _elem884 = new HBPulse();
+            _elem884.read(iprot);
+            struct.pulses.add(_elem884);
           }
         }
         struct.set_pulses_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/LSApprovedWorkers.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSApprovedWorkers.java b/storm-client/src/jvm/org/apache/storm/generated/LSApprovedWorkers.java
index 9e8e5cf..9b13724 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/LSApprovedWorkers.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSApprovedWorkers.java
@@ -365,15 +365,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
           case 1: // APPROVED_WORKERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map780 = iprot.readMapBegin();
-                struct.approved_workers = new HashMap<String,Integer>(2*_map780.size);
-                String _key781;
-                int _val782;
-                for (int _i783 = 0; _i783 < _map780.size; ++_i783)
+                org.apache.thrift.protocol.TMap _map800 = iprot.readMapBegin();
+                struct.approved_workers = new HashMap<String,Integer>(2*_map800.size);
+                String _key801;
+                int _val802;
+                for (int _i803 = 0; _i803 < _map800.size; ++_i803)
                 {
-                  _key781 = iprot.readString();
-                  _val782 = iprot.readI32();
-                  struct.approved_workers.put(_key781, _val782);
+                  _key801 = iprot.readString();
+                  _val802 = iprot.readI32();
+                  struct.approved_workers.put(_key801, _val802);
                 }
                 iprot.readMapEnd();
               }
@@ -399,10 +399,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
         oprot.writeFieldBegin(APPROVED_WORKERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.approved_workers.size()));
-          for (Map.Entry<String, Integer> _iter784 : struct.approved_workers.entrySet())
+          for (Map.Entry<String, Integer> _iter804 : struct.approved_workers.entrySet())
           {
-            oprot.writeString(_iter784.getKey());
-            oprot.writeI32(_iter784.getValue());
+            oprot.writeString(_iter804.getKey());
+            oprot.writeI32(_iter804.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -427,10 +427,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.approved_workers.size());
-        for (Map.Entry<String, Integer> _iter785 : struct.approved_workers.entrySet())
+        for (Map.Entry<String, Integer> _iter805 : struct.approved_workers.entrySet())
         {
-          oprot.writeString(_iter785.getKey());
-          oprot.writeI32(_iter785.getValue());
+          oprot.writeString(_iter805.getKey());
+          oprot.writeI32(_iter805.getValue());
         }
       }
     }
@@ -439,15 +439,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
     public void read(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map786 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.approved_workers = new HashMap<String,Integer>(2*_map786.size);
-        String _key787;
-        int _val788;
-        for (int _i789 = 0; _i789 < _map786.size; ++_i789)
+        org.apache.thrift.protocol.TMap _map806 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.approved_workers = new HashMap<String,Integer>(2*_map806.size);
+        String _key807;
+        int _val808;
+        for (int _i809 = 0; _i809 < _map806.size; ++_i809)
         {
-          _key787 = iprot.readString();
-          _val788 = iprot.readI32();
-          struct.approved_workers.put(_key787, _val788);
+          _key807 = iprot.readString();
+          _val808 = iprot.readI32();
+          struct.approved_workers.put(_key807, _val808);
         }
       }
       struct.set_approved_workers_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorAssignments.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorAssignments.java b/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorAssignments.java
index 64c36e2..95f78ca 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorAssignments.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSSupervisorAssignments.java
@@ -376,16 +376,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
           case 1: // ASSIGNMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map790 = iprot.readMapBegin();
-                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map790.size);
-                int _key791;
-                LocalAssignment _val792;
-                for (int _i793 = 0; _i793 < _map790.size; ++_i793)
+                org.apache.thrift.protocol.TMap _map810 = iprot.readMapBegin();
+                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map810.size);
+                int _key811;
+                LocalAssignment _val812;
+                for (int _i813 = 0; _i813 < _map810.size; ++_i813)
                 {
-                  _key791 = iprot.readI32();
-                  _val792 = new LocalAssignment();
-                  _val792.read(iprot);
-                  struct.assignments.put(_key791, _val792);
+                  _key811 = iprot.readI32();
+                  _val812 = new LocalAssignment();
+                  _val812.read(iprot);
+                  struct.assignments.put(_key811, _val812);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
         oprot.writeFieldBegin(ASSIGNMENTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, struct.assignments.size()));
-          for (Map.Entry<Integer, LocalAssignment> _iter794 : struct.assignments.entrySet())
+          for (Map.Entry<Integer, LocalAssignment> _iter814 : struct.assignments.entrySet())
           {
-            oprot.writeI32(_iter794.getKey());
-            _iter794.getValue().write(oprot);
+            oprot.writeI32(_iter814.getKey());
+            _iter814.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.assignments.size());
-        for (Map.Entry<Integer, LocalAssignment> _iter795 : struct.assignments.entrySet())
+        for (Map.Entry<Integer, LocalAssignment> _iter815 : struct.assignments.entrySet())
         {
-          oprot.writeI32(_iter795.getKey());
-          _iter795.getValue().write(oprot);
+          oprot.writeI32(_iter815.getKey());
+          _iter815.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
     public void read(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignments struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map796 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map796.size);
-        int _key797;
-        LocalAssignment _val798;
-        for (int _i799 = 0; _i799 < _map796.size; ++_i799)
+        org.apache.thrift.protocol.TMap _map816 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map816.size);
+        int _key817;
+        LocalAssignment _val818;
+        for (int _i819 = 0; _i819 < _map816.size; ++_i819)
         {
-          _key797 = iprot.readI32();
-          _val798 = new LocalAssignment();
-          _val798.read(iprot);
-          struct.assignments.put(_key797, _val798);
+          _key817 = iprot.readI32();
+          _val818 = new LocalAssignment();
+          _val818.read(iprot);
+          struct.assignments.put(_key817, _val818);
         }
       }
       struct.set_assignments_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistory.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistory.java b/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistory.java
index 6de53df..432b9b6 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistory.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistory.java
@@ -656,13 +656,13 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
           case 3: // USERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list808 = iprot.readListBegin();
-                struct.users = new ArrayList<String>(_list808.size);
-                String _elem809;
-                for (int _i810 = 0; _i810 < _list808.size; ++_i810)
+                org.apache.thrift.protocol.TList _list828 = iprot.readListBegin();
+                struct.users = new ArrayList<String>(_list828.size);
+                String _elem829;
+                for (int _i830 = 0; _i830 < _list828.size; ++_i830)
                 {
-                  _elem809 = iprot.readString();
-                  struct.users.add(_elem809);
+                  _elem829 = iprot.readString();
+                  struct.users.add(_elem829);
                 }
                 iprot.readListEnd();
               }
@@ -674,13 +674,13 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
           case 4: // GROUPS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list811 = iprot.readListBegin();
-                struct.groups = new ArrayList<String>(_list811.size);
-                String _elem812;
-                for (int _i813 = 0; _i813 < _list811.size; ++_i813)
+                org.apache.thrift.protocol.TList _list831 = iprot.readListBegin();
+                struct.groups = new ArrayList<String>(_list831.size);
+                String _elem832;
+                for (int _i833 = 0; _i833 < _list831.size; ++_i833)
                 {
-                  _elem812 = iprot.readString();
-                  struct.groups.add(_elem812);
+                  _elem832 = iprot.readString();
+                  struct.groups.add(_elem832);
                 }
                 iprot.readListEnd();
               }
@@ -714,9 +714,9 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
         oprot.writeFieldBegin(USERS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.users.size()));
-          for (String _iter814 : struct.users)
+          for (String _iter834 : struct.users)
           {
-            oprot.writeString(_iter814);
+            oprot.writeString(_iter834);
           }
           oprot.writeListEnd();
         }
@@ -726,9 +726,9 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
         oprot.writeFieldBegin(GROUPS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groups.size()));
-          for (String _iter815 : struct.groups)
+          for (String _iter835 : struct.groups)
           {
-            oprot.writeString(_iter815);
+            oprot.writeString(_iter835);
           }
           oprot.writeListEnd();
         }
@@ -755,16 +755,16 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
       oprot.writeI64(struct.time_stamp);
       {
         oprot.writeI32(struct.users.size());
-        for (String _iter816 : struct.users)
+        for (String _iter836 : struct.users)
         {
-          oprot.writeString(_iter816);
+          oprot.writeString(_iter836);
         }
       }
       {
         oprot.writeI32(struct.groups.size());
-        for (String _iter817 : struct.groups)
+        for (String _iter837 : struct.groups)
         {
-          oprot.writeString(_iter817);
+          oprot.writeString(_iter837);
         }
       }
     }
@@ -777,24 +777,24 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
       struct.time_stamp = iprot.readI64();
       struct.set_time_stamp_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list818 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.users = new ArrayList<String>(_list818.size);
-        String _elem819;
-        for (int _i820 = 0; _i820 < _list818.size; ++_i820)
+        org.apache.thrift.protocol.TList _list838 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.users = new ArrayList<String>(_list838.size);
+        String _elem839;
+        for (int _i840 = 0; _i840 < _list838.size; ++_i840)
         {
-          _elem819 = iprot.readString();
-          struct.users.add(_elem819);
+          _elem839 = iprot.readString();
+          struct.users.add(_elem839);
         }
       }
       struct.set_users_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list821 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.groups = new ArrayList<String>(_list821.size);
-        String _elem822;
-        for (int _i823 = 0; _i823 < _list821.size; ++_i823)
+        org.apache.thrift.protocol.TList _list841 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.groups = new ArrayList<String>(_list841.size);
+        String _elem842;
+        for (int _i843 = 0; _i843 < _list841.size; ++_i843)
         {
-          _elem822 = iprot.readString();
-          struct.groups.add(_elem822);
+          _elem842 = iprot.readString();
+          struct.groups.add(_elem842);
         }
       }
       struct.set_groups_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistoryList.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistoryList.java b/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistoryList.java
index 790a6fb..461bd08 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistoryList.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSTopoHistoryList.java
@@ -371,14 +371,14 @@ public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryL
           case 1: // TOPO_HISTORY
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list824 = iprot.readListBegin();
-                struct.topo_history = new ArrayList<LSTopoHistory>(_list824.size);
-                LSTopoHistory _elem825;
-                for (int _i826 = 0; _i826 < _list824.size; ++_i826)
+                org.apache.thrift.protocol.TList _list844 = iprot.readListBegin();
+                struct.topo_history = new ArrayList<LSTopoHistory>(_list844.size);
+                LSTopoHistory _elem845;
+                for (int _i846 = 0; _i846 < _list844.size; ++_i846)
                 {
-                  _elem825 = new LSTopoHistory();
-                  _elem825.read(iprot);
-                  struct.topo_history.add(_elem825);
+                  _elem845 = new LSTopoHistory();
+                  _elem845.read(iprot);
+                  struct.topo_history.add(_elem845);
                 }
                 iprot.readListEnd();
               }
@@ -404,9 +404,9 @@ public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryL
         oprot.writeFieldBegin(TOPO_HISTORY_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.topo_history.size()));
-          for (LSTopoHistory _iter827 : struct.topo_history)
+          for (LSTopoHistory _iter847 : struct.topo_history)
           {
-            _iter827.write(oprot);
+            _iter847.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -431,9 +431,9 @@ public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryL
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.topo_history.size());
-        for (LSTopoHistory _iter828 : struct.topo_history)
+        for (LSTopoHistory _iter848 : struct.topo_history)
         {
-          _iter828.write(oprot);
+          _iter848.write(oprot);
         }
       }
     }
@@ -442,14 +442,14 @@ public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryL
     public void read(org.apache.thrift.protocol.TProtocol prot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list829 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.topo_history = new ArrayList<LSTopoHistory>(_list829.size);
-        LSTopoHistory _elem830;
-        for (int _i831 = 0; _i831 < _list829.size; ++_i831)
+        org.apache.thrift.protocol.TList _list849 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.topo_history = new ArrayList<LSTopoHistory>(_list849.size);
+        LSTopoHistory _elem850;
+        for (int _i851 = 0; _i851 < _list849.size; ++_i851)
         {
-          _elem830 = new LSTopoHistory();
-          _elem830.read(iprot);
-          struct.topo_history.add(_elem830);
+          _elem850 = new LSTopoHistory();
+          _elem850.read(iprot);
+          struct.topo_history.add(_elem850);
         }
       }
       struct.set_topo_history_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/LSWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LSWorkerHeartbeat.java b/storm-client/src/jvm/org/apache/storm/generated/LSWorkerHeartbeat.java
index 6cf386f..73b9c12 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/LSWorkerHeartbeat.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/LSWorkerHeartbeat.java
@@ -638,14 +638,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
           case 3: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list800.size);
-                ExecutorInfo _elem801;
-                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
+                org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list820.size);
+                ExecutorInfo _elem821;
+                for (int _i822 = 0; _i822 < _list820.size; ++_i822)
                 {
-                  _elem801 = new ExecutorInfo();
-                  _elem801.read(iprot);
-                  struct.executors.add(_elem801);
+                  _elem821 = new ExecutorInfo();
+                  _elem821.read(iprot);
+                  struct.executors.add(_elem821);
                 }
                 iprot.readListEnd();
               }
@@ -687,9 +687,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter803 : struct.executors)
+          for (ExecutorInfo _iter823 : struct.executors)
           {
-            _iter803.write(oprot);
+            _iter823.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter804 : struct.executors)
+        for (ExecutorInfo _iter824 : struct.executors)
         {
-          _iter804.write(oprot);
+          _iter824.write(oprot);
         }
       }
       oprot.writeI32(struct.port);
@@ -735,14 +735,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list805.size);
-        ExecutorInfo _elem806;
-        for (int _i807 = 0; _i807 < _list805.size; ++_i807)
+        org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list825.size);
+        ExecutorInfo _elem826;
+        for (int _i827 = 0; _i827 < _list825.size; ++_i827)
         {
-          _elem806 = new ExecutorInfo();
-          _elem806.read(iprot);
-          struct.executors.add(_elem806);
+          _elem826 = new ExecutorInfo();
+          _elem826.read(iprot);
+          struct.executors.add(_elem826);
         }
       }
       struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/LocalAssignment.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LocalAssignment.java b/storm-client/src/jvm/org/apache/storm/generated/LocalAssignment.java
index e4d83aa..3814124 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/LocalAssignment.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/LocalAssignment.java
@@ -710,14 +710,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
           case 2: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list772.size);
-                ExecutorInfo _elem773;
-                for (int _i774 = 0; _i774 < _list772.size; ++_i774)
+                org.apache.thrift.protocol.TList _list792 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list792.size);
+                ExecutorInfo _elem793;
+                for (int _i794 = 0; _i794 < _list792.size; ++_i794)
                 {
-                  _elem773 = new ExecutorInfo();
-                  _elem773.read(iprot);
-                  struct.executors.add(_elem773);
+                  _elem793 = new ExecutorInfo();
+                  _elem793.read(iprot);
+                  struct.executors.add(_elem793);
                 }
                 iprot.readListEnd();
               }
@@ -773,9 +773,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter775 : struct.executors)
+          for (ExecutorInfo _iter795 : struct.executors)
           {
-            _iter775.write(oprot);
+            _iter795.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -820,9 +820,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter776 : struct.executors)
+        for (ExecutorInfo _iter796 : struct.executors)
         {
-          _iter776.write(oprot);
+          _iter796.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -853,14 +853,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list777.size);
-        ExecutorInfo _elem778;
-        for (int _i779 = 0; _i779 < _list777.size; ++_i779)
+        org.apache.thrift.protocol.TList _list797 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list797.size);
+        ExecutorInfo _elem798;
+        for (int _i799 = 0; _i799 < _list797.size; ++_i799)
         {
-          _elem778 = new ExecutorInfo();
-          _elem778.read(iprot);
-          struct.executors.add(_elem778);
+          _elem798 = new ExecutorInfo();
+          _elem798.read(iprot);
+          struct.executors.add(_elem798);
         }
       }
       struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/LocalStateData.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LocalStateData.java b/storm-client/src/jvm/org/apache/storm/generated/LocalStateData.java
index 3536c0b..a99475f 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/LocalStateData.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/LocalStateData.java
@@ -376,16 +376,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
           case 1: // SERIALIZED_PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map762 = iprot.readMapBegin();
-                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map762.size);
-                String _key763;
-                ThriftSerializedObject _val764;
-                for (int _i765 = 0; _i765 < _map762.size; ++_i765)
+                org.apache.thrift.protocol.TMap _map782 = iprot.readMapBegin();
+                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map782.size);
+                String _key783;
+                ThriftSerializedObject _val784;
+                for (int _i785 = 0; _i785 < _map782.size; ++_i785)
                 {
-                  _key763 = iprot.readString();
-                  _val764 = new ThriftSerializedObject();
-                  _val764.read(iprot);
-                  struct.serialized_parts.put(_key763, _val764);
+                  _key783 = iprot.readString();
+                  _val784 = new ThriftSerializedObject();
+                  _val784.read(iprot);
+                  struct.serialized_parts.put(_key783, _val784);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
         oprot.writeFieldBegin(SERIALIZED_PARTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.serialized_parts.size()));
-          for (Map.Entry<String, ThriftSerializedObject> _iter766 : struct.serialized_parts.entrySet())
+          for (Map.Entry<String, ThriftSerializedObject> _iter786 : struct.serialized_parts.entrySet())
           {
-            oprot.writeString(_iter766.getKey());
-            _iter766.getValue().write(oprot);
+            oprot.writeString(_iter786.getKey());
+            _iter786.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.serialized_parts.size());
-        for (Map.Entry<String, ThriftSerializedObject> _iter767 : struct.serialized_parts.entrySet())
+        for (Map.Entry<String, ThriftSerializedObject> _iter787 : struct.serialized_parts.entrySet())
         {
-          oprot.writeString(_iter767.getKey());
-          _iter767.getValue().write(oprot);
+          oprot.writeString(_iter787.getKey());
+          _iter787.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
     public void read(org.apache.thrift.protocol.TProtocol prot, LocalStateData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map768 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map768.size);
-        String _key769;
-        ThriftSerializedObject _val770;
-        for (int _i771 = 0; _i771 < _map768.size; ++_i771)
+        org.apache.thrift.protocol.TMap _map788 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map788.size);
+        String _key789;
+        ThriftSerializedObject _val790;
+        for (int _i791 = 0; _i791 < _map788.size; ++_i791)
         {
-          _key769 = iprot.readString();
-          _val770 = new ThriftSerializedObject();
-          _val770.read(iprot);
-          struct.serialized_parts.put(_key769, _val770);
+          _key789 = iprot.readString();
+          _val790 = new ThriftSerializedObject();
+          _val790.read(iprot);
+          struct.serialized_parts.put(_key789, _val790);
         }
       }
       struct.set_serialized_parts_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/LogConfig.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/LogConfig.java b/storm-client/src/jvm/org/apache/storm/generated/LogConfig.java
index e783e6c..f44203f 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/LogConfig.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/LogConfig.java
@@ -368,16 +368,16 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
           case 2: // NAMED_LOGGER_LEVEL
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map832 = iprot.readMapBegin();
-                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map832.size);
-                String _key833;
-                LogLevel _val834;
-                for (int _i835 = 0; _i835 < _map832.size; ++_i835)
+                org.apache.thrift.protocol.TMap _map852 = iprot.readMapBegin();
+                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map852.size);
+                String _key853;
+                LogLevel _val854;
+                for (int _i855 = 0; _i855 < _map852.size; ++_i855)
                 {
-                  _key833 = iprot.readString();
-                  _val834 = new LogLevel();
-                  _val834.read(iprot);
-                  struct.named_logger_level.put(_key833, _val834);
+                  _key853 = iprot.readString();
+                  _val854 = new LogLevel();
+                  _val854.read(iprot);
+                  struct.named_logger_level.put(_key853, _val854);
                 }
                 iprot.readMapEnd();
               }
@@ -404,10 +404,10 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
           oprot.writeFieldBegin(NAMED_LOGGER_LEVEL_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.named_logger_level.size()));
-            for (Map.Entry<String, LogLevel> _iter836 : struct.named_logger_level.entrySet())
+            for (Map.Entry<String, LogLevel> _iter856 : struct.named_logger_level.entrySet())
             {
-              oprot.writeString(_iter836.getKey());
-              _iter836.getValue().write(oprot);
+              oprot.writeString(_iter856.getKey());
+              _iter856.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -439,10 +439,10 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
       if (struct.is_set_named_logger_level()) {
         {
           oprot.writeI32(struct.named_logger_level.size());
-          for (Map.Entry<String, LogLevel> _iter837 : struct.named_logger_level.entrySet())
+          for (Map.Entry<String, LogLevel> _iter857 : struct.named_logger_level.entrySet())
           {
-            oprot.writeString(_iter837.getKey());
-            _iter837.getValue().write(oprot);
+            oprot.writeString(_iter857.getKey());
+            _iter857.getValue().write(oprot);
           }
         }
       }
@@ -454,16 +454,16 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map838 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.named_logger_level = new HashMap<String,LogLevel>(2*_map838.size);
-          String _key839;
-          LogLevel _val840;
-          for (int _i841 = 0; _i841 < _map838.size; ++_i841)
+          org.apache.thrift.protocol.TMap _map858 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.named_logger_level = new HashMap<String,LogLevel>(2*_map858.size);
+          String _key859;
+          LogLevel _val860;
+          for (int _i861 = 0; _i861 < _map858.size; ++_i861)
           {
-            _key839 = iprot.readString();
-            _val840 = new LogLevel();
-            _val840.read(iprot);
-            struct.named_logger_level.put(_key839, _val840);
+            _key859 = iprot.readString();
+            _val860 = new LogLevel();
+            _val860.read(iprot);
+            struct.named_logger_level.put(_key859, _val860);
           }
         }
         struct.set_named_logger_level_isSet(true);


[3/7] storm git commit: STORM-2887: store metrics into RocksDB

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-client/src/py/storm/ttypes.py b/storm-client/src/py/storm/ttypes.py
index 9e6725c..5d942b2 100644
--- a/storm-client/src/py/storm/ttypes.py
+++ b/storm-client/src/py/storm/ttypes.py
@@ -9452,6 +9452,8 @@ class WorkerResources:
    - cpu
    - shared_mem_on_heap
    - shared_mem_off_heap
+   - resources
+   - shared_resources
   """
 
   thrift_spec = (
@@ -9461,14 +9463,18 @@ class WorkerResources:
     (3, TType.DOUBLE, 'cpu', None, None, ), # 3
     (4, TType.DOUBLE, 'shared_mem_on_heap', None, None, ), # 4
     (5, TType.DOUBLE, 'shared_mem_off_heap', None, None, ), # 5
+    (6, TType.MAP, 'resources', (TType.STRING,None,TType.DOUBLE,None), None, ), # 6
+    (7, TType.MAP, 'shared_resources', (TType.STRING,None,TType.DOUBLE,None), None, ), # 7
   )
 
-  def __init__(self, mem_on_heap=None, mem_off_heap=None, cpu=None, shared_mem_on_heap=None, shared_mem_off_heap=None,):
+  def __init__(self, mem_on_heap=None, mem_off_heap=None, cpu=None, shared_mem_on_heap=None, shared_mem_off_heap=None, resources=None, shared_resources=None,):
     self.mem_on_heap = mem_on_heap
     self.mem_off_heap = mem_off_heap
     self.cpu = cpu
     self.shared_mem_on_heap = shared_mem_on_heap
     self.shared_mem_off_heap = shared_mem_off_heap
+    self.resources = resources
+    self.shared_resources = shared_resources
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -9504,6 +9510,28 @@ class WorkerResources:
           self.shared_mem_off_heap = iprot.readDouble()
         else:
           iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.MAP:
+          self.resources = {}
+          (_ktype596, _vtype597, _size595 ) = iprot.readMapBegin()
+          for _i599 in xrange(_size595):
+            _key600 = iprot.readString().decode('utf-8')
+            _val601 = iprot.readDouble()
+            self.resources[_key600] = _val601
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.MAP:
+          self.shared_resources = {}
+          (_ktype603, _vtype604, _size602 ) = iprot.readMapBegin()
+          for _i606 in xrange(_size602):
+            _key607 = iprot.readString().decode('utf-8')
+            _val608 = iprot.readDouble()
+            self.shared_resources[_key607] = _val608
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -9534,6 +9562,22 @@ class WorkerResources:
       oprot.writeFieldBegin('shared_mem_off_heap', TType.DOUBLE, 5)
       oprot.writeDouble(self.shared_mem_off_heap)
       oprot.writeFieldEnd()
+    if self.resources is not None:
+      oprot.writeFieldBegin('resources', TType.MAP, 6)
+      oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.resources))
+      for kiter609,viter610 in self.resources.items():
+        oprot.writeString(kiter609.encode('utf-8'))
+        oprot.writeDouble(viter610)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.shared_resources is not None:
+      oprot.writeFieldBegin('shared_resources', TType.MAP, 7)
+      oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.shared_resources))
+      for kiter611,viter612 in self.shared_resources.items():
+        oprot.writeString(kiter611.encode('utf-8'))
+        oprot.writeDouble(viter612)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -9548,6 +9592,8 @@ class WorkerResources:
     value = (value * 31) ^ hash(self.cpu)
     value = (value * 31) ^ hash(self.shared_mem_on_heap)
     value = (value * 31) ^ hash(self.shared_mem_off_heap)
+    value = (value * 31) ^ hash(self.resources)
+    value = (value * 31) ^ hash(self.shared_resources)
     return value
 
   def __repr__(self):
@@ -9630,68 +9676,68 @@ class Assignment:
       elif fid == 2:
         if ftype == TType.MAP:
           self.node_host = {}
-          (_ktype596, _vtype597, _size595 ) = iprot.readMapBegin()
-          for _i599 in xrange(_size595):
-            _key600 = iprot.readString().decode('utf-8')
-            _val601 = iprot.readString().decode('utf-8')
-            self.node_host[_key600] = _val601
+          (_ktype614, _vtype615, _size613 ) = iprot.readMapBegin()
+          for _i617 in xrange(_size613):
+            _key618 = iprot.readString().decode('utf-8')
+            _val619 = iprot.readString().decode('utf-8')
+            self.node_host[_key618] = _val619
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.executor_node_port = {}
-          (_ktype603, _vtype604, _size602 ) = iprot.readMapBegin()
-          for _i606 in xrange(_size602):
-            _key607 = []
-            (_etype612, _size609) = iprot.readListBegin()
-            for _i613 in xrange(_size609):
-              _elem614 = iprot.readI64()
-              _key607.append(_elem614)
+          (_ktype621, _vtype622, _size620 ) = iprot.readMapBegin()
+          for _i624 in xrange(_size620):
+            _key625 = []
+            (_etype630, _size627) = iprot.readListBegin()
+            for _i631 in xrange(_size627):
+              _elem632 = iprot.readI64()
+              _key625.append(_elem632)
             iprot.readListEnd()
-            _val608 = NodeInfo()
-            _val608.read(iprot)
-            self.executor_node_port[_key607] = _val608
+            _val626 = NodeInfo()
+            _val626.read(iprot)
+            self.executor_node_port[_key625] = _val626
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.MAP:
           self.executor_start_time_secs = {}
-          (_ktype616, _vtype617, _size615 ) = iprot.readMapBegin()
-          for _i619 in xrange(_size615):
-            _key620 = []
-            (_etype625, _size622) = iprot.readListBegin()
-            for _i626 in xrange(_size622):
-              _elem627 = iprot.readI64()
-              _key620.append(_elem627)
+          (_ktype634, _vtype635, _size633 ) = iprot.readMapBegin()
+          for _i637 in xrange(_size633):
+            _key638 = []
+            (_etype643, _size640) = iprot.readListBegin()
+            for _i644 in xrange(_size640):
+              _elem645 = iprot.readI64()
+              _key638.append(_elem645)
             iprot.readListEnd()
-            _val621 = iprot.readI64()
-            self.executor_start_time_secs[_key620] = _val621
+            _val639 = iprot.readI64()
+            self.executor_start_time_secs[_key638] = _val639
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.MAP:
           self.worker_resources = {}
-          (_ktype629, _vtype630, _size628 ) = iprot.readMapBegin()
-          for _i632 in xrange(_size628):
-            _key633 = NodeInfo()
-            _key633.read(iprot)
-            _val634 = WorkerResources()
-            _val634.read(iprot)
-            self.worker_resources[_key633] = _val634
+          (_ktype647, _vtype648, _size646 ) = iprot.readMapBegin()
+          for _i650 in xrange(_size646):
+            _key651 = NodeInfo()
+            _key651.read(iprot)
+            _val652 = WorkerResources()
+            _val652.read(iprot)
+            self.worker_resources[_key651] = _val652
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.MAP:
           self.total_shared_off_heap = {}
-          (_ktype636, _vtype637, _size635 ) = iprot.readMapBegin()
-          for _i639 in xrange(_size635):
-            _key640 = iprot.readString().decode('utf-8')
-            _val641 = iprot.readDouble()
-            self.total_shared_off_heap[_key640] = _val641
+          (_ktype654, _vtype655, _size653 ) = iprot.readMapBegin()
+          for _i657 in xrange(_size653):
+            _key658 = iprot.readString().decode('utf-8')
+            _val659 = iprot.readDouble()
+            self.total_shared_off_heap[_key658] = _val659
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -9717,47 +9763,47 @@ class Assignment:
     if self.node_host is not None:
       oprot.writeFieldBegin('node_host', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.node_host))
-      for kiter642,viter643 in self.node_host.items():
-        oprot.writeString(kiter642.encode('utf-8'))
-        oprot.writeString(viter643.encode('utf-8'))
+      for kiter660,viter661 in self.node_host.items():
+        oprot.writeString(kiter660.encode('utf-8'))
+        oprot.writeString(viter661.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_node_port is not None:
       oprot.writeFieldBegin('executor_node_port', TType.MAP, 3)
       oprot.writeMapBegin(TType.LIST, TType.STRUCT, len(self.executor_node_port))
-      for kiter644,viter645 in self.executor_node_port.items():
-        oprot.writeListBegin(TType.I64, len(kiter644))
-        for iter646 in kiter644:
-          oprot.writeI64(iter646)
+      for kiter662,viter663 in self.executor_node_port.items():
+        oprot.writeListBegin(TType.I64, len(kiter662))
+        for iter664 in kiter662:
+          oprot.writeI64(iter664)
         oprot.writeListEnd()
-        viter645.write(oprot)
+        viter663.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_start_time_secs is not None:
       oprot.writeFieldBegin('executor_start_time_secs', TType.MAP, 4)
       oprot.writeMapBegin(TType.LIST, TType.I64, len(self.executor_start_time_secs))
-      for kiter647,viter648 in self.executor_start_time_secs.items():
-        oprot.writeListBegin(TType.I64, len(kiter647))
-        for iter649 in kiter647:
-          oprot.writeI64(iter649)
+      for kiter665,viter666 in self.executor_start_time_secs.items():
+        oprot.writeListBegin(TType.I64, len(kiter665))
+        for iter667 in kiter665:
+          oprot.writeI64(iter667)
         oprot.writeListEnd()
-        oprot.writeI64(viter648)
+        oprot.writeI64(viter666)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.worker_resources is not None:
       oprot.writeFieldBegin('worker_resources', TType.MAP, 5)
       oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.worker_resources))
-      for kiter650,viter651 in self.worker_resources.items():
-        kiter650.write(oprot)
-        viter651.write(oprot)
+      for kiter668,viter669 in self.worker_resources.items():
+        kiter668.write(oprot)
+        viter669.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.total_shared_off_heap is not None:
       oprot.writeFieldBegin('total_shared_off_heap', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.total_shared_off_heap))
-      for kiter652,viter653 in self.total_shared_off_heap.items():
-        oprot.writeString(kiter652.encode('utf-8'))
-        oprot.writeDouble(viter653)
+      for kiter670,viter671 in self.total_shared_off_heap.items():
+        oprot.writeString(kiter670.encode('utf-8'))
+        oprot.writeDouble(viter671)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.owner is not None:
@@ -9946,11 +9992,11 @@ class StormBase:
       elif fid == 4:
         if ftype == TType.MAP:
           self.component_executors = {}
-          (_ktype655, _vtype656, _size654 ) = iprot.readMapBegin()
-          for _i658 in xrange(_size654):
-            _key659 = iprot.readString().decode('utf-8')
-            _val660 = iprot.readI32()
-            self.component_executors[_key659] = _val660
+          (_ktype673, _vtype674, _size672 ) = iprot.readMapBegin()
+          for _i676 in xrange(_size672):
+            _key677 = iprot.readString().decode('utf-8')
+            _val678 = iprot.readI32()
+            self.component_executors[_key677] = _val678
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -9978,12 +10024,12 @@ class StormBase:
       elif fid == 9:
         if ftype == TType.MAP:
           self.component_debug = {}
-          (_ktype662, _vtype663, _size661 ) = iprot.readMapBegin()
-          for _i665 in xrange(_size661):
-            _key666 = iprot.readString().decode('utf-8')
-            _val667 = DebugOptions()
-            _val667.read(iprot)
-            self.component_debug[_key666] = _val667
+          (_ktype680, _vtype681, _size679 ) = iprot.readMapBegin()
+          for _i683 in xrange(_size679):
+            _key684 = iprot.readString().decode('utf-8')
+            _val685 = DebugOptions()
+            _val685.read(iprot)
+            self.component_debug[_key684] = _val685
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -10022,9 +10068,9 @@ class StormBase:
     if self.component_executors is not None:
       oprot.writeFieldBegin('component_executors', TType.MAP, 4)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.component_executors))
-      for kiter668,viter669 in self.component_executors.items():
-        oprot.writeString(kiter668.encode('utf-8'))
-        oprot.writeI32(viter669)
+      for kiter686,viter687 in self.component_executors.items():
+        oprot.writeString(kiter686.encode('utf-8'))
+        oprot.writeI32(viter687)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.launch_time_secs is not None:
@@ -10046,9 +10092,9 @@ class StormBase:
     if self.component_debug is not None:
       oprot.writeFieldBegin('component_debug', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug))
-      for kiter670,viter671 in self.component_debug.items():
-        oprot.writeString(kiter670.encode('utf-8'))
-        viter671.write(oprot)
+      for kiter688,viter689 in self.component_debug.items():
+        oprot.writeString(kiter688.encode('utf-8'))
+        viter689.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.principal is not None:
@@ -10138,13 +10184,13 @@ class ClusterWorkerHeartbeat:
       elif fid == 2:
         if ftype == TType.MAP:
           self.executor_stats = {}
-          (_ktype673, _vtype674, _size672 ) = iprot.readMapBegin()
-          for _i676 in xrange(_size672):
-            _key677 = ExecutorInfo()
-            _key677.read(iprot)
-            _val678 = ExecutorStats()
-            _val678.read(iprot)
-            self.executor_stats[_key677] = _val678
+          (_ktype691, _vtype692, _size690 ) = iprot.readMapBegin()
+          for _i694 in xrange(_size690):
+            _key695 = ExecutorInfo()
+            _key695.read(iprot)
+            _val696 = ExecutorStats()
+            _val696.read(iprot)
+            self.executor_stats[_key695] = _val696
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -10175,9 +10221,9 @@ class ClusterWorkerHeartbeat:
     if self.executor_stats is not None:
       oprot.writeFieldBegin('executor_stats', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.executor_stats))
-      for kiter679,viter680 in self.executor_stats.items():
-        kiter679.write(oprot)
-        viter680.write(oprot)
+      for kiter697,viter698 in self.executor_stats.items():
+        kiter697.write(oprot)
+        viter698.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.time_secs is not None:
@@ -10330,12 +10376,12 @@ class LocalStateData:
       if fid == 1:
         if ftype == TType.MAP:
           self.serialized_parts = {}
-          (_ktype682, _vtype683, _size681 ) = iprot.readMapBegin()
-          for _i685 in xrange(_size681):
-            _key686 = iprot.readString().decode('utf-8')
-            _val687 = ThriftSerializedObject()
-            _val687.read(iprot)
-            self.serialized_parts[_key686] = _val687
+          (_ktype700, _vtype701, _size699 ) = iprot.readMapBegin()
+          for _i703 in xrange(_size699):
+            _key704 = iprot.readString().decode('utf-8')
+            _val705 = ThriftSerializedObject()
+            _val705.read(iprot)
+            self.serialized_parts[_key704] = _val705
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -10352,9 +10398,9 @@ class LocalStateData:
     if self.serialized_parts is not None:
       oprot.writeFieldBegin('serialized_parts', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.serialized_parts))
-      for kiter688,viter689 in self.serialized_parts.items():
-        oprot.writeString(kiter688.encode('utf-8'))
-        viter689.write(oprot)
+      for kiter706,viter707 in self.serialized_parts.items():
+        oprot.writeString(kiter706.encode('utf-8'))
+        viter707.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10425,11 +10471,11 @@ class LocalAssignment:
       elif fid == 2:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype693, _size690) = iprot.readListBegin()
-          for _i694 in xrange(_size690):
-            _elem695 = ExecutorInfo()
-            _elem695.read(iprot)
-            self.executors.append(_elem695)
+          (_etype711, _size708) = iprot.readListBegin()
+          for _i712 in xrange(_size708):
+            _elem713 = ExecutorInfo()
+            _elem713.read(iprot)
+            self.executors.append(_elem713)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10466,8 +10512,8 @@ class LocalAssignment:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter696 in self.executors:
-        iter696.write(oprot)
+      for iter714 in self.executors:
+        iter714.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.resources is not None:
@@ -10606,11 +10652,11 @@ class LSApprovedWorkers:
       if fid == 1:
         if ftype == TType.MAP:
           self.approved_workers = {}
-          (_ktype698, _vtype699, _size697 ) = iprot.readMapBegin()
-          for _i701 in xrange(_size697):
-            _key702 = iprot.readString().decode('utf-8')
-            _val703 = iprot.readI32()
-            self.approved_workers[_key702] = _val703
+          (_ktype716, _vtype717, _size715 ) = iprot.readMapBegin()
+          for _i719 in xrange(_size715):
+            _key720 = iprot.readString().decode('utf-8')
+            _val721 = iprot.readI32()
+            self.approved_workers[_key720] = _val721
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -10627,9 +10673,9 @@ class LSApprovedWorkers:
     if self.approved_workers is not None:
       oprot.writeFieldBegin('approved_workers', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.approved_workers))
-      for kiter704,viter705 in self.approved_workers.items():
-        oprot.writeString(kiter704.encode('utf-8'))
-        oprot.writeI32(viter705)
+      for kiter722,viter723 in self.approved_workers.items():
+        oprot.writeString(kiter722.encode('utf-8'))
+        oprot.writeI32(viter723)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10683,12 +10729,12 @@ class LSSupervisorAssignments:
       if fid == 1:
         if ftype == TType.MAP:
           self.assignments = {}
-          (_ktype707, _vtype708, _size706 ) = iprot.readMapBegin()
-          for _i710 in xrange(_size706):
-            _key711 = iprot.readI32()
-            _val712 = LocalAssignment()
-            _val712.read(iprot)
-            self.assignments[_key711] = _val712
+          (_ktype725, _vtype726, _size724 ) = iprot.readMapBegin()
+          for _i728 in xrange(_size724):
+            _key729 = iprot.readI32()
+            _val730 = LocalAssignment()
+            _val730.read(iprot)
+            self.assignments[_key729] = _val730
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -10705,9 +10751,9 @@ class LSSupervisorAssignments:
     if self.assignments is not None:
       oprot.writeFieldBegin('assignments', TType.MAP, 1)
       oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.assignments))
-      for kiter713,viter714 in self.assignments.items():
-        oprot.writeI32(kiter713)
-        viter714.write(oprot)
+      for kiter731,viter732 in self.assignments.items():
+        oprot.writeI32(kiter731)
+        viter732.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10780,11 +10826,11 @@ class LSWorkerHeartbeat:
       elif fid == 3:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype718, _size715) = iprot.readListBegin()
-          for _i719 in xrange(_size715):
-            _elem720 = ExecutorInfo()
-            _elem720.read(iprot)
-            self.executors.append(_elem720)
+          (_etype736, _size733) = iprot.readListBegin()
+          for _i737 in xrange(_size733):
+            _elem738 = ExecutorInfo()
+            _elem738.read(iprot)
+            self.executors.append(_elem738)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10814,8 +10860,8 @@ class LSWorkerHeartbeat:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter721 in self.executors:
-        iter721.write(oprot)
+      for iter739 in self.executors:
+        iter739.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.port is not None:
@@ -10901,20 +10947,20 @@ class LSTopoHistory:
       elif fid == 3:
         if ftype == TType.LIST:
           self.users = []
-          (_etype725, _size722) = iprot.readListBegin()
-          for _i726 in xrange(_size722):
-            _elem727 = iprot.readString().decode('utf-8')
-            self.users.append(_elem727)
+          (_etype743, _size740) = iprot.readListBegin()
+          for _i744 in xrange(_size740):
+            _elem745 = iprot.readString().decode('utf-8')
+            self.users.append(_elem745)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.groups = []
-          (_etype731, _size728) = iprot.readListBegin()
-          for _i732 in xrange(_size728):
-            _elem733 = iprot.readString().decode('utf-8')
-            self.groups.append(_elem733)
+          (_etype749, _size746) = iprot.readListBegin()
+          for _i750 in xrange(_size746):
+            _elem751 = iprot.readString().decode('utf-8')
+            self.groups.append(_elem751)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10939,15 +10985,15 @@ class LSTopoHistory:
     if self.users is not None:
       oprot.writeFieldBegin('users', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.users))
-      for iter734 in self.users:
-        oprot.writeString(iter734.encode('utf-8'))
+      for iter752 in self.users:
+        oprot.writeString(iter752.encode('utf-8'))
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.groups is not None:
       oprot.writeFieldBegin('groups', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.groups))
-      for iter735 in self.groups:
-        oprot.writeString(iter735.encode('utf-8'))
+      for iter753 in self.groups:
+        oprot.writeString(iter753.encode('utf-8'))
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11010,11 +11056,11 @@ class LSTopoHistoryList:
       if fid == 1:
         if ftype == TType.LIST:
           self.topo_history = []
-          (_etype739, _size736) = iprot.readListBegin()
-          for _i740 in xrange(_size736):
-            _elem741 = LSTopoHistory()
-            _elem741.read(iprot)
-            self.topo_history.append(_elem741)
+          (_etype757, _size754) = iprot.readListBegin()
+          for _i758 in xrange(_size754):
+            _elem759 = LSTopoHistory()
+            _elem759.read(iprot)
+            self.topo_history.append(_elem759)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11031,8 +11077,8 @@ class LSTopoHistoryList:
     if self.topo_history is not None:
       oprot.writeFieldBegin('topo_history', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.topo_history))
-      for iter742 in self.topo_history:
-        iter742.write(oprot)
+      for iter760 in self.topo_history:
+        iter760.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11367,12 +11413,12 @@ class LogConfig:
       if fid == 2:
         if ftype == TType.MAP:
           self.named_logger_level = {}
-          (_ktype744, _vtype745, _size743 ) = iprot.readMapBegin()
-          for _i747 in xrange(_size743):
-            _key748 = iprot.readString().decode('utf-8')
-            _val749 = LogLevel()
-            _val749.read(iprot)
-            self.named_logger_level[_key748] = _val749
+          (_ktype762, _vtype763, _size761 ) = iprot.readMapBegin()
+          for _i765 in xrange(_size761):
+            _key766 = iprot.readString().decode('utf-8')
+            _val767 = LogLevel()
+            _val767.read(iprot)
+            self.named_logger_level[_key766] = _val767
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -11389,9 +11435,9 @@ class LogConfig:
     if self.named_logger_level is not None:
       oprot.writeFieldBegin('named_logger_level', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.named_logger_level))
-      for kiter750,viter751 in self.named_logger_level.items():
-        oprot.writeString(kiter750.encode('utf-8'))
-        viter751.write(oprot)
+      for kiter768,viter769 in self.named_logger_level.items():
+        oprot.writeString(kiter768.encode('utf-8'))
+        viter769.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11443,10 +11489,10 @@ class TopologyHistoryInfo:
       if fid == 1:
         if ftype == TType.LIST:
           self.topo_ids = []
-          (_etype755, _size752) = iprot.readListBegin()
-          for _i756 in xrange(_size752):
-            _elem757 = iprot.readString().decode('utf-8')
-            self.topo_ids.append(_elem757)
+          (_etype773, _size770) = iprot.readListBegin()
+          for _i774 in xrange(_size770):
+            _elem775 = iprot.readString().decode('utf-8')
+            self.topo_ids.append(_elem775)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11463,8 +11509,8 @@ class TopologyHistoryInfo:
     if self.topo_ids is not None:
       oprot.writeFieldBegin('topo_ids', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.topo_ids))
-      for iter758 in self.topo_ids:
-        oprot.writeString(iter758.encode('utf-8'))
+      for iter776 in self.topo_ids:
+        oprot.writeString(iter776.encode('utf-8'))
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11778,6 +11824,335 @@ class OwnerResourceSummary:
   def __ne__(self, other):
     return not (self == other)
 
+class WorkerMetricPoint:
+  """
+  Attributes:
+   - metricName
+   - timestamp
+   - metricValue
+   - componentId
+   - executorId
+   - streamId
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'metricName', None, None, ), # 1
+    (2, TType.I64, 'timestamp', None, None, ), # 2
+    (3, TType.DOUBLE, 'metricValue', None, None, ), # 3
+    (4, TType.STRING, 'componentId', None, None, ), # 4
+    (5, TType.STRING, 'executorId', None, None, ), # 5
+    (6, TType.STRING, 'streamId', None, None, ), # 6
+  )
+
+  def __init__(self, metricName=None, timestamp=None, metricValue=None, componentId=None, executorId=None, streamId=None,):
+    self.metricName = metricName
+    self.timestamp = timestamp
+    self.metricValue = metricValue
+    self.componentId = componentId
+    self.executorId = executorId
+    self.streamId = streamId
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.metricName = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I64:
+          self.timestamp = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.DOUBLE:
+          self.metricValue = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.componentId = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.executorId = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.streamId = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WorkerMetricPoint')
+    if self.metricName is not None:
+      oprot.writeFieldBegin('metricName', TType.STRING, 1)
+      oprot.writeString(self.metricName.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.timestamp is not None:
+      oprot.writeFieldBegin('timestamp', TType.I64, 2)
+      oprot.writeI64(self.timestamp)
+      oprot.writeFieldEnd()
+    if self.metricValue is not None:
+      oprot.writeFieldBegin('metricValue', TType.DOUBLE, 3)
+      oprot.writeDouble(self.metricValue)
+      oprot.writeFieldEnd()
+    if self.componentId is not None:
+      oprot.writeFieldBegin('componentId', TType.STRING, 4)
+      oprot.writeString(self.componentId.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.executorId is not None:
+      oprot.writeFieldBegin('executorId', TType.STRING, 5)
+      oprot.writeString(self.executorId.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.streamId is not None:
+      oprot.writeFieldBegin('streamId', TType.STRING, 6)
+      oprot.writeString(self.streamId.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.metricName is None:
+      raise TProtocol.TProtocolException(message='Required field metricName is unset!')
+    if self.timestamp is None:
+      raise TProtocol.TProtocolException(message='Required field timestamp is unset!')
+    if self.metricValue is None:
+      raise TProtocol.TProtocolException(message='Required field metricValue is unset!')
+    if self.componentId is None:
+      raise TProtocol.TProtocolException(message='Required field componentId is unset!')
+    if self.executorId is None:
+      raise TProtocol.TProtocolException(message='Required field executorId is unset!')
+    if self.streamId is None:
+      raise TProtocol.TProtocolException(message='Required field streamId is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.metricName)
+    value = (value * 31) ^ hash(self.timestamp)
+    value = (value * 31) ^ hash(self.metricValue)
+    value = (value * 31) ^ hash(self.componentId)
+    value = (value * 31) ^ hash(self.executorId)
+    value = (value * 31) ^ hash(self.streamId)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class WorkerMetricList:
+  """
+  Attributes:
+   - metrics
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'metrics', (TType.STRUCT,(WorkerMetricPoint, WorkerMetricPoint.thrift_spec)), None, ), # 1
+  )
+
+  def __init__(self, metrics=None,):
+    self.metrics = metrics
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.metrics = []
+          (_etype780, _size777) = iprot.readListBegin()
+          for _i781 in xrange(_size777):
+            _elem782 = WorkerMetricPoint()
+            _elem782.read(iprot)
+            self.metrics.append(_elem782)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WorkerMetricList')
+    if self.metrics is not None:
+      oprot.writeFieldBegin('metrics', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.metrics))
+      for iter783 in self.metrics:
+        iter783.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.metrics)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class WorkerMetrics:
+  """
+  Attributes:
+   - topologyId
+   - port
+   - hostname
+   - metricList
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'topologyId', None, None, ), # 1
+    (2, TType.I32, 'port', None, None, ), # 2
+    (3, TType.STRING, 'hostname', None, None, ), # 3
+    (4, TType.STRUCT, 'metricList', (WorkerMetricList, WorkerMetricList.thrift_spec), None, ), # 4
+  )
+
+  def __init__(self, topologyId=None, port=None, hostname=None, metricList=None,):
+    self.topologyId = topologyId
+    self.port = port
+    self.hostname = hostname
+    self.metricList = metricList
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.topologyId = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.port = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.hostname = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRUCT:
+          self.metricList = WorkerMetricList()
+          self.metricList.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WorkerMetrics')
+    if self.topologyId is not None:
+      oprot.writeFieldBegin('topologyId', TType.STRING, 1)
+      oprot.writeString(self.topologyId.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.port is not None:
+      oprot.writeFieldBegin('port', TType.I32, 2)
+      oprot.writeI32(self.port)
+      oprot.writeFieldEnd()
+    if self.hostname is not None:
+      oprot.writeFieldBegin('hostname', TType.STRING, 3)
+      oprot.writeString(self.hostname.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.metricList is not None:
+      oprot.writeFieldBegin('metricList', TType.STRUCT, 4)
+      self.metricList.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.topologyId is None:
+      raise TProtocol.TProtocolException(message='Required field topologyId is unset!')
+    if self.port is None:
+      raise TProtocol.TProtocolException(message='Required field port is unset!')
+    if self.hostname is None:
+      raise TProtocol.TProtocolException(message='Required field hostname is unset!')
+    if self.metricList is None:
+      raise TProtocol.TProtocolException(message='Required field metricList is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.topologyId)
+    value = (value * 31) ^ hash(self.port)
+    value = (value * 31) ^ hash(self.hostname)
+    value = (value * 31) ^ hash(self.metricList)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class DRPCRequest:
   """
   Attributes:
@@ -12049,11 +12424,11 @@ class HBRecords:
       if fid == 1:
         if ftype == TType.LIST:
           self.pulses = []
-          (_etype762, _size759) = iprot.readListBegin()
-          for _i763 in xrange(_size759):
-            _elem764 = HBPulse()
-            _elem764.read(iprot)
-            self.pulses.append(_elem764)
+          (_etype787, _size784) = iprot.readListBegin()
+          for _i788 in xrange(_size784):
+            _elem789 = HBPulse()
+            _elem789.read(iprot)
+            self.pulses.append(_elem789)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12070,8 +12445,8 @@ class HBRecords:
     if self.pulses is not None:
       oprot.writeFieldBegin('pulses', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.pulses))
-      for iter765 in self.pulses:
-        iter765.write(oprot)
+      for iter790 in self.pulses:
+        iter790.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12123,10 +12498,10 @@ class HBNodes:
       if fid == 1:
         if ftype == TType.LIST:
           self.pulseIds = []
-          (_etype769, _size766) = iprot.readListBegin()
-          for _i770 in xrange(_size766):
-            _elem771 = iprot.readString().decode('utf-8')
-            self.pulseIds.append(_elem771)
+          (_etype794, _size791) = iprot.readListBegin()
+          for _i795 in xrange(_size791):
+            _elem796 = iprot.readString().decode('utf-8')
+            self.pulseIds.append(_elem796)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12143,8 +12518,8 @@ class HBNodes:
     if self.pulseIds is not None:
       oprot.writeFieldBegin('pulseIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.pulseIds))
-      for iter772 in self.pulseIds:
-        oprot.writeString(iter772.encode('utf-8'))
+      for iter797 in self.pulseIds:
+        oprot.writeString(iter797.encode('utf-8'))
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-client/src/storm.thrift b/storm-client/src/storm.thrift
index aff7507..c5140f3 100644
--- a/storm-client/src/storm.thrift
+++ b/storm-client/src/storm.thrift
@@ -672,6 +672,26 @@ struct OwnerResourceSummary {
   18: optional double assigned_off_heap_memory;
 }
 
+struct WorkerMetricPoint {
+  1: required string metricName;
+  2: required i64 timestamp;
+  3: required double metricValue;
+  4: required string componentId;
+  5: required string executorId;
+  6: required string streamId;
+}
+
+struct WorkerMetricList {
+  1: list<WorkerMetricPoint> metrics;
+}
+
+struct WorkerMetrics {
+  1: required string topologyId;
+  2: required i32 port;
+  3: required string hostname;
+  4: required WorkerMetricList metricList;
+}
+
 service Nimbus {
   void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
   void submitTopologyWithOpts(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology, 5: SubmitOptions options) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
@@ -748,6 +768,7 @@ service Nimbus {
   StormTopology getUserTopology(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
   TopologyHistoryInfo getTopologyHistory(1: string user) throws (1: AuthorizationException aze);
   list<OwnerResourceSummary> getOwnerResourceSummaries (1: string owner) throws (1: AuthorizationException aze);
+  void processWorkerMetrics(1: WorkerMetrics metrics);
 }
 
 struct DRPCRequest {
@@ -836,3 +857,5 @@ exception HBAuthorizationException {
 exception HBExecutionException {
   1: required string msg;
 }
+
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/pom.xml
----------------------------------------------------------------------
diff --git a/storm-server/pom.xml b/storm-server/pom.xml
index 7c301a4..cfe2d74 100644
--- a/storm-server/pom.xml
+++ b/storm-server/pom.xml
@@ -64,6 +64,10 @@
             <artifactId>auto-service</artifactId>
             <optional>true</optional>
         </dependency>
+        <dependency>
+            <groupId>org.rocksdb</groupId>
+            <artifactId>rocksdbjni</artifactId>
+        </dependency>
 
         <!-- test -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/DaemonConfig.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/DaemonConfig.java b/storm-server/src/main/java/org/apache/storm/DaemonConfig.java
index f806562..a6881b7 100644
--- a/storm-server/src/main/java/org/apache/storm/DaemonConfig.java
+++ b/storm-server/src/main/java/org/apache/storm/DaemonConfig.java
@@ -19,10 +19,10 @@
 package org.apache.storm;
 
 import static org.apache.storm.validation.ConfigValidationAnnotations.isInteger;
+import static org.apache.storm.validation.ConfigValidationAnnotations.isPositiveNumber;
 import static org.apache.storm.validation.ConfigValidationAnnotations.isString;
 import static org.apache.storm.validation.ConfigValidationAnnotations.isStringList;
 import static org.apache.storm.validation.ConfigValidationAnnotations.isStringOrStringList;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isPositiveNumber;
 import static org.apache.storm.validation.ConfigValidationAnnotations.NotNull;
 import static org.apache.storm.validation.ConfigValidationAnnotations.isListEntryCustom;
 import static org.apache.storm.validation.ConfigValidationAnnotations.isBoolean;
@@ -33,6 +33,7 @@ import static org.apache.storm.validation.ConfigValidationAnnotations.isNoDuplic
 import static org.apache.storm.validation.ConfigValidationAnnotations.isMapEntryCustom;
 
 import org.apache.storm.container.ResourceIsolationInterface;
+import org.apache.storm.metricstore.MetricStore;
 import org.apache.storm.nimbus.ITopologyActionNotifierPlugin;
 import org.apache.storm.scheduler.blacklist.reporters.IReporter;
 import org.apache.storm.scheduler.blacklist.strategies.IBlacklistStrategy;
@@ -55,7 +56,7 @@ import java.util.Map;
 public class DaemonConfig implements Validated {
 
     /**
-     * We check with this interval that whether the Netty channel is writable and try to write pending messages
+     * We check with this interval that whether the Netty channel is writable and try to write pending messages.
      */
     @isInteger
     public static final String STORM_NETTY_FLUSH_CHECK_INTERVAL_MS = "storm.messaging.netty.flush.check.interval.ms";
@@ -164,7 +165,7 @@ public class DaemonConfig implements Validated {
 
     /**
      * The time to allow any given healthcheck script to run before it
-     * is marked failed due to timeout
+     * is marked failed due to timeout.
      */
     @isNumber
     public static final String STORM_HEALTH_CHECK_TIMEOUT_MS = "storm.health.check.timeout.ms";
@@ -809,7 +810,7 @@ public class DaemonConfig implements Validated {
      * Enables user-first classpath. See topology.classpath.beginning.
      */
     @isBoolean
-    public static final String STORM_TOPOLOGY_CLASSPATH_BEGINNING_ENABLED="storm.topology.classpath.beginning.enabled";
+    public static final String STORM_TOPOLOGY_CLASSPATH_BEGINNING_ENABLED = "storm.topology.classpath.beginning.enabled";
 
     /**
      * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers)
@@ -905,17 +906,17 @@ public class DaemonConfig implements Validated {
     public static final String NIMBUS_CODE_SYNC_FREQ_SECS = "nimbus.code.sync.freq.secs";
 
     /**
-     * The plugin to be used for resource isolation
+     * The plugin to be used for resource isolation.
      */
     @isImplementationOfClass(implementsClass = ResourceIsolationInterface.class)
     public static final String STORM_RESOURCE_ISOLATION_PLUGIN = "storm.resource.isolation.plugin";
 
     /**
-     * CGroup Setting below
+     * CGroup Setting below.
      */
 
     /**
-     * resources to to be controlled by cgroups
+     * resources to to be controlled by cgroups.
      */
     @isStringList
     public static final String STORM_CGROUP_RESOURCES = "storm.cgroup.resources";
@@ -1030,6 +1031,48 @@ public class DaemonConfig implements Validated {
     public static String STORM_SUPERVISOR_MEDIUM_MEMORY_GRACE_PERIOD_MS =
         "storm.supervisor.medium.memory.grace.period.ms";
 
+    /**
+     * Class implementing MetricStore.
+     */
+    @NotNull
+    @isImplementationOfClass(implementsClass = MetricStore.class)
+    public static final String STORM_METRIC_STORE_CLASS = "storm.metricstore.class";
+
+    /**
+     * RocksDB file location. This setting is specific to the org.apache.storm.metricstore.rocksdb.RocksDbStore
+     * implementation for the storm.metricstore.class.
+     */
+    @isString
+    public static final String STORM_ROCKSDB_LOCATION = "storm.metricstore.rocksdb.location";
+
+    /**
+     * RocksDB create if missing flag. This setting is specific to the org.apache.storm.metricstore.rocksdb.RocksDbStore
+     * implementation for the storm.metricstore.class.
+     */
+    @isBoolean
+    public static final String STORM_ROCKSDB_CREATE_IF_MISSING = "storm.metricstore.rocksdb.create_if_missing";
+
+    /**
+     * RocksDB metadata cache capacity. This setting is specific to the org.apache.storm.metricstore.rocksdb.RocksDbStore
+     * implementation for the storm.metricstore.class.
+     */
+    @isInteger
+    public static final String STORM_ROCKSDB_METADATA_STRING_CACHE_CAPACITY = "storm.metricstore.rocksdb.metadata_string_cache_capacity";
+
+    /**
+     * RocksDB setting for length of metric retention. This setting is specific to the org.apache.storm.metricstore.rocksdb.RocksDbStore
+     * implementation for the storm.metricstore.class.
+     */
+    @isInteger
+    public static final String STORM_ROCKSDB_METRIC_RETENTION_HOURS = "storm.metricstore.rocksdb.retention_hours";
+
+    /**
+     * RocksDB setting for period of metric deletion thread. This setting is specific to the
+     * org.apache.storm.metricstore.rocksdb.RocksDbStore implementation for the storm.metricstore.class.
+     */
+    @isInteger
+    public static final String STORM_ROCKSDB_METRIC_DELETION_PERIOD_HOURS = "storm.metricstore.rocksdb.deletion_period_hours";
+
     // VALIDATION ONLY CONFIGS
     // Some configs inside Config.java may reference classes we don't want to expose in storm-client, but we still want to validate
     // That they reference a valid class.  To allow this to happen we do part of the validation on the client side with annotations on
@@ -1051,7 +1094,7 @@ public class DaemonConfig implements Validated {
     }
 
     /**
-     * Get the cgroup resources from the conf
+     * Get the cgroup resources from the conf.
      *
      * @param conf the config to read
      * @return the resources.

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/LocalCluster.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/LocalCluster.java b/storm-server/src/main/java/org/apache/storm/LocalCluster.java
index de3053e..502f454 100644
--- a/storm-server/src/main/java/org/apache/storm/LocalCluster.java
+++ b/storm-server/src/main/java/org/apache/storm/LocalCluster.java
@@ -77,6 +77,7 @@ import org.apache.storm.generated.SupervisorPageInfo;
 import org.apache.storm.generated.TopologyHistoryInfo;
 import org.apache.storm.generated.TopologyInfo;
 import org.apache.storm.generated.TopologyPageInfo;
+import org.apache.storm.generated.WorkerMetrics;
 import org.apache.storm.messaging.IContext;
 import org.apache.storm.messaging.local.Context;
 import org.apache.storm.nimbus.ILeaderElector;
@@ -1123,7 +1124,12 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
         // TODO Auto-generated method stub
         throw new RuntimeException("NOT IMPLEMENTED YET");
     }
-    
+
+    @Override
+    public void processWorkerMetrics(WorkerMetrics metrics) throws org.apache.thrift.TException {
+        getNimbus().processWorkerMetrics(metrics);
+    }
+
     public static void main(final String [] args) throws Exception {
         if (args.length < 1) {
             throw new IllegalArgumentException("No class was specified to run");

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
index afb1c28..37141e8 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
@@ -124,6 +124,8 @@ import org.apache.storm.generated.TopologyInitialStatus;
 import org.apache.storm.generated.TopologyPageInfo;
 import org.apache.storm.generated.TopologyStatus;
 import org.apache.storm.generated.TopologySummary;
+import org.apache.storm.generated.WorkerMetricPoint;
+import org.apache.storm.generated.WorkerMetrics;
 import org.apache.storm.generated.WorkerResources;
 import org.apache.storm.generated.WorkerSummary;
 import org.apache.storm.logging.ThriftAccessLogger;
@@ -132,6 +134,10 @@ import org.apache.storm.metric.StormMetricsRegistry;
 import org.apache.storm.metric.api.DataPoint;
 import org.apache.storm.metric.api.IClusterMetricsConsumer;
 import org.apache.storm.metric.api.IClusterMetricsConsumer.ClusterInfo;
+import org.apache.storm.metricstore.AggLevel;
+import org.apache.storm.metricstore.Metric;
+import org.apache.storm.metricstore.MetricStore;
+import org.apache.storm.metricstore.MetricStoreConfig;
 import org.apache.storm.nimbus.DefaultTopologyValidator;
 import org.apache.storm.nimbus.ILeaderElector;
 import org.apache.storm.nimbus.ITopologyActionNotifierPlugin;
@@ -230,6 +236,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
     private static final Meter getOwnerResourceSummariesCalls = StormMetricsRegistry.registerMeter(
             "nimbus:num-getOwnerResourceSummaries-calls");
     private static final Meter shutdownCalls = StormMetricsRegistry.registerMeter("nimbus:num-shutdown-calls");
+    private static final Meter processWorkerMetricsCalls = StormMetricsRegistry.registerMeter("nimbus:process-worker-metric-calls");
     // END Metrics
     
     private static final String STORM_VERSION = VersionInfo.getVersion();
@@ -336,7 +343,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
         return Nimbus.make(base.get_prev_status());
     };
     
-    private static final Map<TopologyStatus, Map<TopologyActions, TopologyStateTransition>> TOPO_STATE_TRANSITIONS = 
+    private static final Map<TopologyStatus, Map<TopologyActions, TopologyStateTransition>> TOPO_STATE_TRANSITIONS =
             new ImmutableMap.Builder<TopologyStatus, Map<TopologyActions, TopologyStateTransition>>()
             .put(TopologyStatus.ACTIVE, new ImmutableMap.Builder<TopologyActions, TopologyStateTransition>()
                     .put(TopologyActions.INACTIVATE, INACTIVE_TRANSITION)
@@ -597,7 +604,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
     
     /**
      * convert {topology-id -> SchedulerAssignment} to
-     *         {topology-id -> {executor [node port]}}
+     *         {topology-id -> {executor [node port]}}.
      * @return
      */
     private static Map<String, Map<List<Long>, List<Object>>> computeTopoToExecToNodePort(Map<String, SchedulerAssignment> schedAssignments) {
@@ -879,7 +886,8 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
         }
     }
     
-    private static StormTopology tryReadTopology(String topoId, TopoCache tc) throws NotAliveException, AuthorizationException, IOException {
+    private static StormTopology tryReadTopology(String topoId, TopoCache tc)
+            throws NotAliveException, AuthorizationException, IOException {
         try {
             return readStormTopologyAsNimbus(topoId, tc);
         } catch (KeyNotFoundException e) {
@@ -1001,10 +1009,10 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
         final Nimbus nimbus = new Nimbus(conf, inimbus);
         nimbus.launchServer();
         final ThriftServer server = new ThriftServer(conf, new Processor<>(nimbus), ThriftConnectionType.NIMBUS);
-        Utils.addShutdownHookWithForceKillIn1Sec(() -> {
+        Utils.addShutdownHookWithDelayedForceKill(() -> {
             nimbus.shutdown();
             server.stop();
-        });
+        }, 10);
         LOG.info("Starting nimbus server for storm version '{}'", STORM_VERSION);
         server.serve();
         return nimbus;
@@ -1022,6 +1030,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
     }
     
     private final Map<String, Object> conf;
+    private MetricStore metricsStore;
     private final NavigableMap<SimpleVersion, List<String>> supervisorClasspaths;
     private final NimbusInfo nimbusHostPortInfo;
     private final INimbus inimbus;
@@ -1098,6 +1107,15 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
             BlobStore blobStore, TopoCache topoCache, ILeaderElector leaderElector, IGroupMappingServiceProvider groupMapper)
         throws Exception {
         this.conf = conf;
+
+        this.metricsStore = null;
+        try {
+            this.metricsStore = MetricStoreConfig.configure(conf);
+        } catch (Exception e) {
+            // the metrics store is not critical to the operation of the cluster, allow Nimbus to come up
+            LOG.error("Failed to initialize metric store", e);
+        }
+
         if (hostPortInfo == null) {
             hostPortInfo = NimbusInfo.fromConf(conf);
         }
@@ -2730,7 +2748,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
             
             // lock protects against multiple topologies being submitted at once and
             // cleanup thread killing topology in b/w assignment and starting the topology
-            synchronized(submitLock) {
+            synchronized (submitLock) {
                 assertTopoActive(topoName, false);
                 //cred-update-lock is not needed here because creds are being added for the first time.
                 if (creds != null) {
@@ -3790,7 +3808,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
                     }
                     Map<WorkerSlot, WorkerResources> workerResources = getWorkerResourcesForTopology(topoId);
                     boolean isAllowed = userTopologies.contains(topoId);
-                    for (WorkerSummary workerSummary: StatsUtil.aggWorkerStats(topoId, topoName, taskToComp, beats, 
+                    for (WorkerSummary workerSummary: StatsUtil.aggWorkerStats(topoId, topoName, taskToComp, beats,
                             exec2NodePort, nodeToHost, workerResources, includeSys, isAllowed, sid)) {
                         pageInfo.add_to_worker_summaries(workerSummary);
                     }
@@ -3808,7 +3826,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
     
     @Override
     public ComponentPageInfo getComponentPageInfo(String topoId, String componentId, String window,
-            boolean includeSys) throws NotAliveException, AuthorizationException, TException {
+                                                  boolean includeSys) throws NotAliveException, AuthorizationException, TException {
         try {
             getComponentPageInfoCalls.mark();
             CommonTopoInfo info = getCommonTopoInfo(topoId, "getComponentPageInfo");
@@ -4004,8 +4022,8 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
         List<NimbusSummary> nimbuses = stormClusterState.nimbuses();
         NimbusInfo leader = leaderElector.getLeader();
         for (NimbusSummary nimbusSummary: nimbuses) {
-            if (leader.getHost().equals(nimbusSummary.get_host()) &&
-                    leader.getPort() == nimbusSummary.get_port()) {
+            if (leader.getHost().equals(nimbusSummary.get_host())
+                    && leader.getPort() == nimbusSummary.get_port()) {
                 nimbusSummary.set_uptime_secs(Time.deltaSecs(nimbusSummary.get_uptime_secs()));
                 nimbusSummary.set_isLeader(true);
                 return nimbusSummary;
@@ -4042,7 +4060,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
             //else, add only this owner (the input paramter) to the map
             Map<String, List<StormBase>> ownerToBasesMap = new HashMap<>();
 
-            if (owner == null){
+            if (owner == null) {
                 // add all the owners to the map
                 for (StormBase base: topoIdToBases.values()) {
                     String baseOwner = base.get_owner();
@@ -4174,6 +4192,9 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
             }
             if (zkClient != null) {
                 zkClient.close();
+            }            
+            if (metricsStore != null) {
+                metricsStore.close();
             }
             LOG.info("Shut down master");
         } catch (Exception e) {
@@ -4187,4 +4208,27 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
     public boolean isWaiting() {
         return timer.isTimerWaiting();
     }
+
+    @Override
+    public void processWorkerMetrics(WorkerMetrics metrics) throws org.apache.thrift.TException {
+        processWorkerMetricsCalls.mark();
+
+        checkAuthorization(null, null, "processWorkerMetrics");
+
+        if (this.metricsStore == null) {
+            return;
+        }
+
+        for (WorkerMetricPoint m : metrics.get_metricList().get_metrics()) {
+            try {
+                Metric metric = new Metric(m.get_metricName(), m.get_timestamp(), metrics.get_topologyId(),
+                        m.get_metricValue(), m.get_componentId(), m.get_executorId(), metrics.get_hostname(),
+                        m.get_streamId(), metrics.get_port(), AggLevel.AGG_LEVEL_NONE);
+                this.metricsStore.insert(metric);
+            } catch (Exception e) {
+                LOG.error("Failed to save metric", e);
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
index ff60a4c..f45ce25 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.daemon.supervisor;
 
 import java.io.BufferedReader;
@@ -39,11 +40,16 @@ import org.apache.storm.container.ResourceIsolationInterface;
 import org.apache.storm.generated.LSWorkerHeartbeat;
 import org.apache.storm.generated.LocalAssignment;
 import org.apache.storm.generated.ProfileRequest;
+import org.apache.storm.generated.WorkerMetricPoint;
+import org.apache.storm.generated.WorkerMetricList;
+import org.apache.storm.generated.WorkerMetrics;
 import org.apache.storm.metric.StormMetricsRegistry;
 import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.LocalState;
+import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.utils.ServerConfigUtils;
 import org.apache.storm.utils.ServerUtils;
-import org.apache.storm.utils.LocalState;
+import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
@@ -53,6 +59,11 @@ import org.yaml.snakeyaml.Yaml;
  */
 public abstract class Container implements Killable {
     private static final Logger LOG = LoggerFactory.getLogger(Container.class);
+    private static final String MEMORY_USED_METRIC = "UsedMemory";
+    private static final String SYSTEM_COMPONENT_ID = "System";
+    private static final String INVALID_EXECUTOR_ID = "-1";
+    private static final String INVALID_STREAM_ID = "None";
+
     public static enum ContainerType {
         LAUNCH(false, false),
         RECOVER_FULL(true, false),
@@ -137,6 +148,7 @@ public abstract class Container implements Killable {
     protected final ResourceIsolationInterface _resourceIsolationManager;
     protected ContainerType _type;
     protected final boolean _symlinksDisabled;
+    private long lastMetricProcessTime = 0L;
 
     /**
      * Create a new Container.
@@ -209,7 +221,7 @@ public abstract class Container implements Killable {
     }
     
     /**
-     * Kill a given process
+     * Kill a given process.
      * @param pid the id of the process to kill
      * @throws IOException
      */
@@ -218,7 +230,7 @@ public abstract class Container implements Killable {
     }
     
     /**
-     * Kill a given process
+     * Kill a given process.
      * @param pid the id of the process to kill
      * @throws IOException
      */
@@ -259,7 +271,7 @@ public abstract class Container implements Killable {
     }
 
     /**
-     * Is a process alive and running?
+     * Is a process alive and running?.
      * @param pid the PID of the running process
      * @param user the user that is expected to own that process
      * @return true if it is, else false
@@ -381,7 +393,7 @@ public abstract class Container implements Killable {
     }
     
     /**
-     * Write out the file used by the log viewer to allow/reject log access
+     * Write out the file used by the log viewer to allow/reject log access.
      * @param user the user this is going to run as
      * @throws IOException on any error
      */
@@ -429,7 +441,7 @@ public abstract class Container implements Killable {
     }
     
     /**
-     * Create symlink from the containers directory/artifacts to the artifacts directory
+     * Create symlink from the containers directory/artifacts to the artifacts directory.
      * @throws IOException on any error
      */
     protected void createArtifactsLink() throws IOException {
@@ -693,4 +705,41 @@ public abstract class Container implements Killable {
     public String getWorkerId() {
         return _workerId;
     }
+
+    /**
+     * Send worker metrics to Nimbus.
+     */
+    void processMetrics() {
+        try {
+            if (_usedMemory.get(_port) != null) {
+                // Make sure we don't process too frequently.
+                long nextMetricProcessTime = this.lastMetricProcessTime + 60L * 1000L;
+                long currentTimeMsec = System.currentTimeMillis();
+                if (currentTimeMsec < nextMetricProcessTime) {
+                    return;
+                }
+
+                String hostname = Utils.hostname();
+
+                // create metric for memory
+                long timestamp = System.currentTimeMillis();
+                double value = _usedMemory.get(_port).memory;
+                WorkerMetricPoint workerMetric = new WorkerMetricPoint(MEMORY_USED_METRIC, timestamp, value, SYSTEM_COMPONENT_ID,
+                        INVALID_EXECUTOR_ID, INVALID_STREAM_ID);
+
+                WorkerMetricList metricList = new WorkerMetricList();
+                metricList.add_to_metrics(workerMetric);
+                WorkerMetrics metrics = new WorkerMetrics(_topologyId, _port, hostname, metricList);
+
+                try (NimbusClient client = NimbusClient.getConfiguredClient(_conf)) {
+                    client.getClient().processWorkerMetrics(metrics);
+                }
+
+                this.lastMetricProcessTime = currentTimeMsec;
+            }
+        } catch (Exception e) {
+            LOG.error("Failed to process metrics", e);
+            this.lastMetricProcessTime = System.currentTimeMillis();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java
index cb41654..fe30c93 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java
@@ -936,6 +936,9 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
             }
             dynamicState = dynamicState.withProfileActions(mod, modPending);
         }
+
+        dynamicState.container.processMetrics();
+
         Time.sleep(staticState.monitorFreqMs);
         return dynamicState;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java b/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java
index c1e6121..28bba3e 100644
--- a/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java
+++ b/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.metric;
 
 import com.codahale.metrics.*;
@@ -53,6 +54,10 @@ public class StormMetricsRegistry {
         return register(name, gauge);
     }
 
+    public static void registerProvidedGauge(final String name, Gauge gauge) {
+        register(name, gauge);
+    }
+
     public static Histogram registerHistogram(String name, Reservoir reservoir) {
         Histogram histogram = new Histogram(reservoir);
         return register(name, histogram);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/AggLevel.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/AggLevel.java b/storm-server/src/main/java/org/apache/storm/metricstore/AggLevel.java
new file mode 100644
index 0000000..662a17c
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/AggLevel.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore;
+
+/**
+ * Specifies the available timeframes for Metric aggregation.
+ */
+public enum AggLevel {
+    AGG_LEVEL_NONE(0),
+    AGG_LEVEL_1_MIN(1),
+    AGG_LEVEL_10_MIN(10),
+    AGG_LEVEL_60_MIN(60);
+
+    private final byte value;
+
+    AggLevel(int value) {
+        this.value = (byte)value;
+    }
+
+    public byte getValue() {
+        return this.value;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/FilterOptions.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/FilterOptions.java b/storm-server/src/main/java/org/apache/storm/metricstore/FilterOptions.java
new file mode 100644
index 0000000..7cfbfbe
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/FilterOptions.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * FilterOptions provides a method to select various filtering options for doing a scan of the metrics database.
+ */
+public class FilterOptions {
+    private Set<AggLevel> aggLevels = null;
+    private long startTime = 0L;
+    private long endTime = -1L;
+    private String topologyId = null;
+    private String componentId = null;
+    private String metricName = null;
+    private String executorId = null;
+    private String hostId = null;
+    private Integer port = null;
+    private String streamId = null;
+
+    public FilterOptions() {
+    }
+
+    public void setTopologyId(String topoId) {
+        this.topologyId = topoId;
+    }
+
+    public String getTopologyId() {
+        return this.topologyId;
+    }
+
+    public void setComponentId(String component) {
+        this.componentId = component;
+    }
+
+    public String getComponentId() {
+        return this.componentId;
+    }
+
+    public void setStartTime(Long time) {
+        this.startTime = time;
+    }
+
+    public long getStartTime() {
+        return this.startTime;
+    }
+
+    public void setEndTime(Long time) {
+        this.endTime = time;
+    }
+
+    /**
+     *  Returns the end time if set, returns the current time otherwise.
+     */
+    public long getEndTime() {
+        if (this.endTime < 0L) {
+            this.endTime = System.currentTimeMillis();
+        }
+        return this.endTime;
+    }
+
+    public void setMetricName(String name) {
+        this.metricName = name;
+    }
+
+    public String getMetricName() {
+        return this.metricName;
+    }
+
+    public void setExecutorId(String id) {
+        this.executorId = id;
+    }
+
+    public String getExecutorId() {
+        return this.executorId;
+    }
+
+    public void setHostId(String id) {
+        this.hostId = id;
+    }
+
+    public String getHostId() {
+        return this.hostId;
+    }
+
+    public void setPort(Integer p) {
+        this.port = p;
+    }
+
+    public Integer getPort() {
+        return this.port;
+    }
+
+    public void setStreamId(String id) {
+        this.streamId = id;
+    }
+
+    public String getStreamId() {
+        return this.streamId;
+    }
+
+    /**
+     *  Add an aggregation level to search for.
+     */
+    public void addAggLevel(AggLevel level) {
+        if (this.aggLevels == null) {
+            this.aggLevels = new HashSet<>(1);
+        }
+        this.aggLevels.add(level);
+    }
+
+    /**
+     *  Set the aggregation levels to search for.
+     */
+    public void setAggLevels(Set<AggLevel> levels) throws MetricException {
+        this.aggLevels = levels;
+        if (this.aggLevels == null || this.aggLevels.isEmpty()) {
+            throw new MetricException("Cannot search for empty AggLevel");
+        }
+    }
+
+    /**
+     *  Get the aggregation levels to search for.
+     */
+    public Set<AggLevel> getAggLevels() {
+        if (this.aggLevels == null) {
+            // assume filter choices have been made and since no selection was made, all levels are valid
+            this.aggLevels = new HashSet<>(4);
+            aggLevels.add(AggLevel.AGG_LEVEL_NONE);
+            aggLevels.add(AggLevel.AGG_LEVEL_1_MIN);
+            aggLevels.add(AggLevel.AGG_LEVEL_10_MIN);
+            aggLevels.add(AggLevel.AGG_LEVEL_60_MIN);
+        }
+        return this.aggLevels;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/Metric.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/Metric.java b/storm-server/src/main/java/org/apache/storm/metricstore/Metric.java
new file mode 100644
index 0000000..716ced0
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/Metric.java
@@ -0,0 +1,270 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore;
+
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class containing metric values and all identifying fields to be stored in a MetricStore.
+ */
+public class Metric implements Comparable<Metric> {
+    private static final Logger LOG = LoggerFactory.getLogger(Metric.class);
+
+    // key fields
+    private String name;
+    private long timestamp;
+    private String topologyId;
+    private String componentId;
+    private String executorId;
+    private String hostname;
+    private String streamId;
+    private int port;
+    private AggLevel aggLevel = AggLevel.AGG_LEVEL_NONE;
+
+    // value fields
+    private double value;
+    private long count = 1;
+    private double min = 0.0;
+    private double max = 0.0;
+    private double sum = 0.0;
+
+
+    /**
+     *  Metric constructor.
+     */
+    public Metric(String name, Long timestamp, String topologyId, double value, String componentId, String executorId,
+                  String hostname, String streamId, int port, AggLevel aggLevel) throws MetricException {
+        this.name = name;
+        this.timestamp = timestamp;
+        this.topologyId = topologyId;
+        this.componentId = componentId;
+        this.executorId = executorId;
+        this.hostname = hostname;
+        this.streamId = streamId;
+        this.port = port;
+        this.setValue(value);
+        setAggLevel(aggLevel);
+    }
+
+    /**
+     *  A Metric constructor with the same settings cloned from another.
+     */
+    public Metric(Metric o) {
+        this.name = o.getMetricName();
+        this.timestamp = o.getTimestamp();
+        this.topologyId = o.getTopologyId();
+        this.value = o.getValue();
+        this.componentId = o.getComponentId();
+        this.executorId = o.getExecutorId();
+        this.hostname = o.getHostname();
+        this.streamId = o.getStreamId();
+        this.port = o.getPort();
+        this.count = o.getCount();
+        this.min = o.getMin();
+        this.max = o.getMax();
+        this.sum = o.getSum();
+        this.aggLevel = o.getAggLevel();
+    }
+
+    /**
+     *  Check if a Metric matches another object.
+     */
+    public boolean equals(Object other) {
+
+        if (!(other instanceof Metric)) {
+            return false;
+        }
+
+        Metric o = (Metric) other;
+
+        return this == other
+                || (this.name.equals(o.getMetricName())
+                && this.timestamp == o.getTimestamp()
+                && this.topologyId.equals(o.getTopologyId())
+                && this.value == o.getValue()
+                && this.componentId.equals(o.getComponentId())
+                && this.executorId.equals(o.getExecutorId())
+                && this.hostname.equals(o.getHostname())
+                && this.streamId.equals(o.getStreamId())
+                && this.port == o.getPort()
+                && this.count == o.getCount()
+                && this.min == o.getMin()
+                && this.max == o.getMax()
+                && this.sum == o.getSum()
+                && this.aggLevel == o.getAggLevel());
+    }
+
+    public AggLevel getAggLevel() {
+        return this.aggLevel;
+    }
+
+    /**
+     *  Set the aggLevel.
+     */
+    public void setAggLevel(AggLevel aggLevel) throws MetricException {
+        if (aggLevel == null) {
+            throw new MetricException("AggLevel not set for metric");
+        }
+        this.aggLevel = aggLevel;
+    }
+
+    /**
+     *  Initialize the metric value.
+     */
+    public void setValue(double value) {
+        this.count = 1L;
+        this.min = value;
+        this.max = value;
+        this.sum = value;
+        this.value = value;
+    }
+
+    /**
+     *  Adds an additional value to the metric.
+     */
+    public void addValue(double value) {
+        this.count += 1;
+        this.min = Math.min(this.min, value);
+        this.max = Math.max(this.max, value);
+        this.sum += value;
+        this.value = this.sum / this.count;
+    }
+
+    public double getSum() {
+        return this.sum;
+    }
+
+    public void setSum(double sum) {
+        this.sum = sum;
+    }
+
+    public long getCount() {
+        return this.count;
+    }
+
+    public void setCount(long count) {
+        this.count = count;
+    }
+
+    public double getMin() {
+        return this.min;
+    }
+
+    public void setMin(double min) {
+        this.min = min;
+    }
+
+    public double getMax() {
+        return this.max;
+    }
+
+    public void setMax(double max) {
+        this.max = max;
+    }
+
+    public String getTopologyId() {
+        return this.topologyId;
+    }
+
+    public void setTopologyId(String topologyId) {
+        this.topologyId = topologyId;
+    }
+
+    public long getTimestamp() {
+        return this.timestamp;
+    }
+
+    public void setTimestamp(long timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    public double getValue() {
+        return this.value;
+    }
+
+    public String getMetricName() {
+        return this.name;
+    }
+
+    public String getComponentId() {
+        return this.componentId;
+    }
+
+    public String getExecutorId() {
+        return this.executorId;
+    }
+
+    public String getHostname() {
+        return this.hostname;
+    }
+
+    public String getStreamId() {
+        return this.streamId;
+    }
+
+    public Integer getPort() {
+        return this.port;
+    }
+
+    @Override
+    public int compareTo(Metric o) {
+        long a = this.getTimestamp();
+        long b = o.getTimestamp();
+        return Long.compare(a, b);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        Date date = new Date(this.timestamp);
+        DateFormat format = new SimpleDateFormat("dd/MM/yyyy HH:mm:ss.SSS");
+        format.setTimeZone(TimeZone.getTimeZone("UTC"));
+        sb.append(format.format(date));
+        sb.append("|");
+        sb.append(this.topologyId);
+        sb.append("|");
+        sb.append(aggLevel);
+        sb.append("|");
+        sb.append(this.name);
+        sb.append("|");
+        sb.append(this.componentId);
+        sb.append("|");
+        sb.append(this.executorId);
+        sb.append("|");
+        sb.append(this.hostname);
+        sb.append("|");
+        sb.append(this.port);
+        sb.append("|");
+        sb.append(this.streamId);
+        return String.format("%s -- count: %d -- value: %f -- min: %f -- max: %f -- sum: %f",
+                sb.toString(),
+                this.count,
+                this.value,
+                this.min,
+                this.max,
+                this.sum);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/MetricException.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/MetricException.java b/storm-server/src/main/java/org/apache/storm/metricstore/MetricException.java
new file mode 100644
index 0000000..e45a451
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/MetricException.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore;
+
+/**
+ * A MetricException is used to describe an error using a MetricStore.
+ */
+public class MetricException extends Exception {
+    public MetricException(String message) {
+        super(message);
+    }
+
+    public MetricException(String message, Throwable e) {
+        super(message, e);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java b/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java
new file mode 100644
index 0000000..166333b
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.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.storm.metricstore;
+
+import java.util.Map;
+
+public interface MetricStore extends AutoCloseable {
+
+    /**
+     * Create metric store instance using the configurations provided via the config map.
+     *
+     * @param config Storm config map
+     * @throws MetricException on preparation error
+     */
+    void prepare(Map config) throws MetricException;
+
+    /**
+     * Stores a metric in the store.
+     *
+     * @param metric  Metric to store
+     * @throws MetricException on error
+     */
+    void insert(Metric metric) throws MetricException;
+
+    /**
+     * Fill out the numeric values for a metric.
+     *
+     * @param metric  Metric to populate
+     * @return   true if the metric was populated, false otherwise
+     * @throws MetricException on error
+     */
+    boolean populateValue(Metric metric) throws MetricException;
+
+    /**
+     * Close the metric store.
+     */
+    void close();
+
+    /**
+     *  Scans all metrics in the store and returns the ones matching the specified filtering options.
+     *
+     * @param filter   options to filter by
+     * @param scanCallback  callback for each Metric found
+     * @throws MetricException  on error
+     */
+    void scan(FilterOptions filter, ScanCallback scanCallback) throws MetricException;
+
+    /**
+     *  Interface used to callback metrics results from a scan.
+     */
+    interface ScanCallback {
+        void cb(Metric metric);
+    }
+}
+
+
+
+


[2/7] storm git commit: STORM-2887: store metrics into RocksDB

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/MetricStoreConfig.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/MetricStoreConfig.java b/storm-server/src/main/java/org/apache/storm/metricstore/MetricStoreConfig.java
new file mode 100644
index 0000000..2f2ad76
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/MetricStoreConfig.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore;
+
+import java.util.Map;
+import org.apache.storm.DaemonConfig;
+
+
+public class MetricStoreConfig {
+
+    /**
+     * Configures metrics store to use the class specified in the conf.
+     * @param conf Storm config map
+     * @return MetricStore prepared store
+     * @throws MetricException  on misconfiguration
+     */
+    public static MetricStore configure(Map conf) throws MetricException {
+
+        try {
+            String storeClass = (String)conf.get(DaemonConfig.STORM_METRIC_STORE_CLASS);
+            MetricStore store = (MetricStore) (Class.forName(storeClass)).newInstance();
+            store.prepare(conf);
+            return store;
+        } catch (Throwable t) {
+            throw new MetricException("Failed to create metric store", t);
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/KeyType.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/KeyType.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/KeyType.java
new file mode 100644
index 0000000..a351be7
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/KeyType.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Specifies all the valid types of keys and their values.
+ */
+public enum KeyType {
+    TOPOLOGY_BLOB(0), // reserved for saving topology data
+    METADATA_STRING_START(1),
+    TOPOLOGY_STRING(1),
+    METRIC_STRING(2),
+    COMPONENT_STRING(3),
+    EXEC_ID_STRING(4),
+    HOST_STRING(5),
+    STREAM_ID_STRING(6),
+    METADATA_STRING_END(7),
+    METRIC_DATA(0x80);
+
+    private final byte value;
+    private static Map<Byte, KeyType> MAP;
+
+    static {
+        MAP = new HashMap<>();
+        for (KeyType type : EnumSet.allOf(KeyType.class)) {
+            MAP.put(type.getValue(), type);
+        }
+        MAP = Collections.unmodifiableMap(MAP);
+    }
+
+    KeyType(int value) {
+        this.value = (byte)value;
+    }
+
+    byte getValue() {
+        return this.value;
+    }
+
+    static KeyType getKeyType(byte value) {
+        KeyType type = MAP.get(value);
+        if (type == null) {
+            throw new RuntimeException("Invalid key type " + value);
+        } else {
+            return type;
+        }
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/MetricsCleaner.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/MetricsCleaner.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/MetricsCleaner.java
new file mode 100644
index 0000000..6618f5d
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/MetricsCleaner.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.metricstore.FilterOptions;
+import org.apache.storm.metricstore.MetricException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class for removing expired metrics and unused metadata from the RocksDB store.
+ */
+public class MetricsCleaner implements Runnable, AutoCloseable {
+    private static final Logger LOG = LoggerFactory.getLogger(MetricsCleaner.class);
+    private static long DEFAULT_SLEEP_MS = 4L * 60L * 60L * 1000L;
+    private RocksDbStore store;
+    private long retentionHours;
+    private volatile boolean shutdown = false;
+    private long sleepMs = DEFAULT_SLEEP_MS;
+    private Meter failureMeter;
+    private long purgeTimestamp = 0L;
+
+    MetricsCleaner(RocksDbStore store, int retentionHours, int hourlyPeriod, Meter failureMeter) {
+        this.store = store;
+        this.retentionHours = retentionHours;
+        if (hourlyPeriod > 0) {
+            this.sleepMs = hourlyPeriod * 60L * 60L * 1000L;
+        }
+        this.failureMeter = failureMeter;
+
+        Gauge<Long> gauge = new Gauge<Long>() {
+            @Override
+            public Long getValue() {
+                return purgeTimestamp;
+            }
+        };
+        StormMetricsRegistry.registerProvidedGauge("MetricsCleaner:purgeTimestamp", gauge);
+    }
+
+    @Override
+    public void close() {
+        shutdown = true;
+    }
+
+    @Override
+    public void run() {
+        while (!shutdown) {
+            try {
+                Thread.sleep(sleepMs);
+            } catch (InterruptedException e) {
+                LOG.error("Sleep interrupted", e);
+                continue;
+            }
+
+            try {
+                purgeMetrics();
+            } catch (MetricException e) {
+                LOG.error("Failed to purge metrics", e);
+                if (this.failureMeter != null) {
+                    this.failureMeter.mark();
+                }
+            }
+        }
+    }
+
+    void purgeMetrics() throws MetricException {
+        purgeTimestamp = System.currentTimeMillis() - this.retentionHours * 60L * 60L * 1000L;
+
+        LOG.info("Purging metrics before {}", purgeTimestamp);
+
+        FilterOptions filter = new FilterOptions();
+        long endTime = purgeTimestamp - 1L;
+        filter.setEndTime(endTime);
+        store.deleteMetrics(filter);
+
+        LOG.info("Purging metadata before " + purgeTimestamp);
+        store.deleteMetadataBefore(purgeTimestamp);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/ReadOnlyStringMetadataCache.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/ReadOnlyStringMetadataCache.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/ReadOnlyStringMetadataCache.java
new file mode 100644
index 0000000..0effbc4
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/ReadOnlyStringMetadataCache.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import org.apache.http.annotation.ThreadSafe;
+
+/**
+ * The read-only interface to a StringMetadataCache allowed to be used by any thread.
+ */
+@ThreadSafe
+public interface ReadOnlyStringMetadataCache {
+
+    /**
+     * Get the string metadata from the cache.
+     *
+     * @param s   The string to look for
+     * @return   the metadata associated with the string or null if not found
+     */
+    StringMetadata get(String s);
+
+    /**
+     * Returns the string matching the string Id if in the cache.
+     *
+     * @param stringId   The string Id to check
+     * @return   the associated string if the Id is in the cache, null otherwise
+     */
+    String getMetadataString(Integer stringId);
+
+    /**
+     * Determines if a string Id is contained in the cache.
+     *
+     * @param stringId   The string Id to check
+     * @return   true if the Id is in the cache, false otherwise
+     */
+    boolean contains(Integer stringId);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbKey.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbKey.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbKey.java
new file mode 100644
index 0000000..7868282
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbKey.java
@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import com.google.common.primitives.UnsignedBytes;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+import javax.xml.bind.DatatypeConverter;
+import org.apache.storm.metricstore.AggLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Class representing the data used as a Key in RocksDB.  Keys can be used either for metadata or metrics.
+ *
+ * <P>Keys are 38 bytes in size.  The fields for a key are:
+ * <pre><
+ * Field             Size         Offset
+ *
+ * Type                 1              0      The type maps to the KeyType enum, specifying a metric or various types of metadata
+ * Aggregation Level    1              1      The aggregation level for a metric (see AggLevel enum).  0 for metadata.
+ * TopologyId           4              2      The metadata string Id representing a topologyId for a metric, or the unique
+ *                                                   string Id for a metadata string
+ * Timestamp            8              6      The timestamp for a metric, unused for metadata
+ * MetricId             4             14      The metadata string Id for the metric name
+ * ComponentId          4             18      The metadata string Id for the component Id
+ * ExecutorId           4             22      The metadata string Id for the executor Id
+ * HostId               4             26      The metadata string Id for the host Id
+ * Port                 4             30      The port number
+ * StreamId             4             34      The metadata string Id for the stream Id
+ * </pre>
+ */
+public class RocksDbKey implements Comparable<RocksDbKey> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDbKey.class);
+    static final int KEY_SIZE = 38;
+    private static Map<Byte, RocksDbKey> PREFIX_MAP = new HashMap<>();
+    private byte[] key;
+
+    static {
+        // pregenerate commonly used keys for scans
+        for (KeyType type : EnumSet.allOf(KeyType.class)) {
+            RocksDbKey key = new RocksDbKey(type, 0);
+            PREFIX_MAP.put(type.getValue(), key);
+        }
+        PREFIX_MAP = Collections.unmodifiableMap(PREFIX_MAP);
+    }
+
+    /**
+     * Constructor for a RocksDB key for a metadata string.
+     *
+     * @param type  type of metadata string
+     * @param metadataStringId  the string Id for the string (stored in the topologyId portion of the key)
+     */
+    RocksDbKey(KeyType type, int metadataStringId) {
+        byte[] key = new byte[KEY_SIZE];
+        ByteBuffer bb = ByteBuffer.wrap(key);
+        bb.put(type.getValue());
+        bb.put(AggLevel.AGG_LEVEL_NONE.getValue());
+        bb.putInt(metadataStringId);
+        this.key = key;
+    }
+
+    /**
+     * Constructor for a RocksDB key from raw data.
+     *
+     * @param raw  the key data
+     */
+    RocksDbKey(byte[] raw) {
+        this.key = raw;
+    }
+
+
+    /**
+     * Get a zeroed key of the specified type.
+     *
+     * @param type  the desired type
+     * @return  a key of the desired type
+     */
+    static RocksDbKey getPrefix(KeyType type) {
+        return PREFIX_MAP.get(type.getValue());
+    }
+
+    /**
+     * get the metadata string Id portion of the key for metadata keys.
+     *
+     * @return  the metadata string Id
+     * @throws  RuntimeException  if the key is not a metadata type
+     */
+    int getMetadataStringId() {
+        if (this.getType().getValue() < KeyType.METADATA_STRING_END.getValue()) {
+            return ByteBuffer.wrap(key, 2, 4).getInt();
+        } else {
+            throw new RuntimeException("Cannot fetch metadata string for key of type " + this.getType());
+        }
+    }
+
+    /**
+     * get the raw key bytes
+     */
+    byte[] getRaw() {
+        return this.key;
+    }
+
+    /**
+     * get the type of key.
+     *
+     * @return  the type of key
+     */
+    KeyType getType() {
+        return KeyType.getKeyType(key[0]);
+    }
+
+    /**
+     * compares to keys on a byte by byte basis.
+     *
+     * @return  comparison of key byte values
+     */
+    @Override
+    public int compareTo(RocksDbKey o) {
+        return UnsignedBytes.lexicographicalComparator().compare(this.getRaw(), o.getRaw());
+    }
+
+    /**
+     * gets the first possible key value for the desired key type.
+     *
+     * @return  the initial key
+     */
+    static RocksDbKey getInitialKey(KeyType type) {
+        return PREFIX_MAP.get(type.getValue());
+    }
+
+    /**
+     * gets the key just larger than the last possible key value for the desired key type.
+     *
+     * @return  the last key
+     */
+    static RocksDbKey getLastKey(KeyType type) {
+        byte value = (byte)(type.getValue() + 1);
+        return PREFIX_MAP.get(value);
+    }
+
+    /**
+     * Creates a metric key with the desired properties.
+     *
+     * @return  the generated key
+     */
+    static RocksDbKey createMetricKey(AggLevel aggLevel, int topologyId, long metricTimestamp, int metricId,
+                                      int componentId, int executorId, int hostId, int port,
+                                      int streamId) {
+        byte[] raw = new byte[KEY_SIZE];
+        ByteBuffer bb = ByteBuffer.wrap(raw);
+        bb.put(KeyType.METRIC_DATA.getValue());
+        bb.put(aggLevel.getValue());
+        bb.putInt(topologyId);       // offset 2
+        bb.putLong(metricTimestamp); // offset 6
+        bb.putInt(metricId);         // offset 14
+        bb.putInt(componentId);      // offset 18
+        bb.putInt(executorId);       // offset 22
+        bb.putInt(hostId);           // offset 26
+        bb.putInt(port);             // offset 30
+        bb.putInt(streamId);         // offset 34
+
+        RocksDbKey key = new RocksDbKey(raw);
+        return key;
+    }
+
+    /**
+     * Get the unique string Id for a metric's topologyId.
+     */
+    int getTopologyId() {
+        int val = ByteBuffer.wrap(key, 2, 4).getInt();
+        return val;
+    }
+
+    long getTimestamp() {
+        return ByteBuffer.wrap(key, 6, 8).getLong();
+    }
+
+    int getMetricId() {
+        return ByteBuffer.wrap(key, 14, 4).getInt();
+    }
+
+    int getComponentId() {
+        return ByteBuffer.wrap(key, 18, 4).getInt();
+    }
+
+    int getExecutorId() {
+        return ByteBuffer.wrap(key, 22, 4).getInt();
+    }
+
+    int getHostnameId() {
+        return ByteBuffer.wrap(key, 26, 4).getInt();
+    }
+
+    int getPort() {
+        return ByteBuffer.wrap(key, 30, 4).getInt();
+    }
+
+    int getStreamId() {
+        return ByteBuffer.wrap(key, 34, 4).getInt();
+    }
+
+    @Override
+    public String toString() {
+        return "[0x" + DatatypeConverter.printHexBinary(key) + "]";
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbMetricsWriter.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbMetricsWriter.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbMetricsWriter.java
new file mode 100644
index 0000000..a050a76
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbMetricsWriter.java
@@ -0,0 +1,320 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import com.codahale.metrics.Meter;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.http.annotation.NotThreadSafe;
+import org.apache.storm.metricstore.AggLevel;
+import org.apache.storm.metricstore.Metric;
+import org.apache.storm.metricstore.MetricException;
+import org.rocksdb.FlushOptions;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class designed to perform all metrics inserts into RocksDB.  Metrics are processed from a blocking queue.  Inserts
+ * to RocksDB are done using a single thread to simplify design (such as looking up existing metric data for aggregation,
+ * and fetching/evicting metadata from the cache).
+ * </P>
+ * A writable LRU StringMetadataCache is used to minimize looking up metadata string Ids.  As entries are added to the full cache, older
+ * entries are evicted from the cache and need to be written to the database.  This happens as the handleEvictedMetadata()
+ * method callback.
+ * </P>
+ * The following issues would need to be addressed to implement a multithreaded metrics writer:
+ * <ul>
+ *     <li>Generation of unique unused IDs for new metadata strings needs to be thread safe.</li>
+ *     <li>Ensuring newly created metadata strings are seen by all threads.</li>
+ *     <li>Maintaining a properly cached state of metadata for multiple writers.  The current LRU cache
+ *     evicts data as new metadata is added.</li>
+ *     <li>Processing the aggregation of a metric requires fetching and updating previous aggregates.  A multithreaded
+ *     design would need to ensure two metrics were not updating an aggregated metric at the same time.</li>
+ *     <li>Investigate performance of multiple threads inserting into RocksDB versus a single ordered insert.</li>
+ * </ul>
+ */
+@NotThreadSafe
+public class RocksDbMetricsWriter implements Runnable, AutoCloseable {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDbMetricsWriter.class);
+    private RocksDbStore store;
+    private BlockingQueue queue;
+    private WritableStringMetadataCache stringMetadataCache;
+    private Set<Integer> unusedIds = new HashSet<>();
+    private TreeMap<RocksDbKey, RocksDbValue> insertBatch = new TreeMap<>(); // RocksDB should insert in sorted key order
+    private WriteOptions writeOpts = new WriteOptions();
+    private volatile boolean shutdown = false;
+    private Meter failureMeter;
+    private ArrayList<AggLevel> aggBuckets = new ArrayList<>();
+
+    /**
+     * Constructor for the RocksDbMetricsWriter.
+     *
+     * @param store   The RocksDB store
+     * @param queue   The queue to receive metrics for insertion
+     */
+    RocksDbMetricsWriter(RocksDbStore store, BlockingQueue queue, Meter failureMeter)  {
+        this.store = store;
+        this.queue = queue;
+        this.failureMeter = failureMeter;
+
+        aggBuckets.add(AggLevel.AGG_LEVEL_1_MIN);
+        aggBuckets.add(AggLevel.AGG_LEVEL_10_MIN);
+        aggBuckets.add(AggLevel.AGG_LEVEL_60_MIN);
+    }
+
+    /**
+     * Init routine called once the Metadata cache has been created.
+     *
+     * @throws MetricException  on cache error
+     */
+    void init() throws MetricException {
+        this.stringMetadataCache = StringMetadataCache.getWritableStringMetadataCache();
+    }
+
+    /**
+     * Run routine to wait for metrics on a queue and insert into RocksDB.
+     */
+    @Override
+    public void run() {
+        while (!shutdown) {
+            try {
+                Metric m = (Metric) queue.take();
+                processInsert(m);
+            } catch (Exception e) {
+                LOG.error("Failed to insert metric", e);
+                if (this.failureMeter != null) {
+                    this.failureMeter.mark();
+                }
+            }
+        }
+    }
+
+    /**
+     * Performs the actual metric insert, and aggregates over all bucket times.
+     *
+     * @param metric  Metric to store
+     * @throws MetricException  if database write fails
+     */
+    private void processInsert(Metric metric) throws MetricException {
+
+        // convert all strings to numeric Ids for the metric key and add to the metadata cache
+        long metricTimestamp = metric.getTimestamp();
+        Integer topologyId = storeMetadataString(KeyType.TOPOLOGY_STRING, metric.getTopologyId(), metricTimestamp);
+        Integer metricId = storeMetadataString(KeyType.METRIC_STRING, metric.getMetricName(), metricTimestamp);
+        Integer componentId = storeMetadataString(KeyType.COMPONENT_STRING, metric.getComponentId(), metricTimestamp);
+        Integer executorId = storeMetadataString(KeyType.EXEC_ID_STRING, metric.getExecutorId(), metricTimestamp);
+        Integer hostId = storeMetadataString(KeyType.HOST_STRING, metric.getHostname(), metricTimestamp);
+        Integer streamId = storeMetadataString(KeyType.STREAM_ID_STRING, metric.getStreamId(), metricTimestamp);
+
+        RocksDbKey key = RocksDbKey.createMetricKey(AggLevel.AGG_LEVEL_NONE, topologyId, metric.getTimestamp(), metricId,
+                componentId, executorId, hostId, metric.getPort(), streamId);
+
+        // save metric key/value to be batched
+        RocksDbValue value = new RocksDbValue(metric);
+        insertBatch.put(key, value);
+
+        // Aggregate matching metrics over bucket timeframes.
+        // We'll process starting with the longest bucket.  If the metric for this does not exist, we don't have to
+        // search for the remaining bucket metrics.
+        ListIterator li = aggBuckets.listIterator(aggBuckets.size());
+        boolean populate = true;
+        while (li.hasPrevious()) {
+            AggLevel bucket = (AggLevel)li.previous();
+            Metric aggMetric = new Metric(metric);
+            aggMetric.setAggLevel(bucket);
+
+            long msToBucket = 1000L * 60L * bucket.getValue();
+            long roundedToBucket = msToBucket * (metric.getTimestamp() / msToBucket);
+            aggMetric.setTimestamp(roundedToBucket);
+
+            RocksDbKey aggKey = RocksDbKey.createMetricKey(bucket, topologyId, aggMetric.getTimestamp(), metricId,
+                    componentId, executorId, hostId, aggMetric.getPort(), streamId);
+
+            if (populate) {
+                // retrieve any existing aggregation matching this one and update the values
+                if (store.populateFromKey(aggKey, aggMetric)) {
+                    aggMetric.addValue(metric.getValue());
+                } else {
+                    // aggregating metric did not exist, don't look for further ones with smaller timestamps
+                    populate = false;
+                }
+            }
+
+            // save metric key/value to be batched
+            RocksDbValue aggVal = new RocksDbValue(aggMetric);
+            insertBatch.put(aggKey, aggVal);
+        }
+
+        processBatchInsert(insertBatch);
+
+        insertBatch.clear();
+    }
+
+    // converts a metadata string into a unique integer.  Updates the timestamp of the string
+    // so we can track when it was last used for later deletion on database cleanup.
+    private int storeMetadataString(KeyType type, String s, long metricTimestamp) throws MetricException {
+        if (s == null) {
+            throw new MetricException("No string for metric metadata string type " + type);
+        }
+
+        // attempt to find it in the string cache
+        StringMetadata stringMetadata = stringMetadataCache.get(s);
+        if (stringMetadata != null) {
+            // make sure the timestamp on the metadata has the latest time
+            stringMetadata.update(metricTimestamp, type);
+            return stringMetadata.getStringId();
+        }
+
+        // attempt to find the string in the database
+        stringMetadata = store.rocksDbGetStringMetadata(type, s);
+        if (stringMetadata != null) {
+            // update to the latest timestamp and add to the string cache
+            stringMetadata.update(metricTimestamp, type);
+            stringMetadataCache.put(s, stringMetadata, false);
+            return stringMetadata.getStringId();
+        }
+
+        // string does not exist, create using an unique string id and add to cache
+        if (LOG.isDebugEnabled()) {
+            LOG.debug(type + "." + s + " does not exist in cache or database");
+        }
+        int stringId = getUniqueMetadataStringId();
+        stringMetadata = new StringMetadata(type, stringId, metricTimestamp);
+        stringMetadataCache.put(s, stringMetadata, true);
+
+        return stringMetadata.getStringId();
+    }
+
+    // get a currently unused unique string id
+    private int getUniqueMetadataStringId() throws MetricException {
+        generateUniqueStringIds();
+        int id = unusedIds.iterator().next();
+        unusedIds.remove(id);
+        return  id;
+    }
+
+    // guarantees a list of unused string Ids exists.  Once the list is empty, creates a new list
+    // by generating a list of random numbers and removing the ones that already are in use.
+    private void generateUniqueStringIds() throws MetricException {
+        int attempts = 0;
+        while (unusedIds.isEmpty()) {
+            attempts++;
+            if (attempts > 100) {
+                String message = "Failed to generate unique ids";
+                LOG.error(message);
+                throw new MetricException(message);
+            }
+            for (int i = 0; i < 600; i++) {
+                int n = ThreadLocalRandom.current().nextInt();
+                if (n == RocksDbStore.INVALID_METADATA_STRING_ID) {
+                    continue;
+                }
+                // remove any entries in the cache
+                if (stringMetadataCache.contains(n)) {
+                    continue;
+                }
+                unusedIds.add(n);
+            }
+            // now scan all metadata and remove any matching string Ids from this list
+            RocksDbKey firstPrefix = RocksDbKey.getPrefix(KeyType.METADATA_STRING_START);
+            RocksDbKey lastPrefix = RocksDbKey.getPrefix(KeyType.METADATA_STRING_END);
+            store.scanRange(firstPrefix, lastPrefix, (key, value) -> {
+                unusedIds.remove(key.getMetadataStringId());
+                return true; // process all metadata
+            });
+        }
+    }
+
+    // writes multiple metric values into the database as a batch operation.  The tree map keeps the keys sorted
+    // for faster insertion to RocksDB.
+    private void processBatchInsert(TreeMap<RocksDbKey, RocksDbValue> batchMap) throws MetricException {
+        try (WriteBatch writeBatch = new WriteBatch()) {
+            // take the batched metric data and write to the database
+            for (RocksDbKey k : batchMap.keySet()) {
+                RocksDbValue v = batchMap.get(k);
+                writeBatch.put(k.getRaw(), v.getRaw());
+            }
+            store.db.write(writeOpts, writeBatch);
+        } catch (Exception e) {
+            String message = "Failed to store data to RocksDB";
+            LOG.error(message, e);
+            throw new MetricException(message, e);
+        }
+    }
+
+    // evicted metadata needs to be stored immediately.  Metadata lookups count on it being in the cache
+    // or database.
+    void handleEvictedMetadata(RocksDbKey key, RocksDbValue val) {
+        try {
+            store.db.put(key.getRaw(), val.getRaw());
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    boolean isShutdown() {
+        return this.shutdown;
+    }
+
+    @Override
+    public void close() {
+        this.shutdown = true;
+
+        // get all metadata from the cache to put into the database
+        TreeMap<RocksDbKey, RocksDbValue> batchMap = new TreeMap<>();  // use a new map to prevent threading issues with writer thread
+        for (Map.Entry entry : stringMetadataCache.entrySet()) {
+            String metadataString = (String)entry.getKey();
+            StringMetadata val = (StringMetadata)entry.getValue();
+            RocksDbValue rval = new RocksDbValue(val.getLastTimestamp(), metadataString);
+
+            for (KeyType type : val.getMetadataTypes()) {   // save the metadata for all types of strings it matches
+                RocksDbKey rkey = new RocksDbKey(type, val.getStringId());
+                batchMap.put(rkey, rval);
+            }
+        }
+
+        try {
+            processBatchInsert(batchMap);
+        } catch (MetricException e) {
+            LOG.error("Failed to insert all metadata", e);
+        }
+
+        // flush db to disk
+        try (FlushOptions flushOps = new FlushOptions()) {
+            flushOps.setWaitForFlush(true);
+            store.db.flush(flushOps);
+        } catch (RocksDBException e) {
+            LOG.error("Failed ot flush RocksDB", e);
+            if (this.failureMeter != null) {
+                this.failureMeter.mark();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
new file mode 100644
index 0000000..2f44aff
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
@@ -0,0 +1,639 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import com.codahale.metrics.Meter;
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.metricstore.AggLevel;
+import org.apache.storm.metricstore.FilterOptions;
+import org.apache.storm.metricstore.Metric;
+import org.apache.storm.metricstore.MetricException;
+import org.apache.storm.metricstore.MetricStore;
+import org.apache.storm.utils.ObjectReader;
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.IndexType;
+import org.rocksdb.Options;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.RocksIterator;
+import org.rocksdb.WriteBatch;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class RocksDbStore implements MetricStore, AutoCloseable {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDbStore.class);
+    private static final int MAX_QUEUE_CAPACITY = 4000;
+    static final int INVALID_METADATA_STRING_ID = 0;
+    RocksDB db;
+    private ReadOnlyStringMetadataCache readOnlyStringMetadataCache = null;
+    private BlockingQueue queue = new LinkedBlockingQueue(MAX_QUEUE_CAPACITY);
+    private RocksDbMetricsWriter metricsWriter = null;
+    private MetricsCleaner metricsCleaner = null;
+    private Meter failureMeter = null;
+
+    interface RocksDbScanCallback {
+        boolean cb(RocksDbKey key, RocksDbValue val);  // return false to stop scan
+    }
+
+    /**
+     * Create metric store instance using the configurations provided via the config map.
+     *
+     * @param config Storm config map
+     * @throws MetricException on preparation error
+     */
+    public void prepare(Map config) throws MetricException {
+        validateConfig(config);
+
+        this.failureMeter = StormMetricsRegistry.registerMeter("RocksDB:metric-failures");
+
+        RocksDB.loadLibrary();
+        boolean createIfMissing = ObjectReader.getBoolean(config.get(DaemonConfig.STORM_ROCKSDB_CREATE_IF_MISSING), false);
+
+        try (Options options = new Options().setCreateIfMissing(createIfMissing)) {
+            // use the hash index for prefix searches
+            BlockBasedTableConfig tfc = new BlockBasedTableConfig();
+            tfc.setIndexType(IndexType.kHashSearch);
+            options.setTableFormatConfig(tfc);
+            options.useCappedPrefixExtractor(RocksDbKey.KEY_SIZE);
+
+            String path = getRocksDbAbsoluteDir(config);
+            LOG.info("Opening RocksDB from {}", path);
+            db = RocksDB.open(options, path);
+        } catch (RocksDBException e) {
+            String message = "Error opening RockDB database";
+            LOG.error(message, e);
+            throw new MetricException(message, e);
+        }
+
+        // create thread to delete old metrics and metadata
+        Integer retentionHours = Integer.parseInt(config.get(DaemonConfig.STORM_ROCKSDB_METRIC_RETENTION_HOURS).toString());
+        Integer deletionPeriod = 0;
+        if (config.containsKey(DaemonConfig.STORM_ROCKSDB_METRIC_DELETION_PERIOD_HOURS)) {
+            deletionPeriod = Integer.parseInt(config.get(DaemonConfig.STORM_ROCKSDB_METRIC_DELETION_PERIOD_HOURS).toString());
+        }
+        metricsCleaner = new MetricsCleaner(this, retentionHours, deletionPeriod, failureMeter);
+
+        // create thread to process insertion of all metrics
+        metricsWriter = new RocksDbMetricsWriter(this, this.queue, this.failureMeter);
+
+        int cacheCapacity = Integer.parseInt(config.get(DaemonConfig.STORM_ROCKSDB_METADATA_STRING_CACHE_CAPACITY).toString());
+        StringMetadataCache.init(metricsWriter, cacheCapacity);
+        readOnlyStringMetadataCache = StringMetadataCache.getReadOnlyStringMetadataCache();
+        metricsWriter.init(); // init the writer once the cache is setup
+
+        // start threads after metadata cache created
+        Thread thread = new Thread(metricsCleaner, "RocksDbMetricsCleaner");
+        thread.setDaemon(true);
+        thread.start();
+
+        thread = new Thread(metricsWriter, "RocksDbMetricsWriter");
+        thread.setDaemon(true);
+        thread.start();
+    }
+
+    /**
+     * Implements configuration validation of Metrics Store, validates storm configuration for Metrics Store.
+     *
+     * @param config Storm config to specify which store type, location of store and creation policy
+     * @throws MetricException if there is a missing required configuration or if the store does not exist but
+     *                         the config specifies not to create the store
+     */
+    private void validateConfig(Map config) throws MetricException {
+        if (!(config.containsKey(DaemonConfig.STORM_ROCKSDB_LOCATION))) {
+            throw new MetricException("Not a vaild RocksDB configuration - Missing store location " + DaemonConfig.STORM_ROCKSDB_LOCATION);
+        }
+
+        if (!(config.containsKey(DaemonConfig.STORM_ROCKSDB_CREATE_IF_MISSING))) {
+            throw new MetricException("Not a vaild RocksDB configuration - Does not specify creation policy "
+                    + DaemonConfig.STORM_ROCKSDB_CREATE_IF_MISSING);
+        }
+
+        // validate path defined
+        String storePath = getRocksDbAbsoluteDir(config);
+
+        boolean createIfMissing = ObjectReader.getBoolean(config.get(DaemonConfig.STORM_ROCKSDB_CREATE_IF_MISSING), false);
+        if (!createIfMissing) {
+            if (!(new File(storePath).exists())) {
+                throw new MetricException("Configuration specifies not to create a store but no store currently exists at " + storePath);
+            }
+        }
+
+        if (!(config.containsKey(DaemonConfig.STORM_ROCKSDB_METADATA_STRING_CACHE_CAPACITY))) {
+            throw new MetricException("Not a valid RocksDB configuration - Missing metadata string cache size "
+                    + DaemonConfig.STORM_ROCKSDB_METADATA_STRING_CACHE_CAPACITY);
+        }
+
+        if (!config.containsKey(DaemonConfig.STORM_ROCKSDB_METRIC_RETENTION_HOURS)) {
+            throw new MetricException("Not a valid RocksDB configuration - Missing metric retention "
+                    + DaemonConfig.STORM_ROCKSDB_METRIC_RETENTION_HOURS);
+        }
+    }
+
+    private String getRocksDbAbsoluteDir(Map conf) throws MetricException {
+        String storePath = (String)conf.get(DaemonConfig.STORM_ROCKSDB_LOCATION);
+        if (storePath == null) {
+            throw new MetricException("Not a vaild RocksDB configuration - Missing store location " + DaemonConfig.STORM_ROCKSDB_LOCATION);
+        } else {
+            if (new File(storePath).isAbsolute()) {
+                return storePath;
+            } else {
+                String stormHome = System.getProperty("storm.home");
+                if (stormHome == null) {
+                    throw new MetricException("storm.home not set");
+                }
+                return (stormHome + File.separator + storePath);
+            }
+        }
+    }
+
+    /**
+     * Stores metrics in the store.
+     *
+     * @param metric  Metric to store
+     * @throws MetricException  if database write fails
+     */
+    public void insert(Metric metric) throws MetricException {
+        try {
+            // don't bother blocking on a full queue, just drop metrics in case we can't keep up
+            if (queue.remainingCapacity() <= 0) {
+                LOG.info("Metrics q full, dropping metric");
+                return;
+            }
+            queue.put(metric);
+        } catch (Exception e) {
+            String message = "Failed to insert metric";
+            LOG.error(message, e);
+            if (this.failureMeter != null) {
+                this.failureMeter.mark();
+            }
+            throw new MetricException(message, e);
+        }
+    }
+
+    /**
+     * Fill out the numeric values for a metric.
+     *
+     * @param metric  Metric to populate
+     * @return   true if the metric was populated, false otherwise
+     * @throws MetricException  if read from database fails
+     */
+    @Override
+    public boolean populateValue(Metric metric) throws MetricException {
+        Map<String, Integer> localLookupCache = new HashMap<>(6);
+
+        int topologyId = lookupMetadataString(KeyType.TOPOLOGY_STRING, metric.getTopologyId(), localLookupCache);
+        if (INVALID_METADATA_STRING_ID == topologyId) {
+            return false;
+        }
+        int metricId = lookupMetadataString(KeyType.METRIC_STRING, metric.getMetricName(), localLookupCache);
+        if (INVALID_METADATA_STRING_ID == metricId) {
+            return false;
+        }
+        int componentId = lookupMetadataString(KeyType.COMPONENT_STRING, metric.getComponentId(), localLookupCache);
+        if (INVALID_METADATA_STRING_ID == componentId) {
+            return false;
+        }
+        int executorId = lookupMetadataString(KeyType.EXEC_ID_STRING, metric.getExecutorId(), localLookupCache);
+        if (INVALID_METADATA_STRING_ID == executorId) {
+            return false;
+        }
+        int hostId = lookupMetadataString(KeyType.HOST_STRING, metric.getHostname(), localLookupCache);
+        if (INVALID_METADATA_STRING_ID == hostId) {
+            return false;
+        }
+        int streamId = lookupMetadataString(KeyType.STREAM_ID_STRING, metric.getStreamId(), localLookupCache);
+        if (INVALID_METADATA_STRING_ID == streamId) {
+            return false;
+        }
+
+        RocksDbKey key = RocksDbKey.createMetricKey(metric.getAggLevel(), topologyId, metric.getTimestamp(), metricId,
+                componentId, executorId, hostId, metric.getPort(), streamId);
+
+        return populateFromKey(key, metric);
+    }
+
+    // populate metric values using the provided key
+    boolean populateFromKey(RocksDbKey key, Metric metric) throws MetricException {
+        try {
+            byte[] value = db.get(key.getRaw());
+            if (value == null) {
+                return false;
+            }
+            RocksDbValue rdbValue = new RocksDbValue(value);
+            rdbValue.populateMetric(metric);
+        } catch (Exception e) {
+            String message = "Failed to populate metric";
+            LOG.error(message, e);
+            if (this.failureMeter != null) {
+                this.failureMeter.mark();
+            }
+            throw new MetricException(message, e);
+        }
+        return true;
+    }
+
+    // attempts to lookup the unique Id for a string that may not exist yet.  Returns INVALID_METADATA_STRING_ID
+    // if it does not exist.
+    private int lookupMetadataString(KeyType type, String s, Map<String, Integer> lookupCache) throws MetricException {
+        if (s == null) {
+            if (this.failureMeter != null) {
+                this.failureMeter.mark();
+            }
+            throw new MetricException("No string for metric metadata string type " + type);
+        }
+
+        // attempt to find it in the string cache, this will update the LRU
+        StringMetadata stringMetadata = readOnlyStringMetadataCache.get(s);
+        if (stringMetadata != null) {
+            return stringMetadata.getStringId();
+        }
+
+        // attempt to find it in callers cache
+        Integer id = lookupCache.get(s);
+        if (id != null) {
+            return id;
+        }
+
+        // attempt to find the string in the database
+        stringMetadata = rocksDbGetStringMetadata(type, s);
+        if (stringMetadata != null) {
+            id = stringMetadata.getStringId();
+
+            // add to the callers cache.  We can't add it to the stringMetadataCache, since that could cause an eviction
+            // database write, which we want to only occur from the inserting DB thread.
+            lookupCache.put(s, id);
+
+            return id;
+        }
+
+        // string does not exist
+        return INVALID_METADATA_STRING_ID;
+    }
+
+    // scans the database to look for a metadata string and returns the metadata info
+    StringMetadata rocksDbGetStringMetadata(KeyType type, String s) {
+        RocksDbKey firstKey = RocksDbKey.getInitialKey(type);
+        RocksDbKey lastKey = RocksDbKey.getLastKey(type);
+        final AtomicReference<StringMetadata> reference = new AtomicReference<>();
+        scanRange(firstKey, lastKey, (key, value) -> {
+            if (s.equals(value.getMetdataString())) {
+                reference.set(value.getStringMetadata(key));
+                return false;
+            } else {
+                return true;  // haven't found string, keep searching
+            }
+        });
+        return reference.get();
+    }
+
+    // scans from key start to the key before end, calling back until callback indicates not to process further
+    void scanRange(RocksDbKey start, RocksDbKey end, RocksDbScanCallback fn) {
+        try (ReadOptions ro = new ReadOptions()) {
+            ro.setTotalOrderSeek(true);
+            RocksIterator iterator = db.newIterator(ro);
+            for (iterator.seek(start.getRaw()); iterator.isValid(); iterator.next()) {
+                RocksDbKey key = new RocksDbKey(iterator.key());
+                if (key.compareTo(end) >= 0) { // past limit, quit
+                    return;
+                }
+
+                RocksDbValue val = new RocksDbValue(iterator.value());
+                if (!fn.cb(key, val)) {
+                    // if cb returns false, we are done with this section of rows
+                    return;
+                }
+            }
+        }
+    }
+
+    /**
+     * Shutdown the store.
+     */
+    @Override
+    public void close() {
+        metricsWriter.close();
+        metricsCleaner.close();
+    }
+
+    /**
+     *  Scans all metrics in the store and returns the ones matching the specified filtering options.
+     *  Callback returns Metric class results.
+     *
+     * @param filter   options to filter by
+     * @param scanCallback  callback for each Metric found
+     * @throws MetricException  on error
+     */
+    public void scan(FilterOptions filter, ScanCallback scanCallback) throws MetricException {
+        scanInternal(filter, scanCallback, null);
+    }
+
+    /**
+     *  Scans all metrics in the store and returns the ones matching the specified filtering options.
+     *  Callback returns raw key/value data.
+     *
+     * @param filter   options to filter by
+     * @param rawCallback  callback for each Metric found
+     * @throws MetricException  on error
+     */
+    private void scanRaw(FilterOptions filter, RocksDbScanCallback rawCallback) throws MetricException {
+        scanInternal(filter, null, rawCallback);
+    }
+
+    // perform a scan given filter options, and return results in either Metric or raw data.
+    private void scanInternal(FilterOptions filter, ScanCallback scanCallback, RocksDbScanCallback rawCallback) throws MetricException {
+
+        Map<String, Integer> stringToIdCache = new HashMap<>();
+        Map<Integer, String> idToStringCache = new HashMap<>();
+
+        int startTopologyId = 0;
+        int endTopologyId = 0xFFFFFFFF;
+        String filterTopologyId = filter.getTopologyId();
+        if (filterTopologyId != null) {
+            int topologyId = lookupMetadataString(KeyType.TOPOLOGY_STRING, filterTopologyId, stringToIdCache);
+            if (INVALID_METADATA_STRING_ID == topologyId) {
+                return;  // string does not exist in database
+            }
+            startTopologyId = topologyId;
+            endTopologyId = topologyId;
+        }
+
+        long startTime = filter.getStartTime();
+        long endTime = filter.getEndTime();
+
+        int startMetricId = 0;
+        int endMetricId = 0xFFFFFFFF;
+        String filterMetricName = filter.getMetricName();
+        if (filterMetricName != null) {
+            int metricId = lookupMetadataString(KeyType.METRIC_STRING, filterMetricName, stringToIdCache);
+            if (INVALID_METADATA_STRING_ID == metricId) {
+                return;  // string does not exist in database
+            }
+            startMetricId = metricId;
+            endMetricId = metricId;
+        }
+
+        int startComponentId = 0;
+        int endComponentId = 0xFFFFFFFF;
+        String filterComponentId = filter.getComponentId();
+        if (filterComponentId != null) {
+            int componentId = lookupMetadataString(KeyType.COMPONENT_STRING, filterComponentId, stringToIdCache);
+            if (INVALID_METADATA_STRING_ID == componentId) {
+                return;  // string does not exist in database
+            }
+            startComponentId = componentId;
+            endComponentId = componentId;
+        }
+
+        int startExecutorId = 0;
+        int endExecutorId = 0xFFFFFFFF;
+        String filterExecutorName = filter.getExecutorId();
+        if (filterExecutorName != null) {
+            int executorId = lookupMetadataString(KeyType.EXEC_ID_STRING, filterExecutorName, stringToIdCache);
+            if (INVALID_METADATA_STRING_ID == executorId) {
+                return;  // string does not exist in database
+            }
+            startExecutorId = executorId;
+            endExecutorId = executorId;
+        }
+
+        int startHostId = 0;
+        int endHostId = 0xFFFFFFFF;
+        String filterHostId = filter.getHostId();
+        if (filterHostId != null) {
+            int hostId = lookupMetadataString(KeyType.HOST_STRING, filterHostId, stringToIdCache);
+            if (INVALID_METADATA_STRING_ID == hostId) {
+                return;  // string does not exist in database
+            }
+            startHostId = hostId;
+            endHostId = hostId;
+        }
+
+        int startPort = 0;
+        int endPort = 0xFFFFFFFF;
+        Integer filterPort = filter.getPort();
+        if (filterPort != null) {
+            startPort = filterPort;
+            endPort = filterPort;
+        }
+
+        int startStreamId = 0;
+        int endStreamId = 0xFFFFFFFF;
+        String filterStreamId = filter.getStreamId();
+        if (filterStreamId != null) {
+            int streamId = lookupMetadataString(KeyType.HOST_STRING, filterStreamId, stringToIdCache);
+            if (INVALID_METADATA_STRING_ID == streamId) {
+                return;  // string does not exist in database
+            }
+            startStreamId = streamId;
+            endStreamId = streamId;
+        }
+
+        ReadOptions ro = new ReadOptions();
+        ro.setTotalOrderSeek(true);
+
+        for (AggLevel aggLevel : filter.getAggLevels()) {
+
+            RocksDbKey startKey = RocksDbKey.createMetricKey(aggLevel, startTopologyId, startTime, startMetricId,
+                    startComponentId, startExecutorId, startHostId, startPort, startStreamId);
+            RocksDbKey endKey = RocksDbKey.createMetricKey(aggLevel, endTopologyId, endTime, endMetricId,
+                    endComponentId, endExecutorId, endHostId, endPort, endStreamId);
+
+            RocksIterator iterator = db.newIterator(ro);
+            for (iterator.seek(startKey.getRaw()); iterator.isValid(); iterator.next()) {
+                RocksDbKey key = new RocksDbKey(iterator.key());
+
+                if (key.compareTo(endKey) > 0) { // past limit, quit
+                    break;
+                }
+
+                if (startTopologyId != 0 && key.getTopologyId() != startTopologyId) {
+                    continue;
+                }
+
+                long timestamp = key.getTimestamp();
+                if (timestamp < startTime || timestamp > endTime) {
+                    continue;
+                }
+
+                if (startMetricId != 0 && key.getMetricId() != startMetricId) {
+                    continue;
+                }
+
+                if (startComponentId != 0 && key.getComponentId() != startComponentId) {
+                    continue;
+                }
+
+                if (startExecutorId != 0 && key.getExecutorId() != startExecutorId) {
+                    continue;
+                }
+
+                if (startHostId != 0 && key.getHostnameId() != startHostId) {
+                    continue;
+                }
+
+                if (startPort != 0 && key.getPort() != startPort) {
+                    continue;
+                }
+
+                if (startStreamId != 0 && key.getStreamId() != startStreamId) {
+                    continue;
+                }
+
+                RocksDbValue val = new RocksDbValue(iterator.value());
+
+                if (scanCallback != null) {
+                    try {
+                        // populate a metric
+                        String metricName = metadataIdToString(KeyType.METRIC_STRING, key.getMetricId(), idToStringCache);
+                        String topologyId = metadataIdToString(KeyType.TOPOLOGY_STRING, key.getTopologyId(), idToStringCache);
+                        String componentId = metadataIdToString(KeyType.COMPONENT_STRING, key.getComponentId(), idToStringCache);
+                        String executorId = metadataIdToString(KeyType.EXEC_ID_STRING, key.getExecutorId(), idToStringCache);
+                        String hostname = metadataIdToString(KeyType.HOST_STRING, key.getHostnameId(), idToStringCache);
+                        String streamId = metadataIdToString(KeyType.STREAM_ID_STRING, key.getStreamId(), idToStringCache);
+
+                        Metric metric = new Metric(metricName, timestamp, topologyId, 0.0, componentId, executorId, hostname,
+                                streamId, key.getPort(), aggLevel);
+
+                        val.populateMetric(metric);
+
+                        // callback to caller
+                        scanCallback.cb(metric);
+                    } catch (MetricException e) {
+                        LOG.warn("Failed to report found metric: {}", e.getMessage());
+                    }
+                } else {
+                    if (!rawCallback.cb(key, val)) {
+                        return;
+                    }
+                }
+            }
+            iterator.close();
+        }
+        ro.close();
+    }
+
+    // Finds the metadata string that matches the string Id and type provided.  The string should exist, as it is
+    // referenced from a metric.
+    private String metadataIdToString(KeyType type, int id, Map<Integer, String> lookupCache) throws MetricException {
+        String s = readOnlyStringMetadataCache.getMetadataString(id);
+        if (s != null) {
+            return s;
+        }
+        s = lookupCache.get(id);
+        if (s != null) {
+            return s;
+        }
+        // get from DB and add to lookup cache
+        RocksDbKey key = new RocksDbKey(type, id);
+        try {
+            byte[] value = db.get(key.getRaw());
+            if (value == null) {
+                throw new MetricException("Failed to find metadata string for id " + id + " of type " + type);
+            }
+            RocksDbValue rdbValue = new RocksDbValue(value);
+            s = rdbValue.getMetdataString();
+            lookupCache.put(id, s);
+            return s;
+        }  catch (RocksDBException e) {
+            if (this.failureMeter != null) {
+                this.failureMeter.mark();
+            }
+            throw new MetricException("Failed to get from RocksDb", e);
+        }
+    }
+
+    // deletes metrics matching the filter options
+    void deleteMetrics(FilterOptions filter) throws MetricException {
+        try (WriteBatch writeBatch = new WriteBatch();
+             WriteOptions writeOps = new WriteOptions()) {
+
+            scanRaw(filter, (RocksDbKey key, RocksDbValue value) -> {
+                writeBatch.remove(key.getRaw());
+                return true;
+            });
+
+            if (writeBatch.count() > 0) {
+                LOG.info("Deleting {} metrics", writeBatch.count());
+                try {
+                    db.write(writeOps, writeBatch);
+                } catch (Exception e) {
+                    String message = "Failed delete metrics";
+                    LOG.error(message, e);
+                    if (this.failureMeter != null) {
+                        this.failureMeter.mark();
+                    }
+                    throw new MetricException(message, e);
+                }
+            }
+        }
+    }
+
+    // deletes metadata strings before the provided timestamp
+    void deleteMetadataBefore(long firstValidTimestamp) throws MetricException {
+        if (firstValidTimestamp < 1L) {
+            if (this.failureMeter != null) {
+                this.failureMeter.mark();
+            }
+            throw new MetricException("Invalid timestamp for deleting metadata: " + firstValidTimestamp);
+        }
+
+        try (WriteBatch writeBatch = new WriteBatch();
+             WriteOptions writeOps = new WriteOptions()) {
+
+            // search all metadata strings
+            RocksDbKey topologyMetadataPrefix = RocksDbKey.getPrefix(KeyType.METADATA_STRING_START);
+            RocksDbKey lastPrefix = RocksDbKey.getPrefix(KeyType.METADATA_STRING_END);
+            scanRange(topologyMetadataPrefix, lastPrefix, (key, value) -> {
+                // we'll assume the metadata was recently used if still in the cache.
+                if (!readOnlyStringMetadataCache.contains(key.getMetadataStringId())) {
+                    if (value.getLastTimestamp() < firstValidTimestamp) {
+                        writeBatch.remove(key.getRaw());
+                    }
+                }
+                return true;
+            });
+
+            if (writeBatch.count() > 0) {
+                LOG.info("Deleting {} metadata strings", writeBatch.count());
+                try {
+                    db.write(writeOps, writeBatch);
+                } catch (Exception e) {
+                    String message = "Failed delete metadata strings";
+                    LOG.error(message, e);
+                    if (this.failureMeter != null) {
+                        this.failureMeter.mark();
+                    }
+                    throw new MetricException(message, e);
+                }
+            }
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbValue.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbValue.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbValue.java
new file mode 100644
index 0000000..58b2c76
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbValue.java
@@ -0,0 +1,144 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import java.nio.ByteBuffer;
+import org.apache.storm.metricstore.Metric;
+
+
+/**
+ * Class representing the data used as a Value in RocksDB.  Values can be used either for metadata or metrics.
+ *
+ * <p>Formats for Metadata String values are:
+ *
+ * <pre>
+ * Field             Size         Offset
+ *
+ * Version              1              0      The current metadata version - allows migrating if the format changes in the future
+ * Timestamp            8              1      The time when the metadata was last used by a metric.  Allows deleting of old metadata.
+ * Metadata String    any              9      The metadata string
+ *</pre>
+ *
+ * <p>Formats for Metric values are:
+ *
+ * <pre>
+ * Field             Size         Offset
+ *
+ * Version              1              0      The current metric version - allows migrating if the format changes in the future
+ * Value                8              1      The metric value
+ * Count                8              9      The metric count
+ * Min                  8             17      The minimum metric value
+ * Max                  8             25      The maximum metric value
+ * Sum                  8             33      The sum of the metric values
+ * </pre>
+ */
+
+class RocksDbValue {
+    private static int METRIC_VALUE_SIZE = 41;
+    private byte[] value;
+    private static final byte CURRENT_METADATA_VERSION = 0;
+    private static final byte CURRENT_METRIC_VERSION = 0;
+    private static int MIN_METADATA_VALUE_SIZE = 9;
+
+    /**
+     * Constructor from raw data.
+     *
+     * @param value  the raw bytes representing the key
+     */
+    RocksDbValue(byte[] value) {
+        this.value = value;
+    }
+
+    /**
+     * Constructor for a metadata string.
+     *
+     * @param lastTimestamp  the last timestamp when the string was used
+     * @param metadataString   the metadata string
+     */
+    RocksDbValue(long lastTimestamp, String metadataString) {
+        this.value = new byte[MIN_METADATA_VALUE_SIZE + metadataString.length()];
+        ByteBuffer bb = ByteBuffer.wrap(value);
+        bb.put(CURRENT_METADATA_VERSION);
+        bb.putLong(lastTimestamp);
+        bb.put(metadataString.getBytes());
+    }
+
+    /**
+     * Constructor for a metric.
+     *
+     * @param m  the metric to create a value from
+     */
+    RocksDbValue(Metric m) {
+        this.value = new byte[METRIC_VALUE_SIZE];
+        ByteBuffer bb = ByteBuffer.wrap(value);
+        bb.put(CURRENT_METRIC_VERSION);
+        bb.putDouble(m.getValue());
+        bb.putLong(m.getCount());
+        bb.putDouble(m.getMin());
+        bb.putDouble(m.getMax());
+        bb.putDouble(m.getSum());
+    }
+
+    /**
+     * Get the metadata string portion of the value.  Assumes the value is metadata.
+     *
+     * @return   the metadata string
+     */
+    String getMetdataString() {
+        if (this.value.length < MIN_METADATA_VALUE_SIZE) {
+            throw new RuntimeException("RocksDB value is too small to be a metadata string!");
+        }
+        return new String(this.value, 9, this.value.length - 9);
+    }
+
+    /**
+     * Gets StringMetadata associated with the key/value pair.
+     */
+    StringMetadata getStringMetadata(RocksDbKey key) {
+        return new StringMetadata(key.getType(), key.getMetadataStringId(), this.getLastTimestamp());
+    }
+
+    /**
+     * Gets the last time a metadata string was used.
+     */
+    long getLastTimestamp() {
+        return ByteBuffer.wrap(value, 1, 8).getLong();
+    }
+
+    /**
+     * get the raw value bytes
+     */
+    byte[] getRaw() {
+        return this.value;
+    }
+
+    /**
+     * populate metric values from the raw data.
+     */
+    void populateMetric(Metric metric) {
+        ByteBuffer bb = ByteBuffer.wrap(this.value, 0, METRIC_VALUE_SIZE);
+        bb.get();  // version
+        metric.setValue(bb.getDouble());
+        metric.setCount(bb.getLong());
+        metric.setMin(bb.getDouble());
+        metric.setMax(bb.getDouble());
+        metric.setSum(bb.getDouble());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/StringMetadata.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/StringMetadata.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/StringMetadata.java
new file mode 100644
index 0000000..6f54a58
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/StringMetadata.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Class that contains the information associated with a metadata string that remains cached in memory.
+ */
+class StringMetadata {
+    private List<KeyType> types = new ArrayList<>(1);  // its possible a string is used by multiple types of metadata strings
+    private int stringId;
+    private long lastTimestamp;
+
+    /**
+     * Constructor for StringMetadata.
+     *
+     * @param metadataType   the type of metadata string
+     * @param stringId       the unique id for the metadata string
+     * @param lastTimestamp   the timestamp when the metric used the metadata string
+     */
+    StringMetadata(KeyType metadataType, Integer stringId, Long lastTimestamp) {
+        this.types.add(metadataType);
+        this.stringId = stringId;
+        this.lastTimestamp = lastTimestamp;
+    }
+
+    int getStringId() {
+        return this.stringId;
+    }
+
+    long getLastTimestamp() {
+        return this.lastTimestamp;
+    }
+
+    List<KeyType> getMetadataTypes() {
+        return this.types;
+    }
+
+    private void addKeyType(KeyType type) {
+        if (!this.types.contains(type)) {
+            this.types.add(type);
+        }
+    }
+
+    /**
+     * Updates the timestamp of when a metadata string was last used.  Adds the type of the string if it is a new
+     * type.
+     *
+     * @param metricTimestamp   the timestamp of the metric using the metadata string
+     * @param type    the type of metadata string for the metric
+     */
+    void update(Long metricTimestamp, KeyType type) {
+        if (metricTimestamp > this.lastTimestamp) {
+            this.lastTimestamp = metricTimestamp;
+        }
+        addKeyType(type);
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/StringMetadataCache.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/StringMetadataCache.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/StringMetadataCache.java
new file mode 100644
index 0000000..7ce8435
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/StringMetadataCache.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.storm.metricstore.MetricException;
+import org.apache.storm.utils.LruMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to create a use a cache that stores Metadata string information in memory.  It allows searching for a
+ * Metadata string's unique id, or looking up the string by the unique id.  The StringMetadata is stored in an
+ * LRU map.  When an entry is added to the cache, an older entry may be evicted, which then needs to be
+ * immediately stored to the database to provide a consistent view of all the metadata strings.
+ *
+ * <p>All write operations adding metadata to RocksDB are done by a single thread (a RocksDbMetricsWriter),
+ * but multiple threads can read values from the cache. To clarify which permissions are accessible by various
+ * threads, the ReadOnlyStringMetadataCache and WritableStringMetadataCache are provided to be used.
+ */
+
+public class StringMetadataCache implements LruMap.CacheEvictionCallback<String, StringMetadata>,
+        WritableStringMetadataCache, ReadOnlyStringMetadataCache {
+    private static final Logger LOG = LoggerFactory.getLogger(StringMetadataCache.class);
+    private Map<String, StringMetadata> lruStringCache;
+    private Map<Integer, String> hashToString = new ConcurrentHashMap<>();
+    private RocksDbMetricsWriter dbWriter;
+    private static StringMetadataCache instance = null;
+
+    /**
+     * Initializes the cache instance.
+     *
+     * @param dbWriter   the RocksDB writer instance to handle writing evicted cache data
+     * @param capacity   the number of StringMetadata instances to hold in memory
+     * @throws MetricException   if creating multiple cache instances
+     */
+    static void init(RocksDbMetricsWriter dbWriter, int capacity) throws MetricException {
+        if (instance == null) {
+            instance = new StringMetadataCache(dbWriter, capacity);
+        } else {
+            throw new MetricException("StringMetadataCache already created");
+        }
+    }
+
+    /**
+     * Provides the WritableStringMetadataCache interface to the cache instance.
+     *
+     * @throws MetricException  if the cache instance was not created
+     */
+    static WritableStringMetadataCache getWritableStringMetadataCache() throws MetricException {
+        if (instance != null) {
+            return instance;
+        } else {
+            throw new MetricException("StringMetadataCache was not initialized");
+        }
+    }
+
+    /**
+     * Provides the ReadOnlyStringMetadataCache interface to the cache instance.
+     *
+     * @throws MetricException  if the cache instance was not created
+     */
+    static ReadOnlyStringMetadataCache getReadOnlyStringMetadataCache() throws MetricException {
+        if (instance != null) {
+            return instance;
+        } else {
+            throw new MetricException("StringMetadataCache was not initialized");
+        }
+    }
+
+    /**
+     * Constructor to create a cache.
+     *
+     * @param dbWriter   The rocks db writer instance the cache should use when evicting data
+     * @param capacity  The cache size
+     */
+    private StringMetadataCache(RocksDbMetricsWriter dbWriter, int capacity) {
+        lruStringCache = Collections.synchronizedMap(new LruMap<>(capacity, this));
+        this.dbWriter = dbWriter;
+    }
+
+    /**
+     * Get the string metadata from the cache.
+     *
+     * @param s   The string to look for
+     * @return   the metadata associated with the string or null if not found
+     */
+    public StringMetadata get(String s) {
+        return lruStringCache.get(s);
+    }
+
+    /**
+     * Add the string metadata to the cache.
+     *
+     * NOTE: this can cause data to be evicted from the cache when full.  When this occurs, the evictionCallback() method
+     * is called to store the metadata back into the RocksDB database.
+     *
+     * This method is only exposed to the WritableStringMetadataCache interface.
+     *
+     * @param s   The string to add
+     * @param stringMetadata  The string's metadata
+     * @param newEntry   Indicates the metadata is being used for the first time and should be written to RocksDB immediately
+     * @throws MetricException   when evicted data fails to save to the database or when the database is shutdown
+     */
+    public void put(String s, StringMetadata stringMetadata, boolean newEntry) throws MetricException {
+        if (dbWriter.isShutdown()) {
+            // another thread could be writing out the metadata cache to the database.
+            throw new MetricException("Shutting down");
+        }
+        try {
+            if (newEntry) {
+                writeMetadataToDisk(s, stringMetadata);
+            }
+            lruStringCache.put(s, stringMetadata);
+            hashToString.put(stringMetadata.getStringId(), s);
+        } catch (Exception e) { // catch any runtime exceptions caused by eviction
+            throw new MetricException("Failed to save string in metadata cache", e);
+        }
+    }
+
+    /**
+     * Callback when data is about to be removed from the cache.  This method then
+     * immediately writes the metadata to RocksDB.
+     *
+     * @param key   The evicted string
+     * @param val  The evicted string's metadata
+     * @throws RuntimeException   when evicted data fails to save to the database
+     */
+    public void evictionCallback(String key, StringMetadata val) {
+        writeMetadataToDisk(key, val);
+    }
+
+    private void writeMetadataToDisk(String key, StringMetadata val) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Writing {} to RocksDB", key);
+        }
+        // remove reverse lookup from map
+        hashToString.remove(val.getStringId());
+
+        // save the evicted key/value to the database immediately
+        RocksDbValue rval = new RocksDbValue(val.getLastTimestamp(), key);
+
+        for (KeyType type : val.getMetadataTypes()) { // save the metadata for all types of strings it matches
+            RocksDbKey rkey = new RocksDbKey(type, val.getStringId());
+            dbWriter.handleEvictedMetadata(rkey, rval);
+        }
+    }
+
+    /**
+     * Determines if a string Id is contained in the cache.
+     *
+     * @param stringId   The string Id to check
+     * @return   true if the Id is in the cache, false otherwise
+     */
+    public boolean contains(Integer stringId) {
+        return hashToString.containsKey(stringId);
+    }
+
+    /**
+     * Returns the string matching the string Id if in the cache.
+     *
+     * @param stringId   The string Id to check
+     * @return   the associated string if the Id is in the cache, null otherwise
+     */
+    public String getMetadataString(Integer stringId) {
+        return hashToString.get(stringId);
+    }
+
+    /**
+     * Get the map of the cache contents.  Provided to allow writing the data to RocksDB on shutdown.
+     *
+     * @return   the string metadata map entrySet
+     */
+    public Set<Map.Entry<String, StringMetadata>> entrySet() {
+        return lruStringCache.entrySet();
+    }
+
+    static void cleanUp() {
+        instance = null;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/WritableStringMetadataCache.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/WritableStringMetadataCache.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/WritableStringMetadataCache.java
new file mode 100644
index 0000000..2d4165f
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/WritableStringMetadataCache.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.storm.metricstore.rocksdb;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.http.annotation.NotThreadSafe;
+import org.apache.storm.metricstore.MetricException;
+
+/**
+ * The writable interface to a StringMetadataCache intended to be used by a single RocksDBMetricwWriter instance.
+ */
+@NotThreadSafe
+public interface WritableStringMetadataCache extends ReadOnlyStringMetadataCache {
+
+    /**
+     * Add the string metadata to the cache.
+     *
+     * * NOTE: this can cause data to be evicted from the cache when full.  When this occurs, the evictionCallback() method
+     * is called to store the metadata back into the RocksDB database.
+     *
+     * This method is only exposed to the WritableStringMetadataCache interface.
+     *
+     * @param s   The string to add
+     * @param stringMetadata  The string's metadata
+     * @param newEntry   Indicates the metadata is being used for the first time and should be written to RocksDB immediately
+     * @throws MetricException   when evicted data fails to save to the database or when the database is shutdown
+     */
+    void put(String s, StringMetadata stringMetadata, boolean newEntry) throws MetricException;
+
+    /**
+     * Get the map of the cache contents.  Provided to allow writing the data to RocksDB on shutdown.
+     *
+     * @return   the string metadata map entrySet
+     */
+    Set<Map.Entry<String, StringMetadata>> entrySet();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/main/java/org/apache/storm/utils/LruMap.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/utils/LruMap.java b/storm-server/src/main/java/org/apache/storm/utils/LruMap.java
new file mode 100644
index 0000000..3ed5d06
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/utils/LruMap.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.utils;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+public class LruMap<A, B> extends LinkedHashMap<A, B> {
+    private int maxSize;
+    private CacheEvictionCallback evCb = null;
+
+    public LruMap(int maxSize) {
+        super(maxSize + 1, 1.0f, true);
+        this.maxSize = maxSize;
+    }
+
+    /**
+     * Creates an LRU map that will call back before data is removed from the map.
+     *
+     * @param maxSize   max capacity for the map
+     * @param evictionCallback   callback to be called before removing data
+     */
+    public LruMap(int maxSize, CacheEvictionCallback evictionCallback) {
+        this(maxSize);
+        this.evCb = evictionCallback;
+    }
+    
+    @Override
+    protected boolean removeEldestEntry(final Map.Entry<A, B> eldest) {
+        boolean evict = size() > this.maxSize;
+        if (evict && this.evCb != null) {
+            this.evCb.evictionCallback(eldest.getKey(), eldest.getValue());
+        }
+        return evict;
+    }
+
+    public interface CacheEvictionCallback<K, V> {
+        void evictionCallback(K key, V val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbKeyTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbKeyTest.java b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbKeyTest.java
new file mode 100644
index 0000000..21d2377
--- /dev/null
+++ b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbKeyTest.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.storm.metricstore.rocksdb;
+
+import org.apache.storm.metricstore.AggLevel;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class RocksDbKeyTest {
+
+    @Test
+    public void testConstructors() {
+        byte[] raw = new byte[RocksDbKey.KEY_SIZE];
+        raw[0] = KeyType.COMPONENT_STRING.getValue();
+        raw[2] = 0x01;
+        raw[3] = 0x02;
+        raw[4] = 0x03;
+        raw[5] = 0x04;
+        RocksDbKey rawKey = new RocksDbKey(raw);
+
+        RocksDbKey metadataKey = new RocksDbKey(KeyType.COMPONENT_STRING, 0x01020304);
+        Assert.assertEquals(0, metadataKey.compareTo(rawKey));
+        Assert.assertEquals(KeyType.COMPONENT_STRING, metadataKey.getType());
+
+        metadataKey = new RocksDbKey(KeyType.TOPOLOGY_STRING, 0x01020304);
+        Assert.assertTrue(metadataKey.compareTo(rawKey) < 0);
+        Assert.assertEquals(KeyType.TOPOLOGY_STRING, metadataKey.getType());
+
+        metadataKey = new RocksDbKey(KeyType.COMPONENT_STRING, 0x01020305);
+        Assert.assertTrue(metadataKey.compareTo(rawKey) > 0);
+
+        Assert.assertEquals(0x01020304, rawKey.getTopologyId());
+        Assert.assertEquals(KeyType.COMPONENT_STRING, rawKey.getType());
+    }
+
+    @Test
+    public void testMetricKey() {
+        AggLevel aggLevel = AggLevel.AGG_LEVEL_10_MIN;
+        int topologyId = 0x45665;
+        long timestamp = System.currentTimeMillis();
+        int metricId = 0xF3916034;
+        int componentId = 0x82915031;
+        int executorId = 0x434738;
+        int hostId = 0x4348394;
+        int port = 3456;
+        int streamId = 0x84221956;
+        RocksDbKey key = RocksDbKey.createMetricKey(aggLevel, topologyId, timestamp, metricId,
+                componentId, executorId, hostId, port, streamId);
+        Assert.assertEquals(topologyId, key.getTopologyId());
+        Assert.assertEquals(timestamp, key.getTimestamp());
+        Assert.assertEquals(metricId, key.getMetricId());
+        Assert.assertEquals(componentId, key.getComponentId());
+        Assert.assertEquals(executorId, key.getExecutorId());
+        Assert.assertEquals(hostId, key.getHostnameId());
+        Assert.assertEquals(port, key.getPort());
+        Assert.assertEquals(streamId, key.getStreamId());
+    }
+}


[5/7] storm git commit: STORM-2887: store metrics into RocksDB

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java b/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java
index ea4dcc9..8b02548 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java
@@ -170,6 +170,8 @@ public class Nimbus {
 
     public List<OwnerResourceSummary> getOwnerResourceSummaries(String owner) throws AuthorizationException, org.apache.thrift.TException;
 
+    public void processWorkerMetrics(WorkerMetrics metrics) throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -266,6 +268,8 @@ public class Nimbus {
 
     public void getOwnerResourceSummaries(String owner, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void processWorkerMetrics(WorkerMetrics metrics, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -1522,6 +1526,26 @@ public class Nimbus {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getOwnerResourceSummaries failed: unknown result");
     }
 
+    public void processWorkerMetrics(WorkerMetrics metrics) throws org.apache.thrift.TException
+    {
+      send_processWorkerMetrics(metrics);
+      recv_processWorkerMetrics();
+    }
+
+    public void send_processWorkerMetrics(WorkerMetrics metrics) throws org.apache.thrift.TException
+    {
+      processWorkerMetrics_args args = new processWorkerMetrics_args();
+      args.set_metrics(metrics);
+      sendBase("processWorkerMetrics", args);
+    }
+
+    public void recv_processWorkerMetrics() throws org.apache.thrift.TException
+    {
+      processWorkerMetrics_result result = new processWorkerMetrics_result();
+      receiveBase(result, "processWorkerMetrics");
+      return;
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -3090,6 +3114,38 @@ public class Nimbus {
       }
     }
 
+    public void processWorkerMetrics(WorkerMetrics metrics, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      processWorkerMetrics_call method_call = new processWorkerMetrics_call(metrics, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class processWorkerMetrics_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private WorkerMetrics metrics;
+      public processWorkerMetrics_call(WorkerMetrics metrics, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.metrics = metrics;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("processWorkerMetrics", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        processWorkerMetrics_args args = new processWorkerMetrics_args();
+        args.set_metrics(metrics);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_processWorkerMetrics();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -3149,6 +3205,7 @@ public class Nimbus {
       processMap.put("getUserTopology", new getUserTopology());
       processMap.put("getTopologyHistory", new getTopologyHistory());
       processMap.put("getOwnerResourceSummaries", new getOwnerResourceSummaries());
+      processMap.put("processWorkerMetrics", new processWorkerMetrics());
       return processMap;
     }
 
@@ -4293,6 +4350,26 @@ public class Nimbus {
       }
     }
 
+    public static class processWorkerMetrics<I extends Iface> extends org.apache.thrift.ProcessFunction<I, processWorkerMetrics_args> {
+      public processWorkerMetrics() {
+        super("processWorkerMetrics");
+      }
+
+      public processWorkerMetrics_args getEmptyArgsInstance() {
+        return new processWorkerMetrics_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public processWorkerMetrics_result getResult(I iface, processWorkerMetrics_args args) throws org.apache.thrift.TException {
+        processWorkerMetrics_result result = new processWorkerMetrics_result();
+        iface.processWorkerMetrics(args.metrics);
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -4352,6 +4429,7 @@ public class Nimbus {
       processMap.put("getUserTopology", new getUserTopology());
       processMap.put("getTopologyHistory", new getTopologyHistory());
       processMap.put("getOwnerResourceSummaries", new getOwnerResourceSummaries());
+      processMap.put("processWorkerMetrics", new processWorkerMetrics());
       return processMap;
     }
 
@@ -7070,6 +7148,56 @@ public class Nimbus {
       }
     }
 
+    public static class processWorkerMetrics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, processWorkerMetrics_args, Void> {
+      public processWorkerMetrics() {
+        super("processWorkerMetrics");
+      }
+
+      public processWorkerMetrics_args getEmptyArgsInstance() {
+        return new processWorkerMetrics_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            processWorkerMetrics_result result = new processWorkerMetrics_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            processWorkerMetrics_result result = new processWorkerMetrics_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, processWorkerMetrics_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.processWorkerMetrics(args.metrics,resultHandler);
+      }
+    }
+
   }
 
   public static class submitTopology_args implements org.apache.thrift.TBase<submitTopology_args, submitTopology_args._Fields>, java.io.Serializable, Cloneable, Comparable<submitTopology_args>   {
@@ -18287,14 +18415,14 @@ public class Nimbus {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list866 = iprot.readListBegin();
-                  struct.success = new ArrayList<ProfileRequest>(_list866.size);
-                  ProfileRequest _elem867;
-                  for (int _i868 = 0; _i868 < _list866.size; ++_i868)
+                  org.apache.thrift.protocol.TList _list894 = iprot.readListBegin();
+                  struct.success = new ArrayList<ProfileRequest>(_list894.size);
+                  ProfileRequest _elem895;
+                  for (int _i896 = 0; _i896 < _list894.size; ++_i896)
                   {
-                    _elem867 = new ProfileRequest();
-                    _elem867.read(iprot);
-                    struct.success.add(_elem867);
+                    _elem895 = new ProfileRequest();
+                    _elem895.read(iprot);
+                    struct.success.add(_elem895);
                   }
                   iprot.readListEnd();
                 }
@@ -18320,9 +18448,9 @@ public class Nimbus {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (ProfileRequest _iter869 : struct.success)
+            for (ProfileRequest _iter897 : struct.success)
             {
-              _iter869.write(oprot);
+              _iter897.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -18353,9 +18481,9 @@ public class Nimbus {
         if (struct.is_set_success()) {
           {
             oprot.writeI32(struct.success.size());
-            for (ProfileRequest _iter870 : struct.success)
+            for (ProfileRequest _iter898 : struct.success)
             {
-              _iter870.write(oprot);
+              _iter898.write(oprot);
             }
           }
         }
@@ -18367,14 +18495,14 @@ public class Nimbus {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list871 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<ProfileRequest>(_list871.size);
-            ProfileRequest _elem872;
-            for (int _i873 = 0; _i873 < _list871.size; ++_i873)
+            org.apache.thrift.protocol.TList _list899 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<ProfileRequest>(_list899.size);
+            ProfileRequest _elem900;
+            for (int _i901 = 0; _i901 < _list899.size; ++_i901)
             {
-              _elem872 = new ProfileRequest();
-              _elem872.read(iprot);
-              struct.success.add(_elem872);
+              _elem900 = new ProfileRequest();
+              _elem900.read(iprot);
+              struct.success.add(_elem900);
             }
           }
           struct.set_success_isSet(true);
@@ -48137,14 +48265,14 @@ public class Nimbus {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list874 = iprot.readListBegin();
-                  struct.success = new ArrayList<OwnerResourceSummary>(_list874.size);
-                  OwnerResourceSummary _elem875;
-                  for (int _i876 = 0; _i876 < _list874.size; ++_i876)
+                  org.apache.thrift.protocol.TList _list902 = iprot.readListBegin();
+                  struct.success = new ArrayList<OwnerResourceSummary>(_list902.size);
+                  OwnerResourceSummary _elem903;
+                  for (int _i904 = 0; _i904 < _list902.size; ++_i904)
                   {
-                    _elem875 = new OwnerResourceSummary();
-                    _elem875.read(iprot);
-                    struct.success.add(_elem875);
+                    _elem903 = new OwnerResourceSummary();
+                    _elem903.read(iprot);
+                    struct.success.add(_elem903);
                   }
                   iprot.readListEnd();
                 }
@@ -48179,9 +48307,9 @@ public class Nimbus {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (OwnerResourceSummary _iter877 : struct.success)
+            for (OwnerResourceSummary _iter905 : struct.success)
             {
-              _iter877.write(oprot);
+              _iter905.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -48220,9 +48348,9 @@ public class Nimbus {
         if (struct.is_set_success()) {
           {
             oprot.writeI32(struct.success.size());
-            for (OwnerResourceSummary _iter878 : struct.success)
+            for (OwnerResourceSummary _iter906 : struct.success)
             {
-              _iter878.write(oprot);
+              _iter906.write(oprot);
             }
           }
         }
@@ -48237,14 +48365,14 @@ public class Nimbus {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list879 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<OwnerResourceSummary>(_list879.size);
-            OwnerResourceSummary _elem880;
-            for (int _i881 = 0; _i881 < _list879.size; ++_i881)
+            org.apache.thrift.protocol.TList _list907 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<OwnerResourceSummary>(_list907.size);
+            OwnerResourceSummary _elem908;
+            for (int _i909 = 0; _i909 < _list907.size; ++_i909)
             {
-              _elem880 = new OwnerResourceSummary();
-              _elem880.read(iprot);
-              struct.success.add(_elem880);
+              _elem908 = new OwnerResourceSummary();
+              _elem908.read(iprot);
+              struct.success.add(_elem908);
             }
           }
           struct.set_success_isSet(true);
@@ -48259,4 +48387,613 @@ public class Nimbus {
 
   }
 
+  public static class processWorkerMetrics_args implements org.apache.thrift.TBase<processWorkerMetrics_args, processWorkerMetrics_args._Fields>, java.io.Serializable, Cloneable, Comparable<processWorkerMetrics_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("processWorkerMetrics_args");
+
+    private static final org.apache.thrift.protocol.TField METRICS_FIELD_DESC = new org.apache.thrift.protocol.TField("metrics", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new processWorkerMetrics_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new processWorkerMetrics_argsTupleSchemeFactory());
+    }
+
+    private WorkerMetrics metrics; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      METRICS((short)1, "metrics");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // METRICS
+            return METRICS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.METRICS, new org.apache.thrift.meta_data.FieldMetaData("metrics", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerMetrics.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(processWorkerMetrics_args.class, metaDataMap);
+    }
+
+    public processWorkerMetrics_args() {
+    }
+
+    public processWorkerMetrics_args(
+      WorkerMetrics metrics)
+    {
+      this();
+      this.metrics = metrics;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public processWorkerMetrics_args(processWorkerMetrics_args other) {
+      if (other.is_set_metrics()) {
+        this.metrics = new WorkerMetrics(other.metrics);
+      }
+    }
+
+    public processWorkerMetrics_args deepCopy() {
+      return new processWorkerMetrics_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.metrics = null;
+    }
+
+    public WorkerMetrics get_metrics() {
+      return this.metrics;
+    }
+
+    public void set_metrics(WorkerMetrics metrics) {
+      this.metrics = metrics;
+    }
+
+    public void unset_metrics() {
+      this.metrics = null;
+    }
+
+    /** Returns true if field metrics is set (has been assigned a value) and false otherwise */
+    public boolean is_set_metrics() {
+      return this.metrics != null;
+    }
+
+    public void set_metrics_isSet(boolean value) {
+      if (!value) {
+        this.metrics = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case METRICS:
+        if (value == null) {
+          unset_metrics();
+        } else {
+          set_metrics((WorkerMetrics)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case METRICS:
+        return get_metrics();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case METRICS:
+        return is_set_metrics();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof processWorkerMetrics_args)
+        return this.equals((processWorkerMetrics_args)that);
+      return false;
+    }
+
+    public boolean equals(processWorkerMetrics_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_metrics = true && this.is_set_metrics();
+      boolean that_present_metrics = true && that.is_set_metrics();
+      if (this_present_metrics || that_present_metrics) {
+        if (!(this_present_metrics && that_present_metrics))
+          return false;
+        if (!this.metrics.equals(that.metrics))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_metrics = true && (is_set_metrics());
+      list.add(present_metrics);
+      if (present_metrics)
+        list.add(metrics);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(processWorkerMetrics_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_metrics()).compareTo(other.is_set_metrics());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_metrics()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metrics, other.metrics);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("processWorkerMetrics_args(");
+      boolean first = true;
+
+      sb.append("metrics:");
+      if (this.metrics == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.metrics);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (metrics != null) {
+        metrics.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class processWorkerMetrics_argsStandardSchemeFactory implements SchemeFactory {
+      public processWorkerMetrics_argsStandardScheme getScheme() {
+        return new processWorkerMetrics_argsStandardScheme();
+      }
+    }
+
+    private static class processWorkerMetrics_argsStandardScheme extends StandardScheme<processWorkerMetrics_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, processWorkerMetrics_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // METRICS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.metrics = new WorkerMetrics();
+                struct.metrics.read(iprot);
+                struct.set_metrics_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, processWorkerMetrics_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.metrics != null) {
+          oprot.writeFieldBegin(METRICS_FIELD_DESC);
+          struct.metrics.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class processWorkerMetrics_argsTupleSchemeFactory implements SchemeFactory {
+      public processWorkerMetrics_argsTupleScheme getScheme() {
+        return new processWorkerMetrics_argsTupleScheme();
+      }
+    }
+
+    private static class processWorkerMetrics_argsTupleScheme extends TupleScheme<processWorkerMetrics_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, processWorkerMetrics_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_metrics()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.is_set_metrics()) {
+          struct.metrics.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, processWorkerMetrics_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.metrics = new WorkerMetrics();
+          struct.metrics.read(iprot);
+          struct.set_metrics_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class processWorkerMetrics_result implements org.apache.thrift.TBase<processWorkerMetrics_result, processWorkerMetrics_result._Fields>, java.io.Serializable, Cloneable, Comparable<processWorkerMetrics_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("processWorkerMetrics_result");
+
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new processWorkerMetrics_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new processWorkerMetrics_resultTupleSchemeFactory());
+    }
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(processWorkerMetrics_result.class, metaDataMap);
+    }
+
+    public processWorkerMetrics_result() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public processWorkerMetrics_result(processWorkerMetrics_result other) {
+    }
+
+    public processWorkerMetrics_result deepCopy() {
+      return new processWorkerMetrics_result(this);
+    }
+
+    @Override
+    public void clear() {
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof processWorkerMetrics_result)
+        return this.equals((processWorkerMetrics_result)that);
+      return false;
+    }
+
+    public boolean equals(processWorkerMetrics_result that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(processWorkerMetrics_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("processWorkerMetrics_result(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class processWorkerMetrics_resultStandardSchemeFactory implements SchemeFactory {
+      public processWorkerMetrics_resultStandardScheme getScheme() {
+        return new processWorkerMetrics_resultStandardScheme();
+      }
+    }
+
+    private static class processWorkerMetrics_resultStandardScheme extends StandardScheme<processWorkerMetrics_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, processWorkerMetrics_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, processWorkerMetrics_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class processWorkerMetrics_resultTupleSchemeFactory implements SchemeFactory {
+      public processWorkerMetrics_resultTupleScheme getScheme() {
+        return new processWorkerMetrics_resultTupleScheme();
+      }
+    }
+
+    private static class processWorkerMetrics_resultTupleScheme extends TupleScheme<processWorkerMetrics_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, processWorkerMetrics_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, processWorkerMetrics_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+      }
+    }
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/StormBase.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/StormBase.java b/storm-client/src/jvm/org/apache/storm/generated/StormBase.java
index b1b205c..733da58 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/StormBase.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/StormBase.java
@@ -1252,15 +1252,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           case 4: // COMPONENT_EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map732 = iprot.readMapBegin();
-                struct.component_executors = new HashMap<String,Integer>(2*_map732.size);
-                String _key733;
-                int _val734;
-                for (int _i735 = 0; _i735 < _map732.size; ++_i735)
+                org.apache.thrift.protocol.TMap _map752 = iprot.readMapBegin();
+                struct.component_executors = new HashMap<String,Integer>(2*_map752.size);
+                String _key753;
+                int _val754;
+                for (int _i755 = 0; _i755 < _map752.size; ++_i755)
                 {
-                  _key733 = iprot.readString();
-                  _val734 = iprot.readI32();
-                  struct.component_executors.put(_key733, _val734);
+                  _key753 = iprot.readString();
+                  _val754 = iprot.readI32();
+                  struct.component_executors.put(_key753, _val754);
                 }
                 iprot.readMapEnd();
               }
@@ -1305,16 +1305,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           case 9: // COMPONENT_DEBUG
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map736 = iprot.readMapBegin();
-                struct.component_debug = new HashMap<String,DebugOptions>(2*_map736.size);
-                String _key737;
-                DebugOptions _val738;
-                for (int _i739 = 0; _i739 < _map736.size; ++_i739)
+                org.apache.thrift.protocol.TMap _map756 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map756.size);
+                String _key757;
+                DebugOptions _val758;
+                for (int _i759 = 0; _i759 < _map756.size; ++_i759)
                 {
-                  _key737 = iprot.readString();
-                  _val738 = new DebugOptions();
-                  _val738.read(iprot);
-                  struct.component_debug.put(_key737, _val738);
+                  _key757 = iprot.readString();
+                  _val758 = new DebugOptions();
+                  _val758.read(iprot);
+                  struct.component_debug.put(_key757, _val758);
                 }
                 iprot.readMapEnd();
               }
@@ -1370,10 +1370,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           oprot.writeFieldBegin(COMPONENT_EXECUTORS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.component_executors.size()));
-            for (Map.Entry<String, Integer> _iter740 : struct.component_executors.entrySet())
+            for (Map.Entry<String, Integer> _iter760 : struct.component_executors.entrySet())
             {
-              oprot.writeString(_iter740.getKey());
-              oprot.writeI32(_iter740.getValue());
+              oprot.writeString(_iter760.getKey());
+              oprot.writeI32(_iter760.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1411,10 +1411,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           oprot.writeFieldBegin(COMPONENT_DEBUG_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.component_debug.size()));
-            for (Map.Entry<String, DebugOptions> _iter741 : struct.component_debug.entrySet())
+            for (Map.Entry<String, DebugOptions> _iter761 : struct.component_debug.entrySet())
             {
-              oprot.writeString(_iter741.getKey());
-              _iter741.getValue().write(oprot);
+              oprot.writeString(_iter761.getKey());
+              _iter761.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1484,10 +1484,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_component_executors()) {
         {
           oprot.writeI32(struct.component_executors.size());
-          for (Map.Entry<String, Integer> _iter742 : struct.component_executors.entrySet())
+          for (Map.Entry<String, Integer> _iter762 : struct.component_executors.entrySet())
           {
-            oprot.writeString(_iter742.getKey());
-            oprot.writeI32(_iter742.getValue());
+            oprot.writeString(_iter762.getKey());
+            oprot.writeI32(_iter762.getValue());
           }
         }
       }
@@ -1506,10 +1506,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_component_debug()) {
         {
           oprot.writeI32(struct.component_debug.size());
-          for (Map.Entry<String, DebugOptions> _iter743 : struct.component_debug.entrySet())
+          for (Map.Entry<String, DebugOptions> _iter763 : struct.component_debug.entrySet())
           {
-            oprot.writeString(_iter743.getKey());
-            _iter743.getValue().write(oprot);
+            oprot.writeString(_iter763.getKey());
+            _iter763.getValue().write(oprot);
           }
         }
       }
@@ -1533,15 +1533,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       BitSet incoming = iprot.readBitSet(8);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map744 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-          struct.component_executors = new HashMap<String,Integer>(2*_map744.size);
-          String _key745;
-          int _val746;
-          for (int _i747 = 0; _i747 < _map744.size; ++_i747)
+          org.apache.thrift.protocol.TMap _map764 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+          struct.component_executors = new HashMap<String,Integer>(2*_map764.size);
+          String _key765;
+          int _val766;
+          for (int _i767 = 0; _i767 < _map764.size; ++_i767)
           {
-            _key745 = iprot.readString();
-            _val746 = iprot.readI32();
-            struct.component_executors.put(_key745, _val746);
+            _key765 = iprot.readString();
+            _val766 = iprot.readI32();
+            struct.component_executors.put(_key765, _val766);
           }
         }
         struct.set_component_executors_isSet(true);
@@ -1565,16 +1565,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TMap _map748 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.component_debug = new HashMap<String,DebugOptions>(2*_map748.size);
-          String _key749;
-          DebugOptions _val750;
-          for (int _i751 = 0; _i751 < _map748.size; ++_i751)
+          org.apache.thrift.protocol.TMap _map768 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.component_debug = new HashMap<String,DebugOptions>(2*_map768.size);
+          String _key769;
+          DebugOptions _val770;
+          for (int _i771 = 0; _i771 < _map768.size; ++_i771)
           {
-            _key749 = iprot.readString();
-            _val750 = new DebugOptions();
-            _val750.read(iprot);
-            struct.component_debug.put(_key749, _val750);
+            _key769 = iprot.readString();
+            _val770 = new DebugOptions();
+            _val770.read(iprot);
+            struct.component_debug.put(_key769, _val770);
           }
         }
         struct.set_component_debug_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/TopologyHistoryInfo.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/TopologyHistoryInfo.java b/storm-client/src/jvm/org/apache/storm/generated/TopologyHistoryInfo.java
index 0fc7cab..51a7625 100644
--- a/storm-client/src/jvm/org/apache/storm/generated/TopologyHistoryInfo.java
+++ b/storm-client/src/jvm/org/apache/storm/generated/TopologyHistoryInfo.java
@@ -364,13 +364,13 @@ public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHist
           case 1: // TOPO_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list842 = iprot.readListBegin();
-                struct.topo_ids = new ArrayList<String>(_list842.size);
-                String _elem843;
-                for (int _i844 = 0; _i844 < _list842.size; ++_i844)
+                org.apache.thrift.protocol.TList _list862 = iprot.readListBegin();
+                struct.topo_ids = new ArrayList<String>(_list862.size);
+                String _elem863;
+                for (int _i864 = 0; _i864 < _list862.size; ++_i864)
                 {
-                  _elem843 = iprot.readString();
-                  struct.topo_ids.add(_elem843);
+                  _elem863 = iprot.readString();
+                  struct.topo_ids.add(_elem863);
                 }
                 iprot.readListEnd();
               }
@@ -396,9 +396,9 @@ public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHist
         oprot.writeFieldBegin(TOPO_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.topo_ids.size()));
-          for (String _iter845 : struct.topo_ids)
+          for (String _iter865 : struct.topo_ids)
           {
-            oprot.writeString(_iter845);
+            oprot.writeString(_iter865);
           }
           oprot.writeListEnd();
         }
@@ -429,9 +429,9 @@ public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHist
       if (struct.is_set_topo_ids()) {
         {
           oprot.writeI32(struct.topo_ids.size());
-          for (String _iter846 : struct.topo_ids)
+          for (String _iter866 : struct.topo_ids)
           {
-            oprot.writeString(_iter846);
+            oprot.writeString(_iter866);
           }
         }
       }
@@ -443,13 +443,13 @@ public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHist
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list847 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.topo_ids = new ArrayList<String>(_list847.size);
-          String _elem848;
-          for (int _i849 = 0; _i849 < _list847.size; ++_i849)
+          org.apache.thrift.protocol.TList _list867 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.topo_ids = new ArrayList<String>(_list867.size);
+          String _elem868;
+          for (int _i869 = 0; _i869 < _list867.size; ++_i869)
           {
-            _elem848 = iprot.readString();
-            struct.topo_ids.add(_elem848);
+            _elem868 = iprot.readString();
+            struct.topo_ids.add(_elem868);
           }
         }
         struct.set_topo_ids_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/48e23a99/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricList.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricList.java b/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricList.java
new file mode 100644
index 0000000..0bf95b5
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricList.java
@@ -0,0 +1,466 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class WorkerMetricList implements org.apache.thrift.TBase<WorkerMetricList, WorkerMetricList._Fields>, java.io.Serializable, Cloneable, Comparable<WorkerMetricList> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WorkerMetricList");
+
+  private static final org.apache.thrift.protocol.TField METRICS_FIELD_DESC = new org.apache.thrift.protocol.TField("metrics", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WorkerMetricListStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WorkerMetricListTupleSchemeFactory());
+  }
+
+  private List<WorkerMetricPoint> metrics; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    METRICS((short)1, "metrics");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // METRICS
+          return METRICS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.METRICS, new org.apache.thrift.meta_data.FieldMetaData("metrics", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerMetricPoint.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WorkerMetricList.class, metaDataMap);
+  }
+
+  public WorkerMetricList() {
+  }
+
+  public WorkerMetricList(
+    List<WorkerMetricPoint> metrics)
+  {
+    this();
+    this.metrics = metrics;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WorkerMetricList(WorkerMetricList other) {
+    if (other.is_set_metrics()) {
+      List<WorkerMetricPoint> __this__metrics = new ArrayList<WorkerMetricPoint>(other.metrics.size());
+      for (WorkerMetricPoint other_element : other.metrics) {
+        __this__metrics.add(new WorkerMetricPoint(other_element));
+      }
+      this.metrics = __this__metrics;
+    }
+  }
+
+  public WorkerMetricList deepCopy() {
+    return new WorkerMetricList(this);
+  }
+
+  @Override
+  public void clear() {
+    this.metrics = null;
+  }
+
+  public int get_metrics_size() {
+    return (this.metrics == null) ? 0 : this.metrics.size();
+  }
+
+  public java.util.Iterator<WorkerMetricPoint> get_metrics_iterator() {
+    return (this.metrics == null) ? null : this.metrics.iterator();
+  }
+
+  public void add_to_metrics(WorkerMetricPoint elem) {
+    if (this.metrics == null) {
+      this.metrics = new ArrayList<WorkerMetricPoint>();
+    }
+    this.metrics.add(elem);
+  }
+
+  public List<WorkerMetricPoint> get_metrics() {
+    return this.metrics;
+  }
+
+  public void set_metrics(List<WorkerMetricPoint> metrics) {
+    this.metrics = metrics;
+  }
+
+  public void unset_metrics() {
+    this.metrics = null;
+  }
+
+  /** Returns true if field metrics is set (has been assigned a value) and false otherwise */
+  public boolean is_set_metrics() {
+    return this.metrics != null;
+  }
+
+  public void set_metrics_isSet(boolean value) {
+    if (!value) {
+      this.metrics = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case METRICS:
+      if (value == null) {
+        unset_metrics();
+      } else {
+        set_metrics((List<WorkerMetricPoint>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case METRICS:
+      return get_metrics();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case METRICS:
+      return is_set_metrics();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WorkerMetricList)
+      return this.equals((WorkerMetricList)that);
+    return false;
+  }
+
+  public boolean equals(WorkerMetricList that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_metrics = true && this.is_set_metrics();
+    boolean that_present_metrics = true && that.is_set_metrics();
+    if (this_present_metrics || that_present_metrics) {
+      if (!(this_present_metrics && that_present_metrics))
+        return false;
+      if (!this.metrics.equals(that.metrics))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_metrics = true && (is_set_metrics());
+    list.add(present_metrics);
+    if (present_metrics)
+      list.add(metrics);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WorkerMetricList other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_metrics()).compareTo(other.is_set_metrics());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_metrics()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metrics, other.metrics);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WorkerMetricList(");
+    boolean first = true;
+
+    sb.append("metrics:");
+    if (this.metrics == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.metrics);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WorkerMetricListStandardSchemeFactory implements SchemeFactory {
+    public WorkerMetricListStandardScheme getScheme() {
+      return new WorkerMetricListStandardScheme();
+    }
+  }
+
+  private static class WorkerMetricListStandardScheme extends StandardScheme<WorkerMetricList> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WorkerMetricList struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // METRICS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list870 = iprot.readListBegin();
+                struct.metrics = new ArrayList<WorkerMetricPoint>(_list870.size);
+                WorkerMetricPoint _elem871;
+                for (int _i872 = 0; _i872 < _list870.size; ++_i872)
+                {
+                  _elem871 = new WorkerMetricPoint();
+                  _elem871.read(iprot);
+                  struct.metrics.add(_elem871);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_metrics_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WorkerMetricList struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.metrics != null) {
+        oprot.writeFieldBegin(METRICS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.metrics.size()));
+          for (WorkerMetricPoint _iter873 : struct.metrics)
+          {
+            _iter873.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WorkerMetricListTupleSchemeFactory implements SchemeFactory {
+    public WorkerMetricListTupleScheme getScheme() {
+      return new WorkerMetricListTupleScheme();
+    }
+  }
+
+  private static class WorkerMetricListTupleScheme extends TupleScheme<WorkerMetricList> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WorkerMetricList struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_metrics()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_metrics()) {
+        {
+          oprot.writeI32(struct.metrics.size());
+          for (WorkerMetricPoint _iter874 : struct.metrics)
+          {
+            _iter874.write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WorkerMetricList struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list875 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.metrics = new ArrayList<WorkerMetricPoint>(_list875.size);
+          WorkerMetricPoint _elem876;
+          for (int _i877 = 0; _i877 < _list875.size; ++_i877)
+          {
+            _elem876 = new WorkerMetricPoint();
+            _elem876.read(iprot);
+            struct.metrics.add(_elem876);
+          }
+        }
+        struct.set_metrics_isSet(true);
+      }
+    }
+  }
+
+}
+