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);
+ }
+ }
+ }
+
+}
+