From e94b735a1381193fbf406657430284f381f9d8a7 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 21 Sep 2017 10:55:24 -0500 Subject: [PATCH 1/2] STORM-2438: added in rebalance changes to support RAS --- bin/storm.py | 23 +- .../hdfs/blobstore/HdfsClientBlobStore.java | 5 + .../storm/blobstore/ClientBlobStore.java | 14 +- .../blobstore/LocalModeClientBlobStore.java | 121 ++++ .../daemon/supervisor/AdvancedFSOps.java | 30 + .../daemon/supervisor/IAdvancedFSOps.java | 36 ++ .../apache/storm/generated/Assignment.java | 288 +++++----- .../generated/ClusterWorkerHeartbeat.java | 52 +- .../apache/storm/generated/Credentials.java | 44 +- .../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 +- .../storm/generated/ListBlobsResult.java | 32 +- .../storm/generated/LocalAssignment.java | 36 +- .../storm/generated/LocalStateData.java | 48 +- .../org/apache/storm/generated/LogConfig.java | 48 +- .../org/apache/storm/generated/Nimbus.java | 72 +-- .../org/apache/storm/generated/NodeInfo.java | 32 +- .../storm/generated/RebalanceOptions.java | 457 ++++++++++++++- .../storm/generated/SettableBlobMeta.java | 36 +- .../org/apache/storm/generated/StormBase.java | 92 ++-- .../storm/generated/SupervisorInfo.java | 152 ++--- .../storm/generated/TopologyHistoryInfo.java | 32 +- .../storm/topology/TopologyBuilder.java | 17 +- .../src/jvm/org/apache/storm/utils/Utils.java | 14 +- storm-client/src/py/storm/Nimbus.py | 28 +- storm-client/src/py/storm/ttypes.py | 515 +++++++++-------- storm-client/src/storm.thrift | 4 + .../storm/blobstore/ClientBlobStoreTest.java | 4 + .../org/apache/storm/command/Rebalance.java | 56 +- .../apache/storm/daemon/nimbus/Nimbus.java | 48 ++ .../daemon/supervisor/ReadClusterState.java | 7 - .../apache/storm/daemon/supervisor/Slot.java | 518 ++++++++++++++---- .../storm/localizer/AsyncLocalizer.java | 501 ++++++++--------- .../storm/localizer/BlobChangingCallback.java | 40 ++ .../org/apache/storm/localizer/GoodToGo.java | 70 +++ .../localizer/LocalDownloadedResource.java | 68 +-- .../storm/localizer/LocalizedResource.java | 13 + .../LocalizedResourceRetentionSet.java | 422 ++++++++++---- .../storm/localizer/LocallyCachedBlob.java | 220 ++++++++ .../localizer/LocallyCachedTopologyBlob.java | 364 ++++++++++++ .../storm/localizer/PortAndAssignment.java | 71 +++ .../scheduler/resource/ResourceUtils.java | 66 +++ .../org/apache/storm/utils/ServerUtils.java | 17 +- .../java/org/apache/storm/TestRebalance.java | 170 ++++++ .../storm/daemon/supervisor/SlotTest.java | 196 +++++-- .../storm/localizer/AsyncLocalizerTest.java | 120 ++-- .../LocalizedResourceRetentionSetTest.java | 31 +- 52 files changed, 3985 insertions(+), 1541 deletions(-) create mode 100644 storm-client/src/jvm/org/apache/storm/blobstore/LocalModeClientBlobStore.java create mode 100644 storm-server/src/main/java/org/apache/storm/localizer/BlobChangingCallback.java create mode 100644 storm-server/src/main/java/org/apache/storm/localizer/GoodToGo.java create mode 100644 storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java create mode 100644 storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java create mode 100644 storm-server/src/main/java/org/apache/storm/localizer/PortAndAssignment.java create mode 100644 storm-server/src/test/java/org/apache/storm/TestRebalance.java diff --git a/bin/storm.py b/bin/storm.py index e9d4bee9589..798cf99bd72 100755 --- a/bin/storm.py +++ b/bin/storm.py @@ -561,10 +561,10 @@ def deactivate(*args): extrajars=[USER_CONF_DIR, STORM_BIN_DIR]) def rebalance(*args): - """Syntax: [storm rebalance topology-name [-w wait-time-secs] [-n new-num-workers] [-e component=parallelism]*] + """Syntax: [storm rebalance topology-name [-w wait-time-secs] [-n new-num-workers] [-e component=parallelism]* [-r '{"component1": {"resource1": new_amount, "resource2": new_amount, ... }*}'] [-t '{"conf1": newValue, *}']] - Sometimes you may wish to spread out where the workers for a topology - are running. For example, let's say you have a 10 node cluster running + Sometimes you may wish to spread out the workers for a running topology. + For example, let's say you have a 10 node cluster running 4 workers per node, and then let's say you add another 10 nodes to the cluster. You may wish to have Storm spread out the workers for the running topology so that each node runs 2 workers. One way to do this @@ -572,14 +572,15 @@ def rebalance(*args): command that provides an easier way to do this. Rebalance will first deactivate the topology for the duration of the - message timeout (overridable with the -w flag) and then redistribute - the workers evenly around the cluster. The topology will then return to - its previous state of activation (so a deactivated topology will still - be deactivated and an activated topology will go back to being activated). - - The rebalance command can also be used to change the parallelism of a running topology. - Use the -n and -e switches to change the number of workers or number of executors of a component - respectively. + message timeout (overridable with the -w flag) make requested adjustments to the topology + and let the scheduler try to find a better scheduling based off of the + new situation. The topology will then return to its previous state of activation + (so a deactivated topology will still be deactivated and an activated + topology will go back to being activated). + + Some of what you can change about a topology includes the number of requested workers (-n flag) + The number of executors for a given component (-e flag) the resources each component is + requesting as used by the resource aware scheduler (-r flag) and configs (-t flag). """ if not args: print_usage(command="rebalance") diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java index 389dc71b3b0..e68d5e3e5d8 100644 --- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java @@ -117,6 +117,11 @@ public void createStateInZookeeper(String key) { @Override public void shutdown() { + close(); + } + + @Override + public void close() { if(client != null) { client.close(); client = null; diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java b/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java index 3291eb77dbe..d84f88bbfe3 100644 --- a/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java +++ b/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.blobstore; import org.apache.storm.daemon.Shutdownable; @@ -45,11 +46,8 @@ * * For more detailed implementation * @see org.apache.storm.blobstore.NimbusBlobStore - * @see org.apache.storm.blobstore.LocalFsBlobStore - * @see org.apache.storm.hdfs.blobstore.HdfsClientBlobStore - * @see org.apache.storm.hdfs.blobstore.HdfsBlobStore */ -public abstract class ClientBlobStore implements Shutdownable { +public abstract class ClientBlobStore implements Shutdownable, AutoCloseable { protected Map conf; public interface WithBlobstore { @@ -58,12 +56,8 @@ public interface WithBlobstore { public static void withConfiguredClient(WithBlobstore withBlobstore) throws Exception { Map conf = ConfigUtils.readStormConfig(); - ClientBlobStore blobStore = Utils.getClientBlobStore(conf); - - try { + try (ClientBlobStore blobStore = Utils.getClientBlobStore(conf)) { withBlobstore.run(blobStore); - } finally { - blobStore.shutdown(); } } @@ -168,6 +162,8 @@ public static void withConfiguredClient(WithBlobstore withBlobstore) throws Exce */ public abstract void createStateInZookeeper(String key); + public abstract void close(); + /** * Client facing API to create a blob. * @param key blob key name. diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/LocalModeClientBlobStore.java b/storm-client/src/jvm/org/apache/storm/blobstore/LocalModeClientBlobStore.java new file mode 100644 index 00000000000..1ca11efd93a --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/blobstore/LocalModeClientBlobStore.java @@ -0,0 +1,121 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.storm.blobstore; + +import java.util.Iterator; +import java.util.Map; +import org.apache.storm.generated.AuthorizationException; +import org.apache.storm.generated.KeyAlreadyExistsException; +import org.apache.storm.generated.KeyNotFoundException; +import org.apache.storm.generated.ReadableBlobMeta; +import org.apache.storm.generated.SettableBlobMeta; +import org.apache.storm.utils.NimbusClient; + +/** + * A Client blob store for LocalMode. + */ +public class LocalModeClientBlobStore extends ClientBlobStore { + private final BlobStore wrapped; + + public LocalModeClientBlobStore(BlobStore wrapped) { + this.wrapped = wrapped; + } + + @Override + public void shutdown() { + wrapped.shutdown(); + } + + @Override + public void prepare(Map conf) { + //NOOP prepare should have already been called + } + + @Override + protected AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException { + return wrapped.createBlob(key, meta, null); + } + + @Override + public AtomicOutputStream updateBlob(String key) throws AuthorizationException, KeyNotFoundException { + return wrapped.updateBlob(key, null); + } + + @Override + public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException { + return wrapped.getBlobMeta(key, null); + } + + @Override + protected void setBlobMetaToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException { + wrapped.setBlobMeta(key, meta, null); + } + + @Override + public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException { + wrapped.deleteBlob(key, null); + } + + @Override + public InputStreamWithMeta getBlob(String key) throws AuthorizationException, KeyNotFoundException { + return wrapped.getBlob(key, null); + } + + @Override + public Iterator listKeys() { + return wrapped.listKeys(); + } + + @Override + public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException { + try { + return wrapped.getBlobReplication(key, null); + } catch (AuthorizationException | KeyNotFoundException rethrow) { + throw rethrow; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException { + try { + return wrapped.updateBlobReplication(key, replication, null); + } catch (AuthorizationException | KeyNotFoundException rethrow) { + throw rethrow; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public boolean setClient(Map conf, NimbusClient client) { + return true; + } + + @Override + public void createStateInZookeeper(String key) { + //NOOP + } + + @Override + public void close() { + wrapped.shutdown(); + } +} \ No newline at end of file diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java index bee45883acb..f1740f25332 100644 --- a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java +++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.io.OutputStream; import java.io.Writer; +import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardCopyOption; @@ -279,6 +280,25 @@ public void forceMkdir(File path) throws IOException { FileUtils.forceMkdir(path); } + /** + * Makes a directory, including any necessary but nonexistent parent + * directories. + * + * @param path the directory to create + * @throws IOException on any error + */ + public void forceMkdir(Path path) throws IOException { + Files.createDirectories(path); + } + + public DirectoryStream newDirectoryStream(Path dir, DirectoryStream.Filter filter) throws IOException { + return Files.newDirectoryStream(dir, filter); + } + + public DirectoryStream newDirectoryStream(Path dir) throws IOException { + return Files.newDirectoryStream(dir); + } + /** * Check if a file exists or not * @param path the path to check @@ -289,6 +309,16 @@ public boolean fileExists(File path) throws IOException { return path.exists(); } + /** + * Check if a file exists or not + * @param path the path to check + * @return true if it exists else false + * @throws IOException on any error. + */ + public boolean fileExists(Path path) throws IOException { + return Files.exists(path); + } + /** * Get a writer for the given location * @param file the file to write to diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java index 5f23774ae74..2b09d390101 100644 --- a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java +++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java @@ -23,6 +23,8 @@ import java.io.IOException; import java.io.OutputStream; import java.io.Writer; +import java.nio.file.DirectoryStream; +import java.nio.file.Path; import java.util.Map; public interface IAdvancedFSOps { @@ -115,6 +117,32 @@ public interface IAdvancedFSOps { */ void forceMkdir(File path) throws IOException; + /** + * Makes a directory, including any necessary but nonexistent parent + * directories. + * + * @param path the directory to create + * @throws IOException on any error + */ + void forceMkdir(Path path) throws IOException; + + /** + * List the contents of a directory. + * @param dir the driectory to list the contents of + * @param filter a filter to decide if it should be included or not + * @return A stream of directory entries + * @throws IOException on any error + */ + DirectoryStream newDirectoryStream(Path dir, DirectoryStream.Filter filter) throws IOException; + + /** + * List the contents of a directory. + * @param dir the driectory to list the contents of + * @return A stream of directory entries + * @throws IOException on any error + */ + DirectoryStream newDirectoryStream(Path dir) throws IOException; + /** * Check if a file exists or not * @param path the path to check @@ -123,6 +151,14 @@ public interface IAdvancedFSOps { */ boolean fileExists(File path) throws IOException; + /** + * Check if a file exists or not + * @param path the path to check + * @return true if it exists else false + * @throws IOException on any error. + */ + boolean fileExists(Path path) throws IOException; + /** * Get a writer for the given location * @param file the file to write to 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 ccae34b025a..c3436d52f6f 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 void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) case 2: // NODE_HOST if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map646 = iprot.readMapBegin(); - struct.node_host = new HashMap(2*_map646.size); - String _key647; - String _val648; - for (int _i649 = 0; _i649 < _map646.size; ++_i649) + org.apache.thrift.protocol.TMap _map666 = iprot.readMapBegin(); + struct.node_host = new HashMap(2*_map666.size); + String _key667; + String _val668; + for (int _i669 = 0; _i669 < _map666.size; ++_i669) { - _key647 = iprot.readString(); - _val648 = iprot.readString(); - struct.node_host.put(_key647, _val648); + _key667 = iprot.readString(); + _val668 = iprot.readString(); + struct.node_host.put(_key667, _val668); } iprot.readMapEnd(); } @@ -986,26 +986,26 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) case 3: // EXECUTOR_NODE_PORT if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map650 = iprot.readMapBegin(); - struct.executor_node_port = new HashMap,NodeInfo>(2*_map650.size); - List _key651; - NodeInfo _val652; - for (int _i653 = 0; _i653 < _map650.size; ++_i653) + org.apache.thrift.protocol.TMap _map670 = iprot.readMapBegin(); + struct.executor_node_port = new HashMap,NodeInfo>(2*_map670.size); + List _key671; + NodeInfo _val672; + for (int _i673 = 0; _i673 < _map670.size; ++_i673) { { - org.apache.thrift.protocol.TList _list654 = iprot.readListBegin(); - _key651 = new ArrayList(_list654.size); - long _elem655; - for (int _i656 = 0; _i656 < _list654.size; ++_i656) + org.apache.thrift.protocol.TList _list674 = iprot.readListBegin(); + _key671 = new ArrayList(_list674.size); + long _elem675; + for (int _i676 = 0; _i676 < _list674.size; ++_i676) { - _elem655 = iprot.readI64(); - _key651.add(_elem655); + _elem675 = iprot.readI64(); + _key671.add(_elem675); } iprot.readListEnd(); } - _val652 = new NodeInfo(); - _val652.read(iprot); - struct.executor_node_port.put(_key651, _val652); + _val672 = new NodeInfo(); + _val672.read(iprot); + struct.executor_node_port.put(_key671, _val672); } iprot.readMapEnd(); } @@ -1017,25 +1017,25 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) case 4: // EXECUTOR_START_TIME_SECS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map657 = iprot.readMapBegin(); - struct.executor_start_time_secs = new HashMap,Long>(2*_map657.size); - List _key658; - long _val659; - for (int _i660 = 0; _i660 < _map657.size; ++_i660) + org.apache.thrift.protocol.TMap _map677 = iprot.readMapBegin(); + struct.executor_start_time_secs = new HashMap,Long>(2*_map677.size); + List _key678; + long _val679; + for (int _i680 = 0; _i680 < _map677.size; ++_i680) { { - org.apache.thrift.protocol.TList _list661 = iprot.readListBegin(); - _key658 = new ArrayList(_list661.size); - long _elem662; - for (int _i663 = 0; _i663 < _list661.size; ++_i663) + org.apache.thrift.protocol.TList _list681 = iprot.readListBegin(); + _key678 = new ArrayList(_list681.size); + long _elem682; + for (int _i683 = 0; _i683 < _list681.size; ++_i683) { - _elem662 = iprot.readI64(); - _key658.add(_elem662); + _elem682 = iprot.readI64(); + _key678.add(_elem682); } iprot.readListEnd(); } - _val659 = iprot.readI64(); - struct.executor_start_time_secs.put(_key658, _val659); + _val679 = iprot.readI64(); + struct.executor_start_time_secs.put(_key678, _val679); } iprot.readMapEnd(); } @@ -1047,17 +1047,17 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) case 5: // WORKER_RESOURCES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map664 = iprot.readMapBegin(); - struct.worker_resources = new HashMap(2*_map664.size); - NodeInfo _key665; - WorkerResources _val666; - for (int _i667 = 0; _i667 < _map664.size; ++_i667) + org.apache.thrift.protocol.TMap _map684 = iprot.readMapBegin(); + struct.worker_resources = new HashMap(2*_map684.size); + NodeInfo _key685; + WorkerResources _val686; + for (int _i687 = 0; _i687 < _map684.size; ++_i687) { - _key665 = new NodeInfo(); - _key665.read(iprot); - _val666 = new WorkerResources(); - _val666.read(iprot); - struct.worker_resources.put(_key665, _val666); + _key685 = new NodeInfo(); + _key685.read(iprot); + _val686 = new WorkerResources(); + _val686.read(iprot); + struct.worker_resources.put(_key685, _val686); } iprot.readMapEnd(); } @@ -1069,15 +1069,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) case 6: // TOTAL_SHARED_OFF_HEAP if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map668 = iprot.readMapBegin(); - struct.total_shared_off_heap = new HashMap(2*_map668.size); - String _key669; - double _val670; - for (int _i671 = 0; _i671 < _map668.size; ++_i671) + org.apache.thrift.protocol.TMap _map688 = iprot.readMapBegin(); + struct.total_shared_off_heap = new HashMap(2*_map688.size); + String _key689; + double _val690; + for (int _i691 = 0; _i691 < _map688.size; ++_i691) { - _key669 = iprot.readString(); - _val670 = iprot.readDouble(); - struct.total_shared_off_heap.put(_key669, _val670); + _key689 = iprot.readString(); + _val690 = iprot.readDouble(); + struct.total_shared_off_heap.put(_key689, _val690); } iprot.readMapEnd(); } @@ -1117,10 +1117,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) 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 _iter672 : struct.node_host.entrySet()) + for (Map.Entry _iter692 : struct.node_host.entrySet()) { - oprot.writeString(_iter672.getKey()); - oprot.writeString(_iter672.getValue()); + oprot.writeString(_iter692.getKey()); + oprot.writeString(_iter692.getValue()); } oprot.writeMapEnd(); } @@ -1132,17 +1132,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) 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, NodeInfo> _iter673 : struct.executor_node_port.entrySet()) + for (Map.Entry, NodeInfo> _iter693 : struct.executor_node_port.entrySet()) { { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter673.getKey().size())); - for (long _iter674 : _iter673.getKey()) + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter693.getKey().size())); + for (long _iter694 : _iter693.getKey()) { - oprot.writeI64(_iter674); + oprot.writeI64(_iter694); } oprot.writeListEnd(); } - _iter673.getValue().write(oprot); + _iter693.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -1154,17 +1154,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) 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, Long> _iter675 : struct.executor_start_time_secs.entrySet()) + for (Map.Entry, Long> _iter695 : struct.executor_start_time_secs.entrySet()) { { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter675.getKey().size())); - for (long _iter676 : _iter675.getKey()) + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter695.getKey().size())); + for (long _iter696 : _iter695.getKey()) { - oprot.writeI64(_iter676); + oprot.writeI64(_iter696); } oprot.writeListEnd(); } - oprot.writeI64(_iter675.getValue()); + oprot.writeI64(_iter695.getValue()); } oprot.writeMapEnd(); } @@ -1176,10 +1176,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) 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 _iter677 : struct.worker_resources.entrySet()) + for (Map.Entry _iter697 : struct.worker_resources.entrySet()) { - _iter677.getKey().write(oprot); - _iter677.getValue().write(oprot); + _iter697.getKey().write(oprot); + _iter697.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -1191,10 +1191,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) 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 _iter678 : struct.total_shared_off_heap.entrySet()) + for (Map.Entry _iter698 : struct.total_shared_off_heap.entrySet()) { - oprot.writeString(_iter678.getKey()); - oprot.writeDouble(_iter678.getValue()); + oprot.writeString(_iter698.getKey()); + oprot.writeDouble(_iter698.getValue()); } oprot.writeMapEnd(); } @@ -1249,62 +1249,62 @@ public void write(org.apache.thrift.protocol.TProtocol prot, Assignment struct) if (struct.is_set_node_host()) { { oprot.writeI32(struct.node_host.size()); - for (Map.Entry _iter679 : struct.node_host.entrySet()) + for (Map.Entry _iter699 : struct.node_host.entrySet()) { - oprot.writeString(_iter679.getKey()); - oprot.writeString(_iter679.getValue()); + oprot.writeString(_iter699.getKey()); + oprot.writeString(_iter699.getValue()); } } } if (struct.is_set_executor_node_port()) { { oprot.writeI32(struct.executor_node_port.size()); - for (Map.Entry, NodeInfo> _iter680 : struct.executor_node_port.entrySet()) + for (Map.Entry, NodeInfo> _iter700 : struct.executor_node_port.entrySet()) { { - oprot.writeI32(_iter680.getKey().size()); - for (long _iter681 : _iter680.getKey()) + oprot.writeI32(_iter700.getKey().size()); + for (long _iter701 : _iter700.getKey()) { - oprot.writeI64(_iter681); + oprot.writeI64(_iter701); } } - _iter680.getValue().write(oprot); + _iter700.getValue().write(oprot); } } } if (struct.is_set_executor_start_time_secs()) { { oprot.writeI32(struct.executor_start_time_secs.size()); - for (Map.Entry, Long> _iter682 : struct.executor_start_time_secs.entrySet()) + for (Map.Entry, Long> _iter702 : struct.executor_start_time_secs.entrySet()) { { - oprot.writeI32(_iter682.getKey().size()); - for (long _iter683 : _iter682.getKey()) + oprot.writeI32(_iter702.getKey().size()); + for (long _iter703 : _iter702.getKey()) { - oprot.writeI64(_iter683); + oprot.writeI64(_iter703); } } - oprot.writeI64(_iter682.getValue()); + oprot.writeI64(_iter702.getValue()); } } } if (struct.is_set_worker_resources()) { { oprot.writeI32(struct.worker_resources.size()); - for (Map.Entry _iter684 : struct.worker_resources.entrySet()) + for (Map.Entry _iter704 : struct.worker_resources.entrySet()) { - _iter684.getKey().write(oprot); - _iter684.getValue().write(oprot); + _iter704.getKey().write(oprot); + _iter704.getValue().write(oprot); } } } if (struct.is_set_total_shared_off_heap()) { { oprot.writeI32(struct.total_shared_off_heap.size()); - for (Map.Entry _iter685 : struct.total_shared_off_heap.entrySet()) + for (Map.Entry _iter705 : struct.total_shared_off_heap.entrySet()) { - oprot.writeString(_iter685.getKey()); - oprot.writeDouble(_iter685.getValue()); + oprot.writeString(_iter705.getKey()); + oprot.writeDouble(_iter705.getValue()); } } } @@ -1321,96 +1321,96 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Assignment struct) t BitSet incoming = iprot.readBitSet(6); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map686 = 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(2*_map686.size); - String _key687; - String _val688; - for (int _i689 = 0; _i689 < _map686.size; ++_i689) + 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(2*_map706.size); + String _key707; + String _val708; + for (int _i709 = 0; _i709 < _map706.size; ++_i709) { - _key687 = iprot.readString(); - _val688 = iprot.readString(); - struct.node_host.put(_key687, _val688); + _key707 = iprot.readString(); + _val708 = iprot.readString(); + struct.node_host.put(_key707, _val708); } } struct.set_node_host_isSet(true); } if (incoming.get(1)) { { - org.apache.thrift.protocol.TMap _map690 = 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,NodeInfo>(2*_map690.size); - List _key691; - NodeInfo _val692; - for (int _i693 = 0; _i693 < _map690.size; ++_i693) + 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,NodeInfo>(2*_map710.size); + List _key711; + NodeInfo _val712; + for (int _i713 = 0; _i713 < _map710.size; ++_i713) { { - org.apache.thrift.protocol.TList _list694 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - _key691 = new ArrayList(_list694.size); - long _elem695; - for (int _i696 = 0; _i696 < _list694.size; ++_i696) + org.apache.thrift.protocol.TList _list714 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _key711 = new ArrayList(_list714.size); + long _elem715; + for (int _i716 = 0; _i716 < _list714.size; ++_i716) { - _elem695 = iprot.readI64(); - _key691.add(_elem695); + _elem715 = iprot.readI64(); + _key711.add(_elem715); } } - _val692 = new NodeInfo(); - _val692.read(iprot); - struct.executor_node_port.put(_key691, _val692); + _val712 = new NodeInfo(); + _val712.read(iprot); + struct.executor_node_port.put(_key711, _val712); } } struct.set_executor_node_port_isSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map697 = 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,Long>(2*_map697.size); - List _key698; - long _val699; - for (int _i700 = 0; _i700 < _map697.size; ++_i700) + 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,Long>(2*_map717.size); + List _key718; + long _val719; + for (int _i720 = 0; _i720 < _map717.size; ++_i720) { { - org.apache.thrift.protocol.TList _list701 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - _key698 = new ArrayList(_list701.size); - long _elem702; - for (int _i703 = 0; _i703 < _list701.size; ++_i703) + org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _key718 = new ArrayList(_list721.size); + long _elem722; + for (int _i723 = 0; _i723 < _list721.size; ++_i723) { - _elem702 = iprot.readI64(); - _key698.add(_elem702); + _elem722 = iprot.readI64(); + _key718.add(_elem722); } } - _val699 = iprot.readI64(); - struct.executor_start_time_secs.put(_key698, _val699); + _val719 = iprot.readI64(); + struct.executor_start_time_secs.put(_key718, _val719); } } struct.set_executor_start_time_secs_isSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map704 = 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(2*_map704.size); - NodeInfo _key705; - WorkerResources _val706; - for (int _i707 = 0; _i707 < _map704.size; ++_i707) + 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(2*_map724.size); + NodeInfo _key725; + WorkerResources _val726; + for (int _i727 = 0; _i727 < _map724.size; ++_i727) { - _key705 = new NodeInfo(); - _key705.read(iprot); - _val706 = new WorkerResources(); - _val706.read(iprot); - struct.worker_resources.put(_key705, _val706); + _key725 = new NodeInfo(); + _key725.read(iprot); + _val726 = new WorkerResources(); + _val726.read(iprot); + struct.worker_resources.put(_key725, _val726); } } struct.set_worker_resources_isSet(true); } if (incoming.get(4)) { { - org.apache.thrift.protocol.TMap _map708 = 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(2*_map708.size); - String _key709; - double _val710; - for (int _i711 = 0; _i711 < _map708.size; ++_i711) + 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(2*_map728.size); + String _key729; + double _val730; + for (int _i731 = 0; _i731 < _map728.size; ++_i731) { - _key709 = iprot.readString(); - _val710 = iprot.readDouble(); - struct.total_shared_off_heap.put(_key709, _val710); + _key729 = iprot.readString(); + _val730 = iprot.readDouble(); + struct.total_shared_off_heap.put(_key729, _val730); } } struct.set_total_shared_off_heap_isSet(true); 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 f5a8acc21df..1613778f7d0 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 void read(org.apache.thrift.protocol.TProtocol iprot, ClusterWorkerHeartb case 2: // EXECUTOR_STATS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map732 = iprot.readMapBegin(); - struct.executor_stats = new HashMap(2*_map732.size); - ExecutorInfo _key733; - ExecutorStats _val734; - for (int _i735 = 0; _i735 < _map732.size; ++_i735) + org.apache.thrift.protocol.TMap _map752 = iprot.readMapBegin(); + struct.executor_stats = new HashMap(2*_map752.size); + ExecutorInfo _key753; + ExecutorStats _val754; + for (int _i755 = 0; _i755 < _map752.size; ++_i755) { - _key733 = new ExecutorInfo(); - _key733.read(iprot); - _val734 = new ExecutorStats(); - _val734.read(iprot); - struct.executor_stats.put(_key733, _val734); + _key753 = new ExecutorInfo(); + _key753.read(iprot); + _val754 = new ExecutorStats(); + _val754.read(iprot); + struct.executor_stats.put(_key753, _val754); } iprot.readMapEnd(); } @@ -692,10 +692,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ClusterWorkerHeart 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 _iter736 : struct.executor_stats.entrySet()) + for (Map.Entry _iter756 : struct.executor_stats.entrySet()) { - _iter736.getKey().write(oprot); - _iter736.getValue().write(oprot); + _iter756.getKey().write(oprot); + _iter756.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -727,10 +727,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClusterWorkerHeartb oprot.writeString(struct.storm_id); { oprot.writeI32(struct.executor_stats.size()); - for (Map.Entry _iter737 : struct.executor_stats.entrySet()) + for (Map.Entry _iter757 : struct.executor_stats.entrySet()) { - _iter737.getKey().write(oprot); - _iter737.getValue().write(oprot); + _iter757.getKey().write(oprot); + _iter757.getValue().write(oprot); } } oprot.writeI32(struct.time_secs); @@ -743,17 +743,17 @@ public void read(org.apache.thrift.protocol.TProtocol prot, ClusterWorkerHeartbe struct.storm_id = iprot.readString(); struct.set_storm_id_isSet(true); { - org.apache.thrift.protocol.TMap _map738 = 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(2*_map738.size); - ExecutorInfo _key739; - ExecutorStats _val740; - for (int _i741 = 0; _i741 < _map738.size; ++_i741) + 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(2*_map758.size); + ExecutorInfo _key759; + ExecutorStats _val760; + for (int _i761 = 0; _i761 < _map758.size; ++_i761) { - _key739 = new ExecutorInfo(); - _key739.read(iprot); - _val740 = new ExecutorStats(); - _val740.read(iprot); - struct.executor_stats.put(_key739, _val740); + _key759 = new ExecutorInfo(); + _key759.read(iprot); + _val760 = new ExecutorStats(); + _val760.read(iprot); + struct.executor_stats.put(_key759, _val760); } } struct.set_executor_stats_isSet(true); diff --git a/storm-client/src/jvm/org/apache/storm/generated/Credentials.java b/storm-client/src/jvm/org/apache/storm/generated/Credentials.java index 4d7b6940e26..84a8fb90425 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/Credentials.java +++ b/storm-client/src/jvm/org/apache/storm/generated/Credentials.java @@ -365,15 +365,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Credentials struct) case 1: // CREDS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map576 = iprot.readMapBegin(); - struct.creds = new HashMap(2*_map576.size); - String _key577; - String _val578; - for (int _i579 = 0; _i579 < _map576.size; ++_i579) + org.apache.thrift.protocol.TMap _map596 = iprot.readMapBegin(); + struct.creds = new HashMap(2*_map596.size); + String _key597; + String _val598; + for (int _i599 = 0; _i599 < _map596.size; ++_i599) { - _key577 = iprot.readString(); - _val578 = iprot.readString(); - struct.creds.put(_key577, _val578); + _key597 = iprot.readString(); + _val598 = iprot.readString(); + struct.creds.put(_key597, _val598); } iprot.readMapEnd(); } @@ -399,10 +399,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Credentials struct oprot.writeFieldBegin(CREDS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.creds.size())); - for (Map.Entry _iter580 : struct.creds.entrySet()) + for (Map.Entry _iter600 : struct.creds.entrySet()) { - oprot.writeString(_iter580.getKey()); - oprot.writeString(_iter580.getValue()); + oprot.writeString(_iter600.getKey()); + oprot.writeString(_iter600.getValue()); } oprot.writeMapEnd(); } @@ -427,10 +427,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, Credentials struct) TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.creds.size()); - for (Map.Entry _iter581 : struct.creds.entrySet()) + for (Map.Entry _iter601 : struct.creds.entrySet()) { - oprot.writeString(_iter581.getKey()); - oprot.writeString(_iter581.getValue()); + oprot.writeString(_iter601.getKey()); + oprot.writeString(_iter601.getValue()); } } } @@ -439,15 +439,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, Credentials struct) public void read(org.apache.thrift.protocol.TProtocol prot, Credentials struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; { - org.apache.thrift.protocol.TMap _map582 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.creds = new HashMap(2*_map582.size); - String _key583; - String _val584; - for (int _i585 = 0; _i585 < _map582.size; ++_i585) + org.apache.thrift.protocol.TMap _map602 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.creds = new HashMap(2*_map602.size); + String _key603; + String _val604; + for (int _i605 = 0; _i605 < _map602.size; ++_i605) { - _key583 = iprot.readString(); - _val584 = iprot.readString(); - struct.creds.put(_key583, _val584); + _key603 = iprot.readString(); + _val604 = iprot.readString(); + struct.creds.put(_key603, _val604); } } struct.set_creds_isSet(true); 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 bdec1afe99d..75c5c6d4ad3 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 void read(org.apache.thrift.protocol.TProtocol iprot, HBNodes struct) thr case 1: // PULSE_IDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list838 = iprot.readListBegin(); - struct.pulseIds = new ArrayList(_list838.size); - String _elem839; - for (int _i840 = 0; _i840 < _list838.size; ++_i840) + org.apache.thrift.protocol.TList _list858 = iprot.readListBegin(); + struct.pulseIds = new ArrayList(_list858.size); + String _elem859; + for (int _i860 = 0; _i860 < _list858.size; ++_i860) { - _elem839 = iprot.readString(); - struct.pulseIds.add(_elem839); + _elem859 = iprot.readString(); + struct.pulseIds.add(_elem859); } iprot.readListEnd(); } @@ -396,9 +396,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, HBNodes struct) th 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 _iter841 : struct.pulseIds) + for (String _iter861 : struct.pulseIds) { - oprot.writeString(_iter841); + oprot.writeString(_iter861); } oprot.writeListEnd(); } @@ -429,9 +429,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, HBNodes struct) thr if (struct.is_set_pulseIds()) { { oprot.writeI32(struct.pulseIds.size()); - for (String _iter842 : struct.pulseIds) + for (String _iter862 : struct.pulseIds) { - oprot.writeString(_iter842); + oprot.writeString(_iter862); } } } @@ -443,13 +443,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, HBNodes struct) thro BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list843 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.pulseIds = new ArrayList(_list843.size); - String _elem844; - for (int _i845 = 0; _i845 < _list843.size; ++_i845) + org.apache.thrift.protocol.TList _list863 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.pulseIds = new ArrayList(_list863.size); + String _elem864; + for (int _i865 = 0; _i865 < _list863.size; ++_i865) { - _elem844 = iprot.readString(); - struct.pulseIds.add(_elem844); + _elem864 = iprot.readString(); + struct.pulseIds.add(_elem864); } } struct.set_pulseIds_isSet(true); 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 83d49bfc58a..f726e5cf399 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 void read(org.apache.thrift.protocol.TProtocol iprot, HBRecords struct) t case 1: // PULSES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list830 = iprot.readListBegin(); - struct.pulses = new ArrayList(_list830.size); - HBPulse _elem831; - for (int _i832 = 0; _i832 < _list830.size; ++_i832) + org.apache.thrift.protocol.TList _list850 = iprot.readListBegin(); + struct.pulses = new ArrayList(_list850.size); + HBPulse _elem851; + for (int _i852 = 0; _i852 < _list850.size; ++_i852) { - _elem831 = new HBPulse(); - _elem831.read(iprot); - struct.pulses.add(_elem831); + _elem851 = new HBPulse(); + _elem851.read(iprot); + struct.pulses.add(_elem851); } iprot.readListEnd(); } @@ -400,9 +400,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, HBRecords struct) oprot.writeFieldBegin(PULSES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pulses.size())); - for (HBPulse _iter833 : struct.pulses) + for (HBPulse _iter853 : struct.pulses) { - _iter833.write(oprot); + _iter853.write(oprot); } oprot.writeListEnd(); } @@ -433,9 +433,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, HBRecords struct) t if (struct.is_set_pulses()) { { oprot.writeI32(struct.pulses.size()); - for (HBPulse _iter834 : struct.pulses) + for (HBPulse _iter854 : struct.pulses) { - _iter834.write(oprot); + _iter854.write(oprot); } } } @@ -447,14 +447,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, HBRecords struct) th BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list835 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.pulses = new ArrayList(_list835.size); - HBPulse _elem836; - for (int _i837 = 0; _i837 < _list835.size; ++_i837) + org.apache.thrift.protocol.TList _list855 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.pulses = new ArrayList(_list855.size); + HBPulse _elem856; + for (int _i857 = 0; _i857 < _list855.size; ++_i857) { - _elem836 = new HBPulse(); - _elem836.read(iprot); - struct.pulses.add(_elem836); + _elem856 = new HBPulse(); + _elem856.read(iprot); + struct.pulses.add(_elem856); } } struct.set_pulses_isSet(true); 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 fc7fe5a28f6..9e8e5cf7d64 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 void read(org.apache.thrift.protocol.TProtocol iprot, LSApprovedWorkers s case 1: // APPROVED_WORKERS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map760 = iprot.readMapBegin(); - struct.approved_workers = new HashMap(2*_map760.size); - String _key761; - int _val762; - for (int _i763 = 0; _i763 < _map760.size; ++_i763) + org.apache.thrift.protocol.TMap _map780 = iprot.readMapBegin(); + struct.approved_workers = new HashMap(2*_map780.size); + String _key781; + int _val782; + for (int _i783 = 0; _i783 < _map780.size; ++_i783) { - _key761 = iprot.readString(); - _val762 = iprot.readI32(); - struct.approved_workers.put(_key761, _val762); + _key781 = iprot.readString(); + _val782 = iprot.readI32(); + struct.approved_workers.put(_key781, _val782); } iprot.readMapEnd(); } @@ -399,10 +399,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LSApprovedWorkers 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 _iter764 : struct.approved_workers.entrySet()) + for (Map.Entry _iter784 : struct.approved_workers.entrySet()) { - oprot.writeString(_iter764.getKey()); - oprot.writeI32(_iter764.getValue()); + oprot.writeString(_iter784.getKey()); + oprot.writeI32(_iter784.getValue()); } oprot.writeMapEnd(); } @@ -427,10 +427,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers s TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.approved_workers.size()); - for (Map.Entry _iter765 : struct.approved_workers.entrySet()) + for (Map.Entry _iter785 : struct.approved_workers.entrySet()) { - oprot.writeString(_iter765.getKey()); - oprot.writeI32(_iter765.getValue()); + oprot.writeString(_iter785.getKey()); + oprot.writeI32(_iter785.getValue()); } } } @@ -439,15 +439,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers s 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 _map766 = 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(2*_map766.size); - String _key767; - int _val768; - for (int _i769 = 0; _i769 < _map766.size; ++_i769) + 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(2*_map786.size); + String _key787; + int _val788; + for (int _i789 = 0; _i789 < _map786.size; ++_i789) { - _key767 = iprot.readString(); - _val768 = iprot.readI32(); - struct.approved_workers.put(_key767, _val768); + _key787 = iprot.readString(); + _val788 = iprot.readI32(); + struct.approved_workers.put(_key787, _val788); } } struct.set_approved_workers_isSet(true); 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 ddf6456462c..64c36e27e24 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 void read(org.apache.thrift.protocol.TProtocol iprot, LSSupervisorAssignm case 1: // ASSIGNMENTS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map770 = iprot.readMapBegin(); - struct.assignments = new HashMap(2*_map770.size); - int _key771; - LocalAssignment _val772; - for (int _i773 = 0; _i773 < _map770.size; ++_i773) + org.apache.thrift.protocol.TMap _map790 = iprot.readMapBegin(); + struct.assignments = new HashMap(2*_map790.size); + int _key791; + LocalAssignment _val792; + for (int _i793 = 0; _i793 < _map790.size; ++_i793) { - _key771 = iprot.readI32(); - _val772 = new LocalAssignment(); - _val772.read(iprot); - struct.assignments.put(_key771, _val772); + _key791 = iprot.readI32(); + _val792 = new LocalAssignment(); + _val792.read(iprot); + struct.assignments.put(_key791, _val792); } iprot.readMapEnd(); } @@ -411,10 +411,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LSSupervisorAssign 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 _iter774 : struct.assignments.entrySet()) + for (Map.Entry _iter794 : struct.assignments.entrySet()) { - oprot.writeI32(_iter774.getKey()); - _iter774.getValue().write(oprot); + oprot.writeI32(_iter794.getKey()); + _iter794.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -439,10 +439,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignm TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.assignments.size()); - for (Map.Entry _iter775 : struct.assignments.entrySet()) + for (Map.Entry _iter795 : struct.assignments.entrySet()) { - oprot.writeI32(_iter775.getKey()); - _iter775.getValue().write(oprot); + oprot.writeI32(_iter795.getKey()); + _iter795.getValue().write(oprot); } } } @@ -451,16 +451,16 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignm 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 _map776 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.assignments = new HashMap(2*_map776.size); - int _key777; - LocalAssignment _val778; - for (int _i779 = 0; _i779 < _map776.size; ++_i779) + 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(2*_map796.size); + int _key797; + LocalAssignment _val798; + for (int _i799 = 0; _i799 < _map796.size; ++_i799) { - _key777 = iprot.readI32(); - _val778 = new LocalAssignment(); - _val778.read(iprot); - struct.assignments.put(_key777, _val778); + _key797 = iprot.readI32(); + _val798 = new LocalAssignment(); + _val798.read(iprot); + struct.assignments.put(_key797, _val798); } } struct.set_assignments_isSet(true); 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 8896a0dc55f..6de53df9b3b 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 void read(org.apache.thrift.protocol.TProtocol iprot, LSTopoHistory struc case 3: // USERS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list788 = iprot.readListBegin(); - struct.users = new ArrayList(_list788.size); - String _elem789; - for (int _i790 = 0; _i790 < _list788.size; ++_i790) + org.apache.thrift.protocol.TList _list808 = iprot.readListBegin(); + struct.users = new ArrayList(_list808.size); + String _elem809; + for (int _i810 = 0; _i810 < _list808.size; ++_i810) { - _elem789 = iprot.readString(); - struct.users.add(_elem789); + _elem809 = iprot.readString(); + struct.users.add(_elem809); } iprot.readListEnd(); } @@ -674,13 +674,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, LSTopoHistory struc case 4: // GROUPS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list791 = iprot.readListBegin(); - struct.groups = new ArrayList(_list791.size); - String _elem792; - for (int _i793 = 0; _i793 < _list791.size; ++_i793) + org.apache.thrift.protocol.TList _list811 = iprot.readListBegin(); + struct.groups = new ArrayList(_list811.size); + String _elem812; + for (int _i813 = 0; _i813 < _list811.size; ++_i813) { - _elem792 = iprot.readString(); - struct.groups.add(_elem792); + _elem812 = iprot.readString(); + struct.groups.add(_elem812); } iprot.readListEnd(); } @@ -714,9 +714,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LSTopoHistory stru oprot.writeFieldBegin(USERS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.users.size())); - for (String _iter794 : struct.users) + for (String _iter814 : struct.users) { - oprot.writeString(_iter794); + oprot.writeString(_iter814); } oprot.writeListEnd(); } @@ -726,9 +726,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LSTopoHistory stru oprot.writeFieldBegin(GROUPS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groups.size())); - for (String _iter795 : struct.groups) + for (String _iter815 : struct.groups) { - oprot.writeString(_iter795); + oprot.writeString(_iter815); } oprot.writeListEnd(); } @@ -755,16 +755,16 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSTopoHistory struc oprot.writeI64(struct.time_stamp); { oprot.writeI32(struct.users.size()); - for (String _iter796 : struct.users) + for (String _iter816 : struct.users) { - oprot.writeString(_iter796); + oprot.writeString(_iter816); } } { oprot.writeI32(struct.groups.size()); - for (String _iter797 : struct.groups) + for (String _iter817 : struct.groups) { - oprot.writeString(_iter797); + oprot.writeString(_iter817); } } } @@ -777,24 +777,24 @@ public void read(org.apache.thrift.protocol.TProtocol prot, LSTopoHistory struct struct.time_stamp = iprot.readI64(); struct.set_time_stamp_isSet(true); { - org.apache.thrift.protocol.TList _list798 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.users = new ArrayList(_list798.size); - String _elem799; - for (int _i800 = 0; _i800 < _list798.size; ++_i800) + org.apache.thrift.protocol.TList _list818 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.users = new ArrayList(_list818.size); + String _elem819; + for (int _i820 = 0; _i820 < _list818.size; ++_i820) { - _elem799 = iprot.readString(); - struct.users.add(_elem799); + _elem819 = iprot.readString(); + struct.users.add(_elem819); } } struct.set_users_isSet(true); { - org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.groups = new ArrayList(_list801.size); - String _elem802; - for (int _i803 = 0; _i803 < _list801.size; ++_i803) + org.apache.thrift.protocol.TList _list821 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.groups = new ArrayList(_list821.size); + String _elem822; + for (int _i823 = 0; _i823 < _list821.size; ++_i823) { - _elem802 = iprot.readString(); - struct.groups.add(_elem802); + _elem822 = iprot.readString(); + struct.groups.add(_elem822); } } struct.set_groups_isSet(true); 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 5d8f526e134..790a6fba3df 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 void read(org.apache.thrift.protocol.TProtocol iprot, LSTopoHistoryList s case 1: // TOPO_HISTORY if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list804 = iprot.readListBegin(); - struct.topo_history = new ArrayList(_list804.size); - LSTopoHistory _elem805; - for (int _i806 = 0; _i806 < _list804.size; ++_i806) + org.apache.thrift.protocol.TList _list824 = iprot.readListBegin(); + struct.topo_history = new ArrayList(_list824.size); + LSTopoHistory _elem825; + for (int _i826 = 0; _i826 < _list824.size; ++_i826) { - _elem805 = new LSTopoHistory(); - _elem805.read(iprot); - struct.topo_history.add(_elem805); + _elem825 = new LSTopoHistory(); + _elem825.read(iprot); + struct.topo_history.add(_elem825); } iprot.readListEnd(); } @@ -404,9 +404,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LSTopoHistoryList 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 _iter807 : struct.topo_history) + for (LSTopoHistory _iter827 : struct.topo_history) { - _iter807.write(oprot); + _iter827.write(oprot); } oprot.writeListEnd(); } @@ -431,9 +431,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSTopoHistoryList s TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.topo_history.size()); - for (LSTopoHistory _iter808 : struct.topo_history) + for (LSTopoHistory _iter828 : struct.topo_history) { - _iter808.write(oprot); + _iter828.write(oprot); } } } @@ -442,14 +442,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSTopoHistoryList s 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 _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.topo_history = new ArrayList(_list809.size); - LSTopoHistory _elem810; - for (int _i811 = 0; _i811 < _list809.size; ++_i811) + 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(_list829.size); + LSTopoHistory _elem830; + for (int _i831 = 0; _i831 < _list829.size; ++_i831) { - _elem810 = new LSTopoHistory(); - _elem810.read(iprot); - struct.topo_history.add(_elem810); + _elem830 = new LSTopoHistory(); + _elem830.read(iprot); + struct.topo_history.add(_elem830); } } struct.set_topo_history_isSet(true); 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 6db82981518..6cf386f106e 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 void read(org.apache.thrift.protocol.TProtocol iprot, LSWorkerHeartbeat s case 3: // EXECUTORS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list780 = iprot.readListBegin(); - struct.executors = new ArrayList(_list780.size); - ExecutorInfo _elem781; - for (int _i782 = 0; _i782 < _list780.size; ++_i782) + org.apache.thrift.protocol.TList _list800 = iprot.readListBegin(); + struct.executors = new ArrayList(_list800.size); + ExecutorInfo _elem801; + for (int _i802 = 0; _i802 < _list800.size; ++_i802) { - _elem781 = new ExecutorInfo(); - _elem781.read(iprot); - struct.executors.add(_elem781); + _elem801 = new ExecutorInfo(); + _elem801.read(iprot); + struct.executors.add(_elem801); } iprot.readListEnd(); } @@ -687,9 +687,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LSWorkerHeartbeat oprot.writeFieldBegin(EXECUTORS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size())); - for (ExecutorInfo _iter783 : struct.executors) + for (ExecutorInfo _iter803 : struct.executors) { - _iter783.write(oprot); + _iter803.write(oprot); } oprot.writeListEnd(); } @@ -719,9 +719,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSWorkerHeartbeat s oprot.writeString(struct.topology_id); { oprot.writeI32(struct.executors.size()); - for (ExecutorInfo _iter784 : struct.executors) + for (ExecutorInfo _iter804 : struct.executors) { - _iter784.write(oprot); + _iter804.write(oprot); } } oprot.writeI32(struct.port); @@ -735,14 +735,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, LSWorkerHeartbeat st struct.topology_id = iprot.readString(); struct.set_topology_id_isSet(true); { - org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.executors = new ArrayList(_list785.size); - ExecutorInfo _elem786; - for (int _i787 = 0; _i787 < _list785.size; ++_i787) + org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.executors = new ArrayList(_list805.size); + ExecutorInfo _elem806; + for (int _i807 = 0; _i807 < _list805.size; ++_i807) { - _elem786 = new ExecutorInfo(); - _elem786.read(iprot); - struct.executors.add(_elem786); + _elem806 = new ExecutorInfo(); + _elem806.read(iprot); + struct.executors.add(_elem806); } } struct.set_executors_isSet(true); diff --git a/storm-client/src/jvm/org/apache/storm/generated/ListBlobsResult.java b/storm-client/src/jvm/org/apache/storm/generated/ListBlobsResult.java index ef24599b6a9..21b3e5ba80f 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/ListBlobsResult.java +++ b/storm-client/src/jvm/org/apache/storm/generated/ListBlobsResult.java @@ -453,13 +453,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ListBlobsResult str case 1: // KEYS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list594 = iprot.readListBegin(); - struct.keys = new ArrayList(_list594.size); - String _elem595; - for (int _i596 = 0; _i596 < _list594.size; ++_i596) + org.apache.thrift.protocol.TList _list614 = iprot.readListBegin(); + struct.keys = new ArrayList(_list614.size); + String _elem615; + for (int _i616 = 0; _i616 < _list614.size; ++_i616) { - _elem595 = iprot.readString(); - struct.keys.add(_elem595); + _elem615 = iprot.readString(); + struct.keys.add(_elem615); } iprot.readListEnd(); } @@ -493,9 +493,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ListBlobsResult st oprot.writeFieldBegin(KEYS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.keys.size())); - for (String _iter597 : struct.keys) + for (String _iter617 : struct.keys) { - oprot.writeString(_iter597); + oprot.writeString(_iter617); } oprot.writeListEnd(); } @@ -525,9 +525,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ListBlobsResult str TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.keys.size()); - for (String _iter598 : struct.keys) + for (String _iter618 : struct.keys) { - oprot.writeString(_iter598); + oprot.writeString(_iter618); } } oprot.writeString(struct.session); @@ -537,13 +537,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ListBlobsResult str public void read(org.apache.thrift.protocol.TProtocol prot, ListBlobsResult struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.keys = new ArrayList(_list599.size); - String _elem600; - for (int _i601 = 0; _i601 < _list599.size; ++_i601) + org.apache.thrift.protocol.TList _list619 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.keys = new ArrayList(_list619.size); + String _elem620; + for (int _i621 = 0; _i621 < _list619.size; ++_i621) { - _elem600 = iprot.readString(); - struct.keys.add(_elem600); + _elem620 = iprot.readString(); + struct.keys.add(_elem620); } } struct.set_keys_isSet(true); 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 88ce03987d6..e4d83aafed4 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 void read(org.apache.thrift.protocol.TProtocol iprot, LocalAssignment str case 2: // EXECUTORS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list752 = iprot.readListBegin(); - struct.executors = new ArrayList(_list752.size); - ExecutorInfo _elem753; - for (int _i754 = 0; _i754 < _list752.size; ++_i754) + org.apache.thrift.protocol.TList _list772 = iprot.readListBegin(); + struct.executors = new ArrayList(_list772.size); + ExecutorInfo _elem773; + for (int _i774 = 0; _i774 < _list772.size; ++_i774) { - _elem753 = new ExecutorInfo(); - _elem753.read(iprot); - struct.executors.add(_elem753); + _elem773 = new ExecutorInfo(); + _elem773.read(iprot); + struct.executors.add(_elem773); } iprot.readListEnd(); } @@ -773,9 +773,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LocalAssignment st oprot.writeFieldBegin(EXECUTORS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size())); - for (ExecutorInfo _iter755 : struct.executors) + for (ExecutorInfo _iter775 : struct.executors) { - _iter755.write(oprot); + _iter775.write(oprot); } oprot.writeListEnd(); } @@ -820,9 +820,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LocalAssignment str oprot.writeString(struct.topology_id); { oprot.writeI32(struct.executors.size()); - for (ExecutorInfo _iter756 : struct.executors) + for (ExecutorInfo _iter776 : struct.executors) { - _iter756.write(oprot); + _iter776.write(oprot); } } BitSet optionals = new BitSet(); @@ -853,14 +853,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, LocalAssignment stru struct.topology_id = iprot.readString(); struct.set_topology_id_isSet(true); { - org.apache.thrift.protocol.TList _list757 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.executors = new ArrayList(_list757.size); - ExecutorInfo _elem758; - for (int _i759 = 0; _i759 < _list757.size; ++_i759) + org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.executors = new ArrayList(_list777.size); + ExecutorInfo _elem778; + for (int _i779 = 0; _i779 < _list777.size; ++_i779) { - _elem758 = new ExecutorInfo(); - _elem758.read(iprot); - struct.executors.add(_elem758); + _elem778 = new ExecutorInfo(); + _elem778.read(iprot); + struct.executors.add(_elem778); } } struct.set_executors_isSet(true); 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 d644e733002..3536c0b17cf 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 void read(org.apache.thrift.protocol.TProtocol iprot, LocalStateData stru case 1: // SERIALIZED_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map742 = iprot.readMapBegin(); - struct.serialized_parts = new HashMap(2*_map742.size); - String _key743; - ThriftSerializedObject _val744; - for (int _i745 = 0; _i745 < _map742.size; ++_i745) + org.apache.thrift.protocol.TMap _map762 = iprot.readMapBegin(); + struct.serialized_parts = new HashMap(2*_map762.size); + String _key763; + ThriftSerializedObject _val764; + for (int _i765 = 0; _i765 < _map762.size; ++_i765) { - _key743 = iprot.readString(); - _val744 = new ThriftSerializedObject(); - _val744.read(iprot); - struct.serialized_parts.put(_key743, _val744); + _key763 = iprot.readString(); + _val764 = new ThriftSerializedObject(); + _val764.read(iprot); + struct.serialized_parts.put(_key763, _val764); } iprot.readMapEnd(); } @@ -411,10 +411,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LocalStateData str 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 _iter746 : struct.serialized_parts.entrySet()) + for (Map.Entry _iter766 : struct.serialized_parts.entrySet()) { - oprot.writeString(_iter746.getKey()); - _iter746.getValue().write(oprot); + oprot.writeString(_iter766.getKey()); + _iter766.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -439,10 +439,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LocalStateData stru TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.serialized_parts.size()); - for (Map.Entry _iter747 : struct.serialized_parts.entrySet()) + for (Map.Entry _iter767 : struct.serialized_parts.entrySet()) { - oprot.writeString(_iter747.getKey()); - _iter747.getValue().write(oprot); + oprot.writeString(_iter767.getKey()); + _iter767.getValue().write(oprot); } } } @@ -451,16 +451,16 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LocalStateData stru 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 _map748 = 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(2*_map748.size); - String _key749; - ThriftSerializedObject _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.serialized_parts = new HashMap(2*_map768.size); + String _key769; + ThriftSerializedObject _val770; + for (int _i771 = 0; _i771 < _map768.size; ++_i771) { - _key749 = iprot.readString(); - _val750 = new ThriftSerializedObject(); - _val750.read(iprot); - struct.serialized_parts.put(_key749, _val750); + _key769 = iprot.readString(); + _val770 = new ThriftSerializedObject(); + _val770.read(iprot); + struct.serialized_parts.put(_key769, _val770); } } struct.set_serialized_parts_isSet(true); 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 58086a17604..e783e6c459e 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 void read(org.apache.thrift.protocol.TProtocol iprot, LogConfig struct) t case 2: // NAMED_LOGGER_LEVEL if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map812 = iprot.readMapBegin(); - struct.named_logger_level = new HashMap(2*_map812.size); - String _key813; - LogLevel _val814; - for (int _i815 = 0; _i815 < _map812.size; ++_i815) + org.apache.thrift.protocol.TMap _map832 = iprot.readMapBegin(); + struct.named_logger_level = new HashMap(2*_map832.size); + String _key833; + LogLevel _val834; + for (int _i835 = 0; _i835 < _map832.size; ++_i835) { - _key813 = iprot.readString(); - _val814 = new LogLevel(); - _val814.read(iprot); - struct.named_logger_level.put(_key813, _val814); + _key833 = iprot.readString(); + _val834 = new LogLevel(); + _val834.read(iprot); + struct.named_logger_level.put(_key833, _val834); } iprot.readMapEnd(); } @@ -404,10 +404,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LogConfig struct) 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 _iter816 : struct.named_logger_level.entrySet()) + for (Map.Entry _iter836 : struct.named_logger_level.entrySet()) { - oprot.writeString(_iter816.getKey()); - _iter816.getValue().write(oprot); + oprot.writeString(_iter836.getKey()); + _iter836.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -439,10 +439,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LogConfig struct) t if (struct.is_set_named_logger_level()) { { oprot.writeI32(struct.named_logger_level.size()); - for (Map.Entry _iter817 : struct.named_logger_level.entrySet()) + for (Map.Entry _iter837 : struct.named_logger_level.entrySet()) { - oprot.writeString(_iter817.getKey()); - _iter817.getValue().write(oprot); + oprot.writeString(_iter837.getKey()); + _iter837.getValue().write(oprot); } } } @@ -454,16 +454,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, LogConfig struct) th BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map818 = 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(2*_map818.size); - String _key819; - LogLevel _val820; - for (int _i821 = 0; _i821 < _map818.size; ++_i821) + 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(2*_map838.size); + String _key839; + LogLevel _val840; + for (int _i841 = 0; _i841 < _map838.size; ++_i841) { - _key819 = iprot.readString(); - _val820 = new LogLevel(); - _val820.read(iprot); - struct.named_logger_level.put(_key819, _val820); + _key839 = iprot.readString(); + _val840 = new LogLevel(); + _val840.read(iprot); + struct.named_logger_level.put(_key839, _val840); } } struct.set_named_logger_level_isSet(true); 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 f14ddcf9edb..ea4dcc90b76 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java +++ b/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java @@ -18287,14 +18287,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getComponentPending case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list846 = iprot.readListBegin(); - struct.success = new ArrayList(_list846.size); - ProfileRequest _elem847; - for (int _i848 = 0; _i848 < _list846.size; ++_i848) + org.apache.thrift.protocol.TList _list866 = iprot.readListBegin(); + struct.success = new ArrayList(_list866.size); + ProfileRequest _elem867; + for (int _i868 = 0; _i868 < _list866.size; ++_i868) { - _elem847 = new ProfileRequest(); - _elem847.read(iprot); - struct.success.add(_elem847); + _elem867 = new ProfileRequest(); + _elem867.read(iprot); + struct.success.add(_elem867); } iprot.readListEnd(); } @@ -18320,9 +18320,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getComponentPendin oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (ProfileRequest _iter849 : struct.success) + for (ProfileRequest _iter869 : struct.success) { - _iter849.write(oprot); + _iter869.write(oprot); } oprot.writeListEnd(); } @@ -18353,9 +18353,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getComponentPending if (struct.is_set_success()) { { oprot.writeI32(struct.success.size()); - for (ProfileRequest _iter850 : struct.success) + for (ProfileRequest _iter870 : struct.success) { - _iter850.write(oprot); + _iter870.write(oprot); } } } @@ -18367,14 +18367,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getComponentPendingP BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list851 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list851.size); - ProfileRequest _elem852; - for (int _i853 = 0; _i853 < _list851.size; ++_i853) + org.apache.thrift.protocol.TList _list871 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list871.size); + ProfileRequest _elem872; + for (int _i873 = 0; _i873 < _list871.size; ++_i873) { - _elem852 = new ProfileRequest(); - _elem852.read(iprot); - struct.success.add(_elem852); + _elem872 = new ProfileRequest(); + _elem872.read(iprot); + struct.success.add(_elem872); } } struct.set_success_isSet(true); @@ -48137,14 +48137,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getOwnerResourceSum case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list854 = iprot.readListBegin(); - struct.success = new ArrayList(_list854.size); - OwnerResourceSummary _elem855; - for (int _i856 = 0; _i856 < _list854.size; ++_i856) + org.apache.thrift.protocol.TList _list874 = iprot.readListBegin(); + struct.success = new ArrayList(_list874.size); + OwnerResourceSummary _elem875; + for (int _i876 = 0; _i876 < _list874.size; ++_i876) { - _elem855 = new OwnerResourceSummary(); - _elem855.read(iprot); - struct.success.add(_elem855); + _elem875 = new OwnerResourceSummary(); + _elem875.read(iprot); + struct.success.add(_elem875); } iprot.readListEnd(); } @@ -48179,9 +48179,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getOwnerResourceSu oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (OwnerResourceSummary _iter857 : struct.success) + for (OwnerResourceSummary _iter877 : struct.success) { - _iter857.write(oprot); + _iter877.write(oprot); } oprot.writeListEnd(); } @@ -48220,9 +48220,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSum if (struct.is_set_success()) { { oprot.writeI32(struct.success.size()); - for (OwnerResourceSummary _iter858 : struct.success) + for (OwnerResourceSummary _iter878 : struct.success) { - _iter858.write(oprot); + _iter878.write(oprot); } } } @@ -48237,14 +48237,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSumm BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list859 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list859.size); - OwnerResourceSummary _elem860; - for (int _i861 = 0; _i861 < _list859.size; ++_i861) + org.apache.thrift.protocol.TList _list879 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list879.size); + OwnerResourceSummary _elem880; + for (int _i881 = 0; _i881 < _list879.size; ++_i881) { - _elem860 = new OwnerResourceSummary(); - _elem860.read(iprot); - struct.success.add(_elem860); + _elem880 = new OwnerResourceSummary(); + _elem880.read(iprot); + struct.success.add(_elem880); } } struct.set_success_isSet(true); diff --git a/storm-client/src/jvm/org/apache/storm/generated/NodeInfo.java b/storm-client/src/jvm/org/apache/storm/generated/NodeInfo.java index 403892c42e8..6bb9c1f2304 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/NodeInfo.java +++ b/storm-client/src/jvm/org/apache/storm/generated/NodeInfo.java @@ -461,13 +461,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, NodeInfo struct) th case 2: // PORT if (schemeField.type == org.apache.thrift.protocol.TType.SET) { { - org.apache.thrift.protocol.TSet _set638 = iprot.readSetBegin(); - struct.port = new HashSet(2*_set638.size); - long _elem639; - for (int _i640 = 0; _i640 < _set638.size; ++_i640) + org.apache.thrift.protocol.TSet _set658 = iprot.readSetBegin(); + struct.port = new HashSet(2*_set658.size); + long _elem659; + for (int _i660 = 0; _i660 < _set658.size; ++_i660) { - _elem639 = iprot.readI64(); - struct.port.add(_elem639); + _elem659 = iprot.readI64(); + struct.port.add(_elem659); } iprot.readSetEnd(); } @@ -498,9 +498,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, NodeInfo struct) t oprot.writeFieldBegin(PORT_FIELD_DESC); { oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.port.size())); - for (long _iter641 : struct.port) + for (long _iter661 : struct.port) { - oprot.writeI64(_iter641); + oprot.writeI64(_iter661); } oprot.writeSetEnd(); } @@ -526,9 +526,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, NodeInfo struct) th oprot.writeString(struct.node); { oprot.writeI32(struct.port.size()); - for (long _iter642 : struct.port) + for (long _iter662 : struct.port) { - oprot.writeI64(_iter642); + oprot.writeI64(_iter662); } } } @@ -539,13 +539,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, NodeInfo struct) thr struct.node = iprot.readString(); struct.set_node_isSet(true); { - org.apache.thrift.protocol.TSet _set643 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.port = new HashSet(2*_set643.size); - long _elem644; - for (int _i645 = 0; _i645 < _set643.size; ++_i645) + org.apache.thrift.protocol.TSet _set663 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.port = new HashSet(2*_set663.size); + long _elem664; + for (int _i665 = 0; _i665 < _set663.size; ++_i665) { - _elem644 = iprot.readI64(); - struct.port.add(_elem644); + _elem664 = iprot.readI64(); + struct.port.add(_elem664); } } struct.set_port_isSet(true); diff --git a/storm-client/src/jvm/org/apache/storm/generated/RebalanceOptions.java b/storm-client/src/jvm/org/apache/storm/generated/RebalanceOptions.java index ea7c25625e3..9cd8a2e95b9 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/RebalanceOptions.java +++ b/storm-client/src/jvm/org/apache/storm/generated/RebalanceOptions.java @@ -58,6 +58,9 @@ public class RebalanceOptions implements org.apache.thrift.TBase, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -68,12 +71,18 @@ public class RebalanceOptions implements org.apache.thrift.TBase num_executors; // optional + private Map> topology_resources_overrides; // optional + private String topology_conf_overrides; // optional + private String principal; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { WAIT_SECS((short)1, "wait_secs"), NUM_WORKERS((short)2, "num_workers"), - NUM_EXECUTORS((short)3, "num_executors"); + NUM_EXECUTORS((short)3, "num_executors"), + TOPOLOGY_RESOURCES_OVERRIDES((short)4, "topology_resources_overrides"), + TOPOLOGY_CONF_OVERRIDES((short)5, "topology_conf_overrides"), + PRINCIPAL((short)6, "principal"); private static final Map byName = new HashMap(); @@ -94,6 +103,12 @@ public static _Fields findByThriftId(int fieldId) { return NUM_WORKERS; case 3: // NUM_EXECUTORS return NUM_EXECUTORS; + case 4: // TOPOLOGY_RESOURCES_OVERRIDES + return TOPOLOGY_RESOURCES_OVERRIDES; + case 5: // TOPOLOGY_CONF_OVERRIDES + return TOPOLOGY_CONF_OVERRIDES; + case 6: // PRINCIPAL + return PRINCIPAL; default: return null; } @@ -137,7 +152,7 @@ public String getFieldName() { private static final int __WAIT_SECS_ISSET_ID = 0; private static final int __NUM_WORKERS_ISSET_ID = 1; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.WAIT_SECS,_Fields.NUM_WORKERS,_Fields.NUM_EXECUTORS}; + private static final _Fields optionals[] = {_Fields.WAIT_SECS,_Fields.NUM_WORKERS,_Fields.NUM_EXECUTORS,_Fields.TOPOLOGY_RESOURCES_OVERRIDES,_Fields.TOPOLOGY_CONF_OVERRIDES,_Fields.PRINCIPAL}; 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); @@ -149,6 +164,16 @@ public String getFieldName() { new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); + tmpMap.put(_Fields.TOPOLOGY_RESOURCES_OVERRIDES, new org.apache.thrift.meta_data.FieldMetaData("topology_resources_overrides", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))))); + tmpMap.put(_Fields.TOPOLOGY_CONF_OVERRIDES, new org.apache.thrift.meta_data.FieldMetaData("topology_conf_overrides", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RebalanceOptions.class, metaDataMap); } @@ -167,6 +192,27 @@ public RebalanceOptions(RebalanceOptions other) { Map __this__num_executors = new HashMap(other.num_executors); this.num_executors = __this__num_executors; } + if (other.is_set_topology_resources_overrides()) { + Map> __this__topology_resources_overrides = new HashMap>(other.topology_resources_overrides.size()); + for (Map.Entry> other_element : other.topology_resources_overrides.entrySet()) { + + String other_element_key = other_element.getKey(); + Map other_element_value = other_element.getValue(); + + String __this__topology_resources_overrides_copy_key = other_element_key; + + Map __this__topology_resources_overrides_copy_value = new HashMap(other_element_value); + + __this__topology_resources_overrides.put(__this__topology_resources_overrides_copy_key, __this__topology_resources_overrides_copy_value); + } + this.topology_resources_overrides = __this__topology_resources_overrides; + } + if (other.is_set_topology_conf_overrides()) { + this.topology_conf_overrides = other.topology_conf_overrides; + } + if (other.is_set_principal()) { + this.principal = other.principal; + } } public RebalanceOptions deepCopy() { @@ -180,6 +226,9 @@ public void clear() { set_num_workers_isSet(false); this.num_workers = 0; this.num_executors = null; + this.topology_resources_overrides = null; + this.topology_conf_overrides = null; + this.principal = null; } public int get_wait_secs() { @@ -260,6 +309,86 @@ public void set_num_executors_isSet(boolean value) { } } + public int get_topology_resources_overrides_size() { + return (this.topology_resources_overrides == null) ? 0 : this.topology_resources_overrides.size(); + } + + public void put_to_topology_resources_overrides(String key, Map val) { + if (this.topology_resources_overrides == null) { + this.topology_resources_overrides = new HashMap>(); + } + this.topology_resources_overrides.put(key, val); + } + + public Map> get_topology_resources_overrides() { + return this.topology_resources_overrides; + } + + public void set_topology_resources_overrides(Map> topology_resources_overrides) { + this.topology_resources_overrides = topology_resources_overrides; + } + + public void unset_topology_resources_overrides() { + this.topology_resources_overrides = null; + } + + /** Returns true if field topology_resources_overrides is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology_resources_overrides() { + return this.topology_resources_overrides != null; + } + + public void set_topology_resources_overrides_isSet(boolean value) { + if (!value) { + this.topology_resources_overrides = null; + } + } + + public String get_topology_conf_overrides() { + return this.topology_conf_overrides; + } + + public void set_topology_conf_overrides(String topology_conf_overrides) { + this.topology_conf_overrides = topology_conf_overrides; + } + + public void unset_topology_conf_overrides() { + this.topology_conf_overrides = null; + } + + /** Returns true if field topology_conf_overrides is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology_conf_overrides() { + return this.topology_conf_overrides != null; + } + + public void set_topology_conf_overrides_isSet(boolean value) { + if (!value) { + this.topology_conf_overrides = null; + } + } + + public String get_principal() { + return this.principal; + } + + public void set_principal(String principal) { + this.principal = principal; + } + + public void unset_principal() { + this.principal = null; + } + + /** Returns true if field principal is set (has been assigned a value) and false otherwise */ + public boolean is_set_principal() { + return this.principal != null; + } + + public void set_principal_isSet(boolean value) { + if (!value) { + this.principal = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case WAIT_SECS: @@ -286,6 +415,30 @@ public void setFieldValue(_Fields field, Object value) { } break; + case TOPOLOGY_RESOURCES_OVERRIDES: + if (value == null) { + unset_topology_resources_overrides(); + } else { + set_topology_resources_overrides((Map>)value); + } + break; + + case TOPOLOGY_CONF_OVERRIDES: + if (value == null) { + unset_topology_conf_overrides(); + } else { + set_topology_conf_overrides((String)value); + } + break; + + case PRINCIPAL: + if (value == null) { + unset_principal(); + } else { + set_principal((String)value); + } + break; + } } @@ -300,6 +453,15 @@ public Object getFieldValue(_Fields field) { case NUM_EXECUTORS: return get_num_executors(); + case TOPOLOGY_RESOURCES_OVERRIDES: + return get_topology_resources_overrides(); + + case TOPOLOGY_CONF_OVERRIDES: + return get_topology_conf_overrides(); + + case PRINCIPAL: + return get_principal(); + } throw new IllegalStateException(); } @@ -317,6 +479,12 @@ public boolean isSet(_Fields field) { return is_set_num_workers(); case NUM_EXECUTORS: return is_set_num_executors(); + case TOPOLOGY_RESOURCES_OVERRIDES: + return is_set_topology_resources_overrides(); + case TOPOLOGY_CONF_OVERRIDES: + return is_set_topology_conf_overrides(); + case PRINCIPAL: + return is_set_principal(); } throw new IllegalStateException(); } @@ -361,6 +529,33 @@ public boolean equals(RebalanceOptions that) { return false; } + boolean this_present_topology_resources_overrides = true && this.is_set_topology_resources_overrides(); + boolean that_present_topology_resources_overrides = true && that.is_set_topology_resources_overrides(); + if (this_present_topology_resources_overrides || that_present_topology_resources_overrides) { + if (!(this_present_topology_resources_overrides && that_present_topology_resources_overrides)) + return false; + if (!this.topology_resources_overrides.equals(that.topology_resources_overrides)) + return false; + } + + boolean this_present_topology_conf_overrides = true && this.is_set_topology_conf_overrides(); + boolean that_present_topology_conf_overrides = true && that.is_set_topology_conf_overrides(); + if (this_present_topology_conf_overrides || that_present_topology_conf_overrides) { + if (!(this_present_topology_conf_overrides && that_present_topology_conf_overrides)) + return false; + if (!this.topology_conf_overrides.equals(that.topology_conf_overrides)) + return false; + } + + boolean this_present_principal = true && this.is_set_principal(); + boolean that_present_principal = true && that.is_set_principal(); + if (this_present_principal || that_present_principal) { + if (!(this_present_principal && that_present_principal)) + return false; + if (!this.principal.equals(that.principal)) + return false; + } + return true; } @@ -383,6 +578,21 @@ public int hashCode() { if (present_num_executors) list.add(num_executors); + boolean present_topology_resources_overrides = true && (is_set_topology_resources_overrides()); + list.add(present_topology_resources_overrides); + if (present_topology_resources_overrides) + list.add(topology_resources_overrides); + + boolean present_topology_conf_overrides = true && (is_set_topology_conf_overrides()); + list.add(present_topology_conf_overrides); + if (present_topology_conf_overrides) + list.add(topology_conf_overrides); + + boolean present_principal = true && (is_set_principal()); + list.add(present_principal); + if (present_principal) + list.add(principal); + return list.hashCode(); } @@ -424,6 +634,36 @@ public int compareTo(RebalanceOptions other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_topology_resources_overrides()).compareTo(other.is_set_topology_resources_overrides()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology_resources_overrides()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_resources_overrides, other.topology_resources_overrides); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topology_conf_overrides()).compareTo(other.is_set_topology_conf_overrides()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology_conf_overrides()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_conf_overrides, other.topology_conf_overrides); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_principal()).compareTo(other.is_set_principal()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_principal()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.principal, other.principal); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -465,6 +705,36 @@ public String toString() { } first = false; } + if (is_set_topology_resources_overrides()) { + if (!first) sb.append(", "); + sb.append("topology_resources_overrides:"); + if (this.topology_resources_overrides == null) { + sb.append("null"); + } else { + sb.append(this.topology_resources_overrides); + } + first = false; + } + if (is_set_topology_conf_overrides()) { + if (!first) sb.append(", "); + sb.append("topology_conf_overrides:"); + if (this.topology_conf_overrides == null) { + sb.append("null"); + } else { + sb.append(this.topology_conf_overrides); + } + first = false; + } + if (is_set_principal()) { + if (!first) sb.append(", "); + sb.append("principal:"); + if (this.principal == null) { + sb.append("null"); + } else { + sb.append(this.principal); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -546,6 +816,54 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, RebalanceOptions st org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 4: // TOPOLOGY_RESOURCES_OVERRIDES + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map570 = iprot.readMapBegin(); + struct.topology_resources_overrides = new HashMap>(2*_map570.size); + String _key571; + Map _val572; + for (int _i573 = 0; _i573 < _map570.size; ++_i573) + { + _key571 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map574 = iprot.readMapBegin(); + _val572 = new HashMap(2*_map574.size); + String _key575; + double _val576; + for (int _i577 = 0; _i577 < _map574.size; ++_i577) + { + _key575 = iprot.readString(); + _val576 = iprot.readDouble(); + _val572.put(_key575, _val576); + } + iprot.readMapEnd(); + } + struct.topology_resources_overrides.put(_key571, _val572); + } + iprot.readMapEnd(); + } + struct.set_topology_resources_overrides_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // TOPOLOGY_CONF_OVERRIDES + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topology_conf_overrides = iprot.readString(); + struct.set_topology_conf_overrides_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // PRINCIPAL + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.principal = iprot.readString(); + struct.set_principal_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -574,16 +892,53 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, RebalanceOptions s oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.num_executors.size())); - for (Map.Entry _iter570 : struct.num_executors.entrySet()) + for (Map.Entry _iter578 : struct.num_executors.entrySet()) { - oprot.writeString(_iter570.getKey()); - oprot.writeI32(_iter570.getValue()); + oprot.writeString(_iter578.getKey()); + oprot.writeI32(_iter578.getValue()); } oprot.writeMapEnd(); } oprot.writeFieldEnd(); } } + if (struct.topology_resources_overrides != null) { + if (struct.is_set_topology_resources_overrides()) { + oprot.writeFieldBegin(TOPOLOGY_RESOURCES_OVERRIDES_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.topology_resources_overrides.size())); + for (Map.Entry> _iter579 : struct.topology_resources_overrides.entrySet()) + { + oprot.writeString(_iter579.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, _iter579.getValue().size())); + for (Map.Entry _iter580 : _iter579.getValue().entrySet()) + { + oprot.writeString(_iter580.getKey()); + oprot.writeDouble(_iter580.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.topology_conf_overrides != null) { + if (struct.is_set_topology_conf_overrides()) { + oprot.writeFieldBegin(TOPOLOGY_CONF_OVERRIDES_FIELD_DESC); + oprot.writeString(struct.topology_conf_overrides); + oprot.writeFieldEnd(); + } + } + if (struct.principal != null) { + if (struct.is_set_principal()) { + oprot.writeFieldBegin(PRINCIPAL_FIELD_DESC); + oprot.writeString(struct.principal); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -611,7 +966,16 @@ public void write(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions st if (struct.is_set_num_executors()) { optionals.set(2); } - oprot.writeBitSet(optionals, 3); + if (struct.is_set_topology_resources_overrides()) { + optionals.set(3); + } + if (struct.is_set_topology_conf_overrides()) { + optionals.set(4); + } + if (struct.is_set_principal()) { + optionals.set(5); + } + oprot.writeBitSet(optionals, 6); if (struct.is_set_wait_secs()) { oprot.writeI32(struct.wait_secs); } @@ -621,19 +985,42 @@ public void write(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions st if (struct.is_set_num_executors()) { { oprot.writeI32(struct.num_executors.size()); - for (Map.Entry _iter571 : struct.num_executors.entrySet()) + for (Map.Entry _iter581 : struct.num_executors.entrySet()) + { + oprot.writeString(_iter581.getKey()); + oprot.writeI32(_iter581.getValue()); + } + } + } + if (struct.is_set_topology_resources_overrides()) { + { + oprot.writeI32(struct.topology_resources_overrides.size()); + for (Map.Entry> _iter582 : struct.topology_resources_overrides.entrySet()) { - oprot.writeString(_iter571.getKey()); - oprot.writeI32(_iter571.getValue()); + oprot.writeString(_iter582.getKey()); + { + oprot.writeI32(_iter582.getValue().size()); + for (Map.Entry _iter583 : _iter582.getValue().entrySet()) + { + oprot.writeString(_iter583.getKey()); + oprot.writeDouble(_iter583.getValue()); + } + } } } } + if (struct.is_set_topology_conf_overrides()) { + oprot.writeString(struct.topology_conf_overrides); + } + if (struct.is_set_principal()) { + oprot.writeString(struct.principal); + } } @Override public void read(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(3); + BitSet incoming = iprot.readBitSet(6); if (incoming.get(0)) { struct.wait_secs = iprot.readI32(); struct.set_wait_secs_isSet(true); @@ -644,19 +1031,53 @@ public void read(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions str } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map572 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.num_executors = new HashMap(2*_map572.size); - String _key573; - int _val574; - for (int _i575 = 0; _i575 < _map572.size; ++_i575) + org.apache.thrift.protocol.TMap _map584 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.num_executors = new HashMap(2*_map584.size); + String _key585; + int _val586; + for (int _i587 = 0; _i587 < _map584.size; ++_i587) { - _key573 = iprot.readString(); - _val574 = iprot.readI32(); - struct.num_executors.put(_key573, _val574); + _key585 = iprot.readString(); + _val586 = iprot.readI32(); + struct.num_executors.put(_key585, _val586); } } struct.set_num_executors_isSet(true); } + if (incoming.get(3)) { + { + org.apache.thrift.protocol.TMap _map588 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.topology_resources_overrides = new HashMap>(2*_map588.size); + String _key589; + Map _val590; + for (int _i591 = 0; _i591 < _map588.size; ++_i591) + { + _key589 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map592 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); + _val590 = new HashMap(2*_map592.size); + String _key593; + double _val594; + for (int _i595 = 0; _i595 < _map592.size; ++_i595) + { + _key593 = iprot.readString(); + _val594 = iprot.readDouble(); + _val590.put(_key593, _val594); + } + } + struct.topology_resources_overrides.put(_key589, _val590); + } + } + struct.set_topology_resources_overrides_isSet(true); + } + if (incoming.get(4)) { + struct.topology_conf_overrides = iprot.readString(); + struct.set_topology_conf_overrides_isSet(true); + } + if (incoming.get(5)) { + struct.principal = iprot.readString(); + struct.set_principal_isSet(true); + } } } diff --git a/storm-client/src/jvm/org/apache/storm/generated/SettableBlobMeta.java b/storm-client/src/jvm/org/apache/storm/generated/SettableBlobMeta.java index 48e1fd49004..604e446000f 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/SettableBlobMeta.java +++ b/storm-client/src/jvm/org/apache/storm/generated/SettableBlobMeta.java @@ -452,14 +452,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SettableBlobMeta st case 1: // ACL if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list586 = iprot.readListBegin(); - struct.acl = new ArrayList(_list586.size); - AccessControl _elem587; - for (int _i588 = 0; _i588 < _list586.size; ++_i588) + org.apache.thrift.protocol.TList _list606 = iprot.readListBegin(); + struct.acl = new ArrayList(_list606.size); + AccessControl _elem607; + for (int _i608 = 0; _i608 < _list606.size; ++_i608) { - _elem587 = new AccessControl(); - _elem587.read(iprot); - struct.acl.add(_elem587); + _elem607 = new AccessControl(); + _elem607.read(iprot); + struct.acl.add(_elem607); } iprot.readListEnd(); } @@ -493,9 +493,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SettableBlobMeta s oprot.writeFieldBegin(ACL_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.acl.size())); - for (AccessControl _iter589 : struct.acl) + for (AccessControl _iter609 : struct.acl) { - _iter589.write(oprot); + _iter609.write(oprot); } oprot.writeListEnd(); } @@ -525,9 +525,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SettableBlobMeta st TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.acl.size()); - for (AccessControl _iter590 : struct.acl) + for (AccessControl _iter610 : struct.acl) { - _iter590.write(oprot); + _iter610.write(oprot); } } BitSet optionals = new BitSet(); @@ -544,14 +544,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SettableBlobMeta st public void read(org.apache.thrift.protocol.TProtocol prot, SettableBlobMeta struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list591 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.acl = new ArrayList(_list591.size); - AccessControl _elem592; - for (int _i593 = 0; _i593 < _list591.size; ++_i593) + org.apache.thrift.protocol.TList _list611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.acl = new ArrayList(_list611.size); + AccessControl _elem612; + for (int _i613 = 0; _i613 < _list611.size; ++_i613) { - _elem592 = new AccessControl(); - _elem592.read(iprot); - struct.acl.add(_elem592); + _elem612 = new AccessControl(); + _elem612.read(iprot); + struct.acl.add(_elem612); } } struct.set_acl_isSet(true); 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 84051f13677..b1b205c1f90 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 void read(org.apache.thrift.protocol.TProtocol iprot, StormBase struct) t case 4: // COMPONENT_EXECUTORS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map712 = iprot.readMapBegin(); - struct.component_executors = new HashMap(2*_map712.size); - String _key713; - int _val714; - for (int _i715 = 0; _i715 < _map712.size; ++_i715) + org.apache.thrift.protocol.TMap _map732 = iprot.readMapBegin(); + struct.component_executors = new HashMap(2*_map732.size); + String _key733; + int _val734; + for (int _i735 = 0; _i735 < _map732.size; ++_i735) { - _key713 = iprot.readString(); - _val714 = iprot.readI32(); - struct.component_executors.put(_key713, _val714); + _key733 = iprot.readString(); + _val734 = iprot.readI32(); + struct.component_executors.put(_key733, _val734); } iprot.readMapEnd(); } @@ -1305,16 +1305,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, StormBase struct) t case 9: // COMPONENT_DEBUG if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map716 = iprot.readMapBegin(); - struct.component_debug = new HashMap(2*_map716.size); - String _key717; - DebugOptions _val718; - for (int _i719 = 0; _i719 < _map716.size; ++_i719) + org.apache.thrift.protocol.TMap _map736 = iprot.readMapBegin(); + struct.component_debug = new HashMap(2*_map736.size); + String _key737; + DebugOptions _val738; + for (int _i739 = 0; _i739 < _map736.size; ++_i739) { - _key717 = iprot.readString(); - _val718 = new DebugOptions(); - _val718.read(iprot); - struct.component_debug.put(_key717, _val718); + _key737 = iprot.readString(); + _val738 = new DebugOptions(); + _val738.read(iprot); + struct.component_debug.put(_key737, _val738); } iprot.readMapEnd(); } @@ -1370,10 +1370,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, StormBase struct) 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 _iter720 : struct.component_executors.entrySet()) + for (Map.Entry _iter740 : struct.component_executors.entrySet()) { - oprot.writeString(_iter720.getKey()); - oprot.writeI32(_iter720.getValue()); + oprot.writeString(_iter740.getKey()); + oprot.writeI32(_iter740.getValue()); } oprot.writeMapEnd(); } @@ -1411,10 +1411,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, StormBase struct) 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 _iter721 : struct.component_debug.entrySet()) + for (Map.Entry _iter741 : struct.component_debug.entrySet()) { - oprot.writeString(_iter721.getKey()); - _iter721.getValue().write(oprot); + oprot.writeString(_iter741.getKey()); + _iter741.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -1484,10 +1484,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, StormBase struct) t if (struct.is_set_component_executors()) { { oprot.writeI32(struct.component_executors.size()); - for (Map.Entry _iter722 : struct.component_executors.entrySet()) + for (Map.Entry _iter742 : struct.component_executors.entrySet()) { - oprot.writeString(_iter722.getKey()); - oprot.writeI32(_iter722.getValue()); + oprot.writeString(_iter742.getKey()); + oprot.writeI32(_iter742.getValue()); } } } @@ -1506,10 +1506,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, StormBase struct) t if (struct.is_set_component_debug()) { { oprot.writeI32(struct.component_debug.size()); - for (Map.Entry _iter723 : struct.component_debug.entrySet()) + for (Map.Entry _iter743 : struct.component_debug.entrySet()) { - oprot.writeString(_iter723.getKey()); - _iter723.getValue().write(oprot); + oprot.writeString(_iter743.getKey()); + _iter743.getValue().write(oprot); } } } @@ -1533,15 +1533,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, StormBase struct) th BitSet incoming = iprot.readBitSet(8); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map724 = 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(2*_map724.size); - String _key725; - int _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.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.component_executors = new HashMap(2*_map744.size); + String _key745; + int _val746; + for (int _i747 = 0; _i747 < _map744.size; ++_i747) { - _key725 = iprot.readString(); - _val726 = iprot.readI32(); - struct.component_executors.put(_key725, _val726); + _key745 = iprot.readString(); + _val746 = iprot.readI32(); + struct.component_executors.put(_key745, _val746); } } struct.set_component_executors_isSet(true); @@ -1565,16 +1565,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, StormBase struct) th } if (incoming.get(5)) { { - org.apache.thrift.protocol.TMap _map728 = 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(2*_map728.size); - String _key729; - DebugOptions _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.STRUCT, iprot.readI32()); + struct.component_debug = new HashMap(2*_map748.size); + String _key749; + DebugOptions _val750; + for (int _i751 = 0; _i751 < _map748.size; ++_i751) { - _key729 = iprot.readString(); - _val730 = new DebugOptions(); - _val730.read(iprot); - struct.component_debug.put(_key729, _val730); + _key749 = iprot.readString(); + _val750 = new DebugOptions(); + _val750.read(iprot); + struct.component_debug.put(_key749, _val750); } } struct.set_component_debug_isSet(true); diff --git a/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java b/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java index d19bac44761..45293ed59ab 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java +++ b/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java @@ -1085,13 +1085,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru case 4: // USED_PORTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list602 = iprot.readListBegin(); - struct.used_ports = new ArrayList(_list602.size); - long _elem603; - for (int _i604 = 0; _i604 < _list602.size; ++_i604) + org.apache.thrift.protocol.TList _list622 = iprot.readListBegin(); + struct.used_ports = new ArrayList(_list622.size); + long _elem623; + for (int _i624 = 0; _i624 < _list622.size; ++_i624) { - _elem603 = iprot.readI64(); - struct.used_ports.add(_elem603); + _elem623 = iprot.readI64(); + struct.used_ports.add(_elem623); } iprot.readListEnd(); } @@ -1103,13 +1103,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru case 5: // META if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list605 = iprot.readListBegin(); - struct.meta = new ArrayList(_list605.size); - long _elem606; - for (int _i607 = 0; _i607 < _list605.size; ++_i607) + org.apache.thrift.protocol.TList _list625 = iprot.readListBegin(); + struct.meta = new ArrayList(_list625.size); + long _elem626; + for (int _i627 = 0; _i627 < _list625.size; ++_i627) { - _elem606 = iprot.readI64(); - struct.meta.add(_elem606); + _elem626 = iprot.readI64(); + struct.meta.add(_elem626); } iprot.readListEnd(); } @@ -1121,15 +1121,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru case 6: // SCHEDULER_META if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map608 = iprot.readMapBegin(); - struct.scheduler_meta = new HashMap(2*_map608.size); - String _key609; - String _val610; - for (int _i611 = 0; _i611 < _map608.size; ++_i611) + org.apache.thrift.protocol.TMap _map628 = iprot.readMapBegin(); + struct.scheduler_meta = new HashMap(2*_map628.size); + String _key629; + String _val630; + for (int _i631 = 0; _i631 < _map628.size; ++_i631) { - _key609 = iprot.readString(); - _val610 = iprot.readString(); - struct.scheduler_meta.put(_key609, _val610); + _key629 = iprot.readString(); + _val630 = iprot.readString(); + struct.scheduler_meta.put(_key629, _val630); } iprot.readMapEnd(); } @@ -1157,15 +1157,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru case 9: // RESOURCES_MAP if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map612 = iprot.readMapBegin(); - struct.resources_map = new HashMap(2*_map612.size); - String _key613; - double _val614; - for (int _i615 = 0; _i615 < _map612.size; ++_i615) + org.apache.thrift.protocol.TMap _map632 = iprot.readMapBegin(); + struct.resources_map = new HashMap(2*_map632.size); + String _key633; + double _val634; + for (int _i635 = 0; _i635 < _map632.size; ++_i635) { - _key613 = iprot.readString(); - _val614 = iprot.readDouble(); - struct.resources_map.put(_key613, _val614); + _key633 = iprot.readString(); + _val634 = iprot.readDouble(); + struct.resources_map.put(_key633, _val634); } iprot.readMapEnd(); } @@ -1207,9 +1207,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo str oprot.writeFieldBegin(USED_PORTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.used_ports.size())); - for (long _iter616 : struct.used_ports) + for (long _iter636 : struct.used_ports) { - oprot.writeI64(_iter616); + oprot.writeI64(_iter636); } oprot.writeListEnd(); } @@ -1221,9 +1221,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo str oprot.writeFieldBegin(META_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.meta.size())); - for (long _iter617 : struct.meta) + for (long _iter637 : struct.meta) { - oprot.writeI64(_iter617); + oprot.writeI64(_iter637); } oprot.writeListEnd(); } @@ -1235,10 +1235,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo str oprot.writeFieldBegin(SCHEDULER_META_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.scheduler_meta.size())); - for (Map.Entry _iter618 : struct.scheduler_meta.entrySet()) + for (Map.Entry _iter638 : struct.scheduler_meta.entrySet()) { - oprot.writeString(_iter618.getKey()); - oprot.writeString(_iter618.getValue()); + oprot.writeString(_iter638.getKey()); + oprot.writeString(_iter638.getValue()); } oprot.writeMapEnd(); } @@ -1262,10 +1262,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo str oprot.writeFieldBegin(RESOURCES_MAP_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.resources_map.size())); - for (Map.Entry _iter619 : struct.resources_map.entrySet()) + for (Map.Entry _iter639 : struct.resources_map.entrySet()) { - oprot.writeString(_iter619.getKey()); - oprot.writeDouble(_iter619.getValue()); + oprot.writeString(_iter639.getKey()); + oprot.writeDouble(_iter639.getValue()); } oprot.writeMapEnd(); } @@ -1320,28 +1320,28 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo stru if (struct.is_set_used_ports()) { { oprot.writeI32(struct.used_ports.size()); - for (long _iter620 : struct.used_ports) + for (long _iter640 : struct.used_ports) { - oprot.writeI64(_iter620); + oprot.writeI64(_iter640); } } } if (struct.is_set_meta()) { { oprot.writeI32(struct.meta.size()); - for (long _iter621 : struct.meta) + for (long _iter641 : struct.meta) { - oprot.writeI64(_iter621); + oprot.writeI64(_iter641); } } } if (struct.is_set_scheduler_meta()) { { oprot.writeI32(struct.scheduler_meta.size()); - for (Map.Entry _iter622 : struct.scheduler_meta.entrySet()) + for (Map.Entry _iter642 : struct.scheduler_meta.entrySet()) { - oprot.writeString(_iter622.getKey()); - oprot.writeString(_iter622.getValue()); + oprot.writeString(_iter642.getKey()); + oprot.writeString(_iter642.getValue()); } } } @@ -1354,10 +1354,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo stru if (struct.is_set_resources_map()) { { oprot.writeI32(struct.resources_map.size()); - for (Map.Entry _iter623 : struct.resources_map.entrySet()) + for (Map.Entry _iter643 : struct.resources_map.entrySet()) { - oprot.writeString(_iter623.getKey()); - oprot.writeDouble(_iter623.getValue()); + oprot.writeString(_iter643.getKey()); + oprot.writeDouble(_iter643.getValue()); } } } @@ -1377,41 +1377,41 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struc } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list624 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.used_ports = new ArrayList(_list624.size); - long _elem625; - for (int _i626 = 0; _i626 < _list624.size; ++_i626) + org.apache.thrift.protocol.TList _list644 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.used_ports = new ArrayList(_list644.size); + long _elem645; + for (int _i646 = 0; _i646 < _list644.size; ++_i646) { - _elem625 = iprot.readI64(); - struct.used_ports.add(_elem625); + _elem645 = iprot.readI64(); + struct.used_ports.add(_elem645); } } struct.set_used_ports_isSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list627 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.meta = new ArrayList(_list627.size); - long _elem628; - for (int _i629 = 0; _i629 < _list627.size; ++_i629) + org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.meta = new ArrayList(_list647.size); + long _elem648; + for (int _i649 = 0; _i649 < _list647.size; ++_i649) { - _elem628 = iprot.readI64(); - struct.meta.add(_elem628); + _elem648 = iprot.readI64(); + struct.meta.add(_elem648); } } struct.set_meta_isSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map630 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.scheduler_meta = new HashMap(2*_map630.size); - String _key631; - String _val632; - for (int _i633 = 0; _i633 < _map630.size; ++_i633) + org.apache.thrift.protocol.TMap _map650 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.scheduler_meta = new HashMap(2*_map650.size); + String _key651; + String _val652; + for (int _i653 = 0; _i653 < _map650.size; ++_i653) { - _key631 = iprot.readString(); - _val632 = iprot.readString(); - struct.scheduler_meta.put(_key631, _val632); + _key651 = iprot.readString(); + _val652 = iprot.readString(); + struct.scheduler_meta.put(_key651, _val652); } } struct.set_scheduler_meta_isSet(true); @@ -1426,15 +1426,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struc } if (incoming.get(6)) { { - org.apache.thrift.protocol.TMap _map634 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); - struct.resources_map = new HashMap(2*_map634.size); - String _key635; - double _val636; - for (int _i637 = 0; _i637 < _map634.size; ++_i637) + org.apache.thrift.protocol.TMap _map654 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); + struct.resources_map = new HashMap(2*_map654.size); + String _key655; + double _val656; + for (int _i657 = 0; _i657 < _map654.size; ++_i657) { - _key635 = iprot.readString(); - _val636 = iprot.readDouble(); - struct.resources_map.put(_key635, _val636); + _key655 = iprot.readString(); + _val656 = iprot.readDouble(); + struct.resources_map.put(_key655, _val656); } } struct.set_resources_map_isSet(true); 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 58cf9bb4e5d..0fc7cab6828 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 void read(org.apache.thrift.protocol.TProtocol iprot, TopologyHistoryInfo case 1: // TOPO_IDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list822 = iprot.readListBegin(); - struct.topo_ids = new ArrayList(_list822.size); - String _elem823; - for (int _i824 = 0; _i824 < _list822.size; ++_i824) + org.apache.thrift.protocol.TList _list842 = iprot.readListBegin(); + struct.topo_ids = new ArrayList(_list842.size); + String _elem843; + for (int _i844 = 0; _i844 < _list842.size; ++_i844) { - _elem823 = iprot.readString(); - struct.topo_ids.add(_elem823); + _elem843 = iprot.readString(); + struct.topo_ids.add(_elem843); } iprot.readListEnd(); } @@ -396,9 +396,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyHistoryInf 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 _iter825 : struct.topo_ids) + for (String _iter845 : struct.topo_ids) { - oprot.writeString(_iter825); + oprot.writeString(_iter845); } oprot.writeListEnd(); } @@ -429,9 +429,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TopologyHistoryInfo if (struct.is_set_topo_ids()) { { oprot.writeI32(struct.topo_ids.size()); - for (String _iter826 : struct.topo_ids) + for (String _iter846 : struct.topo_ids) { - oprot.writeString(_iter826); + oprot.writeString(_iter846); } } } @@ -443,13 +443,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TopologyHistoryInfo BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list827 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.topo_ids = new ArrayList(_list827.size); - String _elem828; - for (int _i829 = 0; _i829 < _list827.size; ++_i829) + 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(_list847.size); + String _elem848; + for (int _i849 = 0; _i849 < _list847.size; ++_i849) { - _elem828 = iprot.readString(); - struct.topo_ids.add(_elem828); + _elem848 = iprot.readString(); + struct.topo_ids.add(_elem848); } } struct.set_topo_ids_isSet(true); diff --git a/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java b/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java index 99ef9cd9fd9..971e5f349d9 100644 --- a/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java +++ b/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java @@ -46,7 +46,6 @@ import org.apache.storm.utils.Utils; import org.apache.storm.windowing.TupleWindow; import org.json.simple.JSONValue; -import org.json.simple.parser.ParseException; import java.io.NotSerializableException; import java.nio.ByteBuffer; @@ -577,7 +576,7 @@ public T addConfigurations(Map conf) { throw new IllegalArgumentException("Cannot set serializations for a component using fluent API"); } String currConf = _commons.get(_id).get_json_conf(); - _commons.get(_id).set_json_conf(mergeIntoJson(parseJson(currConf), conf)); + _commons.get(_id).set_json_conf(mergeIntoJson(Utils.parseJson(currConf), conf)); return (T) this; } @@ -695,19 +694,7 @@ public BoltDeclarer grouping(GlobalStreamId id, Grouping grouping) { return grouping(id.get_componentId(), id.get_streamId(), grouping); } } - - private static Map parseJson(String json) { - if (json==null) { - return new HashMap(); - } else { - try { - return (Map) JSONValue.parseWithException(json); - } catch (ParseException e) { - throw new RuntimeException(e); - } - } - } - + private static String mergeIntoJson(Map into, Map newMap) { Map res = new HashMap(into); if(newMap!=null) res.putAll(newMap); 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 0a38f61927d..37c29630c79 100644 --- a/storm-client/src/jvm/org/apache/storm/utils/Utils.java +++ b/storm-client/src/jvm/org/apache/storm/utils/Utils.java @@ -905,7 +905,7 @@ public static HashMap reverseMap(List listSeq) { /** * parses the arguments to extract jvm heap memory size in MB. - * @param input + * @param options * @param defaultValue * @return the value of the JVM heap memory setting (in MB) in a java command. */ @@ -1288,6 +1288,18 @@ public static T findOne (IPredicate pred, Map map) { return findOne(pred, (Set) map.entrySet()); } + public static Map parseJson(String json) { + if (json==null) { + return new HashMap<>(); + } else { + try { + return (Map) JSONValue.parseWithException(json); + } catch (ParseException e) { + throw new RuntimeException(e); + } + } + } + // Non-static impl methods exist for mocking purposes. protected void forceDeleteImpl(String path) throws IOException { LOG.debug("Deleting path {}", path); diff --git a/storm-client/src/py/storm/Nimbus.py b/storm-client/src/py/storm/Nimbus.py index 545b05d764b..522921b1c66 100644 --- a/storm-client/src/py/storm/Nimbus.py +++ b/storm-client/src/py/storm/Nimbus.py @@ -4977,11 +4977,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype758, _size755) = iprot.readListBegin() - for _i759 in xrange(_size755): - _elem760 = ProfileRequest() - _elem760.read(iprot) - self.success.append(_elem760) + (_etype776, _size773) = iprot.readListBegin() + for _i777 in xrange(_size773): + _elem778 = ProfileRequest() + _elem778.read(iprot) + self.success.append(_elem778) iprot.readListEnd() else: iprot.skip(ftype) @@ -4998,8 +4998,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter761 in self.success: - iter761.write(oprot) + for iter779 in self.success: + iter779.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -10070,11 +10070,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype765, _size762) = iprot.readListBegin() - for _i766 in xrange(_size762): - _elem767 = OwnerResourceSummary() - _elem767.read(iprot) - self.success.append(_elem767) + (_etype783, _size780) = iprot.readListBegin() + for _i784 in xrange(_size780): + _elem785 = OwnerResourceSummary() + _elem785.read(iprot) + self.success.append(_elem785) iprot.readListEnd() else: iprot.skip(ftype) @@ -10097,8 +10097,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter768 in self.success: - iter768.write(oprot) + for iter786 in self.success: + iter786.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.aze is not None: diff --git a/storm-client/src/py/storm/ttypes.py b/storm-client/src/py/storm/ttypes.py index 0daf3ffc594..6b744c6ed72 100644 --- a/storm-client/src/py/storm/ttypes.py +++ b/storm-client/src/py/storm/ttypes.py @@ -8355,6 +8355,9 @@ class RebalanceOptions: - wait_secs - num_workers - num_executors + - topology_resources_overrides + - topology_conf_overrides + - principal """ thrift_spec = ( @@ -8362,12 +8365,18 @@ class RebalanceOptions: (1, TType.I32, 'wait_secs', None, None, ), # 1 (2, TType.I32, 'num_workers', None, None, ), # 2 (3, TType.MAP, 'num_executors', (TType.STRING,None,TType.I32,None), None, ), # 3 + (4, TType.MAP, 'topology_resources_overrides', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.DOUBLE,None)), None, ), # 4 + (5, TType.STRING, 'topology_conf_overrides', None, None, ), # 5 + (6, TType.STRING, 'principal', None, None, ), # 6 ) - def __init__(self, wait_secs=None, num_workers=None, num_executors=None,): + def __init__(self, wait_secs=None, num_workers=None, num_executors=None, topology_resources_overrides=None, topology_conf_overrides=None, principal=None,): self.wait_secs = wait_secs self.num_workers = num_workers self.num_executors = num_executors + self.topology_resources_overrides = topology_resources_overrides + self.topology_conf_overrides = topology_conf_overrides + self.principal = principal 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: @@ -8399,6 +8408,33 @@ def read(self, iprot): iprot.readMapEnd() else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.MAP: + self.topology_resources_overrides = {} + (_ktype514, _vtype515, _size513 ) = iprot.readMapBegin() + for _i517 in xrange(_size513): + _key518 = iprot.readString().decode('utf-8') + _val519 = {} + (_ktype521, _vtype522, _size520 ) = iprot.readMapBegin() + for _i524 in xrange(_size520): + _key525 = iprot.readString().decode('utf-8') + _val526 = iprot.readDouble() + _val519[_key525] = _val526 + iprot.readMapEnd() + self.topology_resources_overrides[_key518] = _val519 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.topology_conf_overrides = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.STRING: + self.principal = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8420,11 +8456,31 @@ def write(self, oprot): if self.num_executors is not None: oprot.writeFieldBegin('num_executors', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.I32, len(self.num_executors)) - for kiter513,viter514 in self.num_executors.items(): - oprot.writeString(kiter513.encode('utf-8')) - oprot.writeI32(viter514) + for kiter527,viter528 in self.num_executors.items(): + oprot.writeString(kiter527.encode('utf-8')) + oprot.writeI32(viter528) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.topology_resources_overrides is not None: + oprot.writeFieldBegin('topology_resources_overrides', TType.MAP, 4) + oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.topology_resources_overrides)) + for kiter529,viter530 in self.topology_resources_overrides.items(): + oprot.writeString(kiter529.encode('utf-8')) + oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(viter530)) + for kiter531,viter532 in viter530.items(): + oprot.writeString(kiter531.encode('utf-8')) + oprot.writeDouble(viter532) + oprot.writeMapEnd() oprot.writeMapEnd() oprot.writeFieldEnd() + if self.topology_conf_overrides is not None: + oprot.writeFieldBegin('topology_conf_overrides', TType.STRING, 5) + oprot.writeString(self.topology_conf_overrides.encode('utf-8')) + oprot.writeFieldEnd() + if self.principal is not None: + oprot.writeFieldBegin('principal', TType.STRING, 6) + oprot.writeString(self.principal.encode('utf-8')) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8437,6 +8493,9 @@ def __hash__(self): value = (value * 31) ^ hash(self.wait_secs) value = (value * 31) ^ hash(self.num_workers) value = (value * 31) ^ hash(self.num_executors) + value = (value * 31) ^ hash(self.topology_resources_overrides) + value = (value * 31) ^ hash(self.topology_conf_overrides) + value = (value * 31) ^ hash(self.principal) return value def __repr__(self): @@ -8476,11 +8535,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.creds = {} - (_ktype516, _vtype517, _size515 ) = iprot.readMapBegin() - for _i519 in xrange(_size515): - _key520 = iprot.readString().decode('utf-8') - _val521 = iprot.readString().decode('utf-8') - self.creds[_key520] = _val521 + (_ktype534, _vtype535, _size533 ) = iprot.readMapBegin() + for _i537 in xrange(_size533): + _key538 = iprot.readString().decode('utf-8') + _val539 = iprot.readString().decode('utf-8') + self.creds[_key538] = _val539 iprot.readMapEnd() else: iprot.skip(ftype) @@ -8497,9 +8556,9 @@ def write(self, oprot): if self.creds is not None: oprot.writeFieldBegin('creds', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds)) - for kiter522,viter523 in self.creds.items(): - oprot.writeString(kiter522.encode('utf-8')) - oprot.writeString(viter523.encode('utf-8')) + for kiter540,viter541 in self.creds.items(): + oprot.writeString(kiter540.encode('utf-8')) + oprot.writeString(viter541.encode('utf-8')) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -8732,11 +8791,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.acl = [] - (_etype527, _size524) = iprot.readListBegin() - for _i528 in xrange(_size524): - _elem529 = AccessControl() - _elem529.read(iprot) - self.acl.append(_elem529) + (_etype545, _size542) = iprot.readListBegin() + for _i546 in xrange(_size542): + _elem547 = AccessControl() + _elem547.read(iprot) + self.acl.append(_elem547) iprot.readListEnd() else: iprot.skip(ftype) @@ -8758,8 +8817,8 @@ def write(self, oprot): if self.acl is not None: oprot.writeFieldBegin('acl', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.acl)) - for iter530 in self.acl: - iter530.write(oprot) + for iter548 in self.acl: + iter548.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.replication_factor is not None: @@ -8904,10 +8963,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.keys = [] - (_etype534, _size531) = iprot.readListBegin() - for _i535 in xrange(_size531): - _elem536 = iprot.readString().decode('utf-8') - self.keys.append(_elem536) + (_etype552, _size549) = iprot.readListBegin() + for _i553 in xrange(_size549): + _elem554 = iprot.readString().decode('utf-8') + self.keys.append(_elem554) iprot.readListEnd() else: iprot.skip(ftype) @@ -8929,8 +8988,8 @@ def write(self, oprot): if self.keys is not None: oprot.writeFieldBegin('keys', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.keys)) - for iter537 in self.keys: - oprot.writeString(iter537.encode('utf-8')) + for iter555 in self.keys: + oprot.writeString(iter555.encode('utf-8')) oprot.writeListEnd() oprot.writeFieldEnd() if self.session is not None: @@ -9125,31 +9184,31 @@ def read(self, iprot): elif fid == 4: if ftype == TType.LIST: self.used_ports = [] - (_etype541, _size538) = iprot.readListBegin() - for _i542 in xrange(_size538): - _elem543 = iprot.readI64() - self.used_ports.append(_elem543) + (_etype559, _size556) = iprot.readListBegin() + for _i560 in xrange(_size556): + _elem561 = iprot.readI64() + self.used_ports.append(_elem561) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 5: if ftype == TType.LIST: self.meta = [] - (_etype547, _size544) = iprot.readListBegin() - for _i548 in xrange(_size544): - _elem549 = iprot.readI64() - self.meta.append(_elem549) + (_etype565, _size562) = iprot.readListBegin() + for _i566 in xrange(_size562): + _elem567 = iprot.readI64() + self.meta.append(_elem567) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 6: if ftype == TType.MAP: self.scheduler_meta = {} - (_ktype551, _vtype552, _size550 ) = iprot.readMapBegin() - for _i554 in xrange(_size550): - _key555 = iprot.readString().decode('utf-8') - _val556 = iprot.readString().decode('utf-8') - self.scheduler_meta[_key555] = _val556 + (_ktype569, _vtype570, _size568 ) = iprot.readMapBegin() + for _i572 in xrange(_size568): + _key573 = iprot.readString().decode('utf-8') + _val574 = iprot.readString().decode('utf-8') + self.scheduler_meta[_key573] = _val574 iprot.readMapEnd() else: iprot.skip(ftype) @@ -9166,11 +9225,11 @@ def read(self, iprot): elif fid == 9: if ftype == TType.MAP: self.resources_map = {} - (_ktype558, _vtype559, _size557 ) = iprot.readMapBegin() - for _i561 in xrange(_size557): - _key562 = iprot.readString().decode('utf-8') - _val563 = iprot.readDouble() - self.resources_map[_key562] = _val563 + (_ktype576, _vtype577, _size575 ) = iprot.readMapBegin() + for _i579 in xrange(_size575): + _key580 = iprot.readString().decode('utf-8') + _val581 = iprot.readDouble() + self.resources_map[_key580] = _val581 iprot.readMapEnd() else: iprot.skip(ftype) @@ -9199,23 +9258,23 @@ def write(self, oprot): if self.used_ports is not None: oprot.writeFieldBegin('used_ports', TType.LIST, 4) oprot.writeListBegin(TType.I64, len(self.used_ports)) - for iter564 in self.used_ports: - oprot.writeI64(iter564) + for iter582 in self.used_ports: + oprot.writeI64(iter582) oprot.writeListEnd() oprot.writeFieldEnd() if self.meta is not None: oprot.writeFieldBegin('meta', TType.LIST, 5) oprot.writeListBegin(TType.I64, len(self.meta)) - for iter565 in self.meta: - oprot.writeI64(iter565) + for iter583 in self.meta: + oprot.writeI64(iter583) oprot.writeListEnd() oprot.writeFieldEnd() if self.scheduler_meta is not None: oprot.writeFieldBegin('scheduler_meta', TType.MAP, 6) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.scheduler_meta)) - for kiter566,viter567 in self.scheduler_meta.items(): - oprot.writeString(kiter566.encode('utf-8')) - oprot.writeString(viter567.encode('utf-8')) + for kiter584,viter585 in self.scheduler_meta.items(): + oprot.writeString(kiter584.encode('utf-8')) + oprot.writeString(viter585.encode('utf-8')) oprot.writeMapEnd() oprot.writeFieldEnd() if self.uptime_secs is not None: @@ -9229,9 +9288,9 @@ def write(self, oprot): if self.resources_map is not None: oprot.writeFieldBegin('resources_map', TType.MAP, 9) oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.resources_map)) - for kiter568,viter569 in self.resources_map.items(): - oprot.writeString(kiter568.encode('utf-8')) - oprot.writeDouble(viter569) + for kiter586,viter587 in self.resources_map.items(): + oprot.writeString(kiter586.encode('utf-8')) + oprot.writeDouble(viter587) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -9303,10 +9362,10 @@ def read(self, iprot): elif fid == 2: if ftype == TType.SET: self.port = set() - (_etype573, _size570) = iprot.readSetBegin() - for _i574 in xrange(_size570): - _elem575 = iprot.readI64() - self.port.add(_elem575) + (_etype591, _size588) = iprot.readSetBegin() + for _i592 in xrange(_size588): + _elem593 = iprot.readI64() + self.port.add(_elem593) iprot.readSetEnd() else: iprot.skip(ftype) @@ -9327,8 +9386,8 @@ def write(self, oprot): if self.port is not None: oprot.writeFieldBegin('port', TType.SET, 2) oprot.writeSetBegin(TType.I64, len(self.port)) - for iter576 in self.port: - oprot.writeI64(iter576) + for iter594 in self.port: + oprot.writeI64(iter594) oprot.writeSetEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -9545,68 +9604,68 @@ def read(self, iprot): elif fid == 2: if ftype == TType.MAP: self.node_host = {} - (_ktype578, _vtype579, _size577 ) = iprot.readMapBegin() - for _i581 in xrange(_size577): - _key582 = iprot.readString().decode('utf-8') - _val583 = iprot.readString().decode('utf-8') - self.node_host[_key582] = _val583 + (_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 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.MAP: self.executor_node_port = {} - (_ktype585, _vtype586, _size584 ) = iprot.readMapBegin() - for _i588 in xrange(_size584): - _key589 = [] - (_etype594, _size591) = iprot.readListBegin() - for _i595 in xrange(_size591): - _elem596 = iprot.readI64() - _key589.append(_elem596) + (_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) iprot.readListEnd() - _val590 = NodeInfo() - _val590.read(iprot) - self.executor_node_port[_key589] = _val590 + _val608 = NodeInfo() + _val608.read(iprot) + self.executor_node_port[_key607] = _val608 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.MAP: self.executor_start_time_secs = {} - (_ktype598, _vtype599, _size597 ) = iprot.readMapBegin() - for _i601 in xrange(_size597): - _key602 = [] - (_etype607, _size604) = iprot.readListBegin() - for _i608 in xrange(_size604): - _elem609 = iprot.readI64() - _key602.append(_elem609) + (_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) iprot.readListEnd() - _val603 = iprot.readI64() - self.executor_start_time_secs[_key602] = _val603 + _val621 = iprot.readI64() + self.executor_start_time_secs[_key620] = _val621 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 5: if ftype == TType.MAP: self.worker_resources = {} - (_ktype611, _vtype612, _size610 ) = iprot.readMapBegin() - for _i614 in xrange(_size610): - _key615 = NodeInfo() - _key615.read(iprot) - _val616 = WorkerResources() - _val616.read(iprot) - self.worker_resources[_key615] = _val616 + (_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 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 6: if ftype == TType.MAP: self.total_shared_off_heap = {} - (_ktype618, _vtype619, _size617 ) = iprot.readMapBegin() - for _i621 in xrange(_size617): - _key622 = iprot.readString().decode('utf-8') - _val623 = iprot.readDouble() - self.total_shared_off_heap[_key622] = _val623 + (_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 iprot.readMapEnd() else: iprot.skip(ftype) @@ -9632,47 +9691,47 @@ def write(self, oprot): 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 kiter624,viter625 in self.node_host.items(): - oprot.writeString(kiter624.encode('utf-8')) - oprot.writeString(viter625.encode('utf-8')) + for kiter642,viter643 in self.node_host.items(): + oprot.writeString(kiter642.encode('utf-8')) + oprot.writeString(viter643.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 kiter626,viter627 in self.executor_node_port.items(): - oprot.writeListBegin(TType.I64, len(kiter626)) - for iter628 in kiter626: - oprot.writeI64(iter628) + for kiter644,viter645 in self.executor_node_port.items(): + oprot.writeListBegin(TType.I64, len(kiter644)) + for iter646 in kiter644: + oprot.writeI64(iter646) oprot.writeListEnd() - viter627.write(oprot) + viter645.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 kiter629,viter630 in self.executor_start_time_secs.items(): - oprot.writeListBegin(TType.I64, len(kiter629)) - for iter631 in kiter629: - oprot.writeI64(iter631) + for kiter647,viter648 in self.executor_start_time_secs.items(): + oprot.writeListBegin(TType.I64, len(kiter647)) + for iter649 in kiter647: + oprot.writeI64(iter649) oprot.writeListEnd() - oprot.writeI64(viter630) + oprot.writeI64(viter648) 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 kiter632,viter633 in self.worker_resources.items(): - kiter632.write(oprot) - viter633.write(oprot) + for kiter650,viter651 in self.worker_resources.items(): + kiter650.write(oprot) + viter651.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 kiter634,viter635 in self.total_shared_off_heap.items(): - oprot.writeString(kiter634.encode('utf-8')) - oprot.writeDouble(viter635) + for kiter652,viter653 in self.total_shared_off_heap.items(): + oprot.writeString(kiter652.encode('utf-8')) + oprot.writeDouble(viter653) oprot.writeMapEnd() oprot.writeFieldEnd() if self.owner is not None: @@ -9861,11 +9920,11 @@ def read(self, iprot): elif fid == 4: if ftype == TType.MAP: self.component_executors = {} - (_ktype637, _vtype638, _size636 ) = iprot.readMapBegin() - for _i640 in xrange(_size636): - _key641 = iprot.readString().decode('utf-8') - _val642 = iprot.readI32() - self.component_executors[_key641] = _val642 + (_ktype655, _vtype656, _size654 ) = iprot.readMapBegin() + for _i658 in xrange(_size654): + _key659 = iprot.readString().decode('utf-8') + _val660 = iprot.readI32() + self.component_executors[_key659] = _val660 iprot.readMapEnd() else: iprot.skip(ftype) @@ -9893,12 +9952,12 @@ def read(self, iprot): elif fid == 9: if ftype == TType.MAP: self.component_debug = {} - (_ktype644, _vtype645, _size643 ) = iprot.readMapBegin() - for _i647 in xrange(_size643): - _key648 = iprot.readString().decode('utf-8') - _val649 = DebugOptions() - _val649.read(iprot) - self.component_debug[_key648] = _val649 + (_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 iprot.readMapEnd() else: iprot.skip(ftype) @@ -9937,9 +9996,9 @@ def write(self, oprot): 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 kiter650,viter651 in self.component_executors.items(): - oprot.writeString(kiter650.encode('utf-8')) - oprot.writeI32(viter651) + for kiter668,viter669 in self.component_executors.items(): + oprot.writeString(kiter668.encode('utf-8')) + oprot.writeI32(viter669) oprot.writeMapEnd() oprot.writeFieldEnd() if self.launch_time_secs is not None: @@ -9961,9 +10020,9 @@ def write(self, oprot): 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 kiter652,viter653 in self.component_debug.items(): - oprot.writeString(kiter652.encode('utf-8')) - viter653.write(oprot) + for kiter670,viter671 in self.component_debug.items(): + oprot.writeString(kiter670.encode('utf-8')) + viter671.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.principal is not None: @@ -10053,13 +10112,13 @@ def read(self, iprot): elif fid == 2: if ftype == TType.MAP: self.executor_stats = {} - (_ktype655, _vtype656, _size654 ) = iprot.readMapBegin() - for _i658 in xrange(_size654): - _key659 = ExecutorInfo() - _key659.read(iprot) - _val660 = ExecutorStats() - _val660.read(iprot) - self.executor_stats[_key659] = _val660 + (_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 iprot.readMapEnd() else: iprot.skip(ftype) @@ -10090,9 +10149,9 @@ def write(self, oprot): 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 kiter661,viter662 in self.executor_stats.items(): - kiter661.write(oprot) - viter662.write(oprot) + for kiter679,viter680 in self.executor_stats.items(): + kiter679.write(oprot) + viter680.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.time_secs is not None: @@ -10245,12 +10304,12 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.serialized_parts = {} - (_ktype664, _vtype665, _size663 ) = iprot.readMapBegin() - for _i667 in xrange(_size663): - _key668 = iprot.readString().decode('utf-8') - _val669 = ThriftSerializedObject() - _val669.read(iprot) - self.serialized_parts[_key668] = _val669 + (_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 iprot.readMapEnd() else: iprot.skip(ftype) @@ -10267,9 +10326,9 @@ def write(self, oprot): 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 kiter670,viter671 in self.serialized_parts.items(): - oprot.writeString(kiter670.encode('utf-8')) - viter671.write(oprot) + for kiter688,viter689 in self.serialized_parts.items(): + oprot.writeString(kiter688.encode('utf-8')) + viter689.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -10340,11 +10399,11 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.executors = [] - (_etype675, _size672) = iprot.readListBegin() - for _i676 in xrange(_size672): - _elem677 = ExecutorInfo() - _elem677.read(iprot) - self.executors.append(_elem677) + (_etype693, _size690) = iprot.readListBegin() + for _i694 in xrange(_size690): + _elem695 = ExecutorInfo() + _elem695.read(iprot) + self.executors.append(_elem695) iprot.readListEnd() else: iprot.skip(ftype) @@ -10381,8 +10440,8 @@ def write(self, oprot): if self.executors is not None: oprot.writeFieldBegin('executors', TType.LIST, 2) oprot.writeListBegin(TType.STRUCT, len(self.executors)) - for iter678 in self.executors: - iter678.write(oprot) + for iter696 in self.executors: + iter696.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.resources is not None: @@ -10521,11 +10580,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.approved_workers = {} - (_ktype680, _vtype681, _size679 ) = iprot.readMapBegin() - for _i683 in xrange(_size679): - _key684 = iprot.readString().decode('utf-8') - _val685 = iprot.readI32() - self.approved_workers[_key684] = _val685 + (_ktype698, _vtype699, _size697 ) = iprot.readMapBegin() + for _i701 in xrange(_size697): + _key702 = iprot.readString().decode('utf-8') + _val703 = iprot.readI32() + self.approved_workers[_key702] = _val703 iprot.readMapEnd() else: iprot.skip(ftype) @@ -10542,9 +10601,9 @@ def write(self, oprot): 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 kiter686,viter687 in self.approved_workers.items(): - oprot.writeString(kiter686.encode('utf-8')) - oprot.writeI32(viter687) + for kiter704,viter705 in self.approved_workers.items(): + oprot.writeString(kiter704.encode('utf-8')) + oprot.writeI32(viter705) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -10598,12 +10657,12 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.assignments = {} - (_ktype689, _vtype690, _size688 ) = iprot.readMapBegin() - for _i692 in xrange(_size688): - _key693 = iprot.readI32() - _val694 = LocalAssignment() - _val694.read(iprot) - self.assignments[_key693] = _val694 + (_ktype707, _vtype708, _size706 ) = iprot.readMapBegin() + for _i710 in xrange(_size706): + _key711 = iprot.readI32() + _val712 = LocalAssignment() + _val712.read(iprot) + self.assignments[_key711] = _val712 iprot.readMapEnd() else: iprot.skip(ftype) @@ -10620,9 +10679,9 @@ def write(self, oprot): if self.assignments is not None: oprot.writeFieldBegin('assignments', TType.MAP, 1) oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.assignments)) - for kiter695,viter696 in self.assignments.items(): - oprot.writeI32(kiter695) - viter696.write(oprot) + for kiter713,viter714 in self.assignments.items(): + oprot.writeI32(kiter713) + viter714.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -10695,11 +10754,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.executors = [] - (_etype700, _size697) = iprot.readListBegin() - for _i701 in xrange(_size697): - _elem702 = ExecutorInfo() - _elem702.read(iprot) - self.executors.append(_elem702) + (_etype718, _size715) = iprot.readListBegin() + for _i719 in xrange(_size715): + _elem720 = ExecutorInfo() + _elem720.read(iprot) + self.executors.append(_elem720) iprot.readListEnd() else: iprot.skip(ftype) @@ -10729,8 +10788,8 @@ def write(self, oprot): if self.executors is not None: oprot.writeFieldBegin('executors', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.executors)) - for iter703 in self.executors: - iter703.write(oprot) + for iter721 in self.executors: + iter721.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.port is not None: @@ -10816,20 +10875,20 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.users = [] - (_etype707, _size704) = iprot.readListBegin() - for _i708 in xrange(_size704): - _elem709 = iprot.readString().decode('utf-8') - self.users.append(_elem709) + (_etype725, _size722) = iprot.readListBegin() + for _i726 in xrange(_size722): + _elem727 = iprot.readString().decode('utf-8') + self.users.append(_elem727) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.LIST: self.groups = [] - (_etype713, _size710) = iprot.readListBegin() - for _i714 in xrange(_size710): - _elem715 = iprot.readString().decode('utf-8') - self.groups.append(_elem715) + (_etype731, _size728) = iprot.readListBegin() + for _i732 in xrange(_size728): + _elem733 = iprot.readString().decode('utf-8') + self.groups.append(_elem733) iprot.readListEnd() else: iprot.skip(ftype) @@ -10854,15 +10913,15 @@ def write(self, oprot): if self.users is not None: oprot.writeFieldBegin('users', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.users)) - for iter716 in self.users: - oprot.writeString(iter716.encode('utf-8')) + for iter734 in self.users: + oprot.writeString(iter734.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 iter717 in self.groups: - oprot.writeString(iter717.encode('utf-8')) + for iter735 in self.groups: + oprot.writeString(iter735.encode('utf-8')) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -10925,11 +10984,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.topo_history = [] - (_etype721, _size718) = iprot.readListBegin() - for _i722 in xrange(_size718): - _elem723 = LSTopoHistory() - _elem723.read(iprot) - self.topo_history.append(_elem723) + (_etype739, _size736) = iprot.readListBegin() + for _i740 in xrange(_size736): + _elem741 = LSTopoHistory() + _elem741.read(iprot) + self.topo_history.append(_elem741) iprot.readListEnd() else: iprot.skip(ftype) @@ -10946,8 +11005,8 @@ def write(self, oprot): if self.topo_history is not None: oprot.writeFieldBegin('topo_history', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.topo_history)) - for iter724 in self.topo_history: - iter724.write(oprot) + for iter742 in self.topo_history: + iter742.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -11282,12 +11341,12 @@ def read(self, iprot): if fid == 2: if ftype == TType.MAP: self.named_logger_level = {} - (_ktype726, _vtype727, _size725 ) = iprot.readMapBegin() - for _i729 in xrange(_size725): - _key730 = iprot.readString().decode('utf-8') - _val731 = LogLevel() - _val731.read(iprot) - self.named_logger_level[_key730] = _val731 + (_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 iprot.readMapEnd() else: iprot.skip(ftype) @@ -11304,9 +11363,9 @@ def write(self, oprot): 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 kiter732,viter733 in self.named_logger_level.items(): - oprot.writeString(kiter732.encode('utf-8')) - viter733.write(oprot) + for kiter750,viter751 in self.named_logger_level.items(): + oprot.writeString(kiter750.encode('utf-8')) + viter751.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -11358,10 +11417,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.topo_ids = [] - (_etype737, _size734) = iprot.readListBegin() - for _i738 in xrange(_size734): - _elem739 = iprot.readString().decode('utf-8') - self.topo_ids.append(_elem739) + (_etype755, _size752) = iprot.readListBegin() + for _i756 in xrange(_size752): + _elem757 = iprot.readString().decode('utf-8') + self.topo_ids.append(_elem757) iprot.readListEnd() else: iprot.skip(ftype) @@ -11378,8 +11437,8 @@ def write(self, oprot): if self.topo_ids is not None: oprot.writeFieldBegin('topo_ids', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.topo_ids)) - for iter740 in self.topo_ids: - oprot.writeString(iter740.encode('utf-8')) + for iter758 in self.topo_ids: + oprot.writeString(iter758.encode('utf-8')) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -11964,11 +12023,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.pulses = [] - (_etype744, _size741) = iprot.readListBegin() - for _i745 in xrange(_size741): - _elem746 = HBPulse() - _elem746.read(iprot) - self.pulses.append(_elem746) + (_etype762, _size759) = iprot.readListBegin() + for _i763 in xrange(_size759): + _elem764 = HBPulse() + _elem764.read(iprot) + self.pulses.append(_elem764) iprot.readListEnd() else: iprot.skip(ftype) @@ -11985,8 +12044,8 @@ def write(self, oprot): if self.pulses is not None: oprot.writeFieldBegin('pulses', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.pulses)) - for iter747 in self.pulses: - iter747.write(oprot) + for iter765 in self.pulses: + iter765.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -12038,10 +12097,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.pulseIds = [] - (_etype751, _size748) = iprot.readListBegin() - for _i752 in xrange(_size748): - _elem753 = iprot.readString().decode('utf-8') - self.pulseIds.append(_elem753) + (_etype769, _size766) = iprot.readListBegin() + for _i770 in xrange(_size766): + _elem771 = iprot.readString().decode('utf-8') + self.pulseIds.append(_elem771) iprot.readListEnd() else: iprot.skip(ftype) @@ -12058,8 +12117,8 @@ def write(self, oprot): if self.pulseIds is not None: oprot.writeFieldBegin('pulseIds', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.pulseIds)) - for iter754 in self.pulseIds: - oprot.writeString(iter754.encode('utf-8')) + for iter772 in self.pulseIds: + oprot.writeString(iter772.encode('utf-8')) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() diff --git a/storm-client/src/storm.thrift b/storm-client/src/storm.thrift index 11879d126ce..c21bcaaa203 100644 --- a/storm-client/src/storm.thrift +++ b/storm-client/src/storm.thrift @@ -416,6 +416,10 @@ struct RebalanceOptions { 1: optional i32 wait_secs; 2: optional i32 num_workers; 3: optional map num_executors; + 4: optional map> topology_resources_overrides; + 5: optional string topology_conf_overrides; + //This value is not intended to be explicitly set by end users and will be ignored if they do + 6: optional string principal } struct Credentials { diff --git a/storm-client/test/jvm/org/apache/storm/blobstore/ClientBlobStoreTest.java b/storm-client/test/jvm/org/apache/storm/blobstore/ClientBlobStoreTest.java index 1a5d7b0dfca..ee701d37f41 100644 --- a/storm-client/test/jvm/org/apache/storm/blobstore/ClientBlobStoreTest.java +++ b/storm-client/test/jvm/org/apache/storm/blobstore/ClientBlobStoreTest.java @@ -89,6 +89,10 @@ public Iterator listKeys() { public void shutdown() { } + @Override + public void close() { + } + @Override public int getBlobReplication(String key) { return -1; diff --git a/storm-core/src/jvm/org/apache/storm/command/Rebalance.java b/storm-core/src/jvm/org/apache/storm/command/Rebalance.java index 343658c1058..ec34798f029 100644 --- a/storm-core/src/jvm/org/apache/storm/command/Rebalance.java +++ b/storm-core/src/jvm/org/apache/storm/command/Rebalance.java @@ -21,6 +21,8 @@ import org.apache.storm.generated.Nimbus; import org.apache.storm.generated.RebalanceOptions; import org.apache.storm.utils.NimbusClient; +import org.apache.storm.utils.Utils; +import org.json.simple.JSONValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,6 +39,8 @@ public static void main(String[] args) throws Exception { Map cl = CLI.opt("w", "wait", null, CLI.AS_INT) .opt("n", "num-workers", null, CLI.AS_INT) .opt("e", "executor", null, new ExecutorParser(), CLI.INTO_MAP) + .opt("r", "resources", null, new ResourcesParser(), CLI.INTO_MAP) + .opt("t", "topology-conf", null, new ConfParser(), CLI.INTO_MAP) .arg("topologyName", CLI.FIRST_WINS) .parse(args); final String name = (String) cl.get("topologyName"); @@ -44,6 +48,8 @@ public static void main(String[] args) throws Exception { Integer wait = (Integer) cl.get("w"); Integer numWorkers = (Integer) cl.get("n"); Map numExecutors = (Map) cl.get("e"); + Map> resourceOverrides = (Map>) cl.get("r"); + Map confOverrides = (Map) cl.get("t"); if (null != wait) { rebalanceOptions.set_wait_secs(wait); @@ -55,6 +61,14 @@ public static void main(String[] args) throws Exception { rebalanceOptions.set_num_executors(numExecutors); } + if (null != resourceOverrides) { + rebalanceOptions.set_topology_resources_overrides(resourceOverrides); + } + + if (null != confOverrides) { + rebalanceOptions.set_topology_conf_overrides(JSONValue.toJSONString(confOverrides)); + } + NimbusClient.withConfiguredClient(new NimbusClient.WithNimbus() { @Override public void run(Nimbus.Iface nimbus) throws Exception { @@ -64,9 +78,49 @@ public void run(Nimbus.Iface nimbus) throws Exception { }); } + static final class ConfParser implements CLI.Parse { + @Override + public Object parse(String value) { + if (value == null) { + throw new RuntimeException("No arguments found for topology config override!"); + } + try { + return Utils.parseJson(value); + } catch (Exception e) { + throw new RuntimeException("Error trying to parse topology config override", e); + } + } + } + + static final class ResourcesParser implements CLI.Parse { + @Override + public Object parse(String value) { + if (value == null) { + throw new RuntimeException("No arguments found for topology resources override!"); + } + try { + //This is a bit ugly The JSON we are expecting should be in the form + // {"component": {"resource": value, ...}, ...} + // But because value is coming from JSON it is going to be a Number, and we want it to be a Double. + // So the goal is to go through each entry and update it accordingly + Map> ret = new HashMap<>(); + for (Map.Entry compEntry: Utils.parseJson(value).entrySet()) { + String comp = compEntry.getKey(); + Map numResources = (Map) compEntry.getValue(); + Map doubleResource = new HashMap<>(); + for (Map.Entry entry: numResources.entrySet()) { + doubleResource.put(entry.getKey(), entry.getValue().doubleValue()); + } + ret.put(comp, doubleResource); + } + return ret; + } catch (Exception e) { + throw new RuntimeException("Error trying to parse resource override", e); + } + } + } static final class ExecutorParser implements CLI.Parse { - @Override public Object parse(String value) { try { 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 e54aaeb2f45..d050f94d500 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 @@ -1199,6 +1199,7 @@ void doRebalance(String topoId, StormBase stormBase) throws Exception { updated.set_num_workers(rbo.get_num_workers()); } stormClusterState.updateStorm(topoId, updated); + updateBlobStore(topoId, rbo, ServerUtils.principalNameToSubject(rbo.get_principal())); mkAssignments(topoId); } @@ -1275,6 +1276,33 @@ private void setupStormCode(Map conf, String topoId, String tmpJ clusterState.setupBlobstore(codeKey, hostPortInfo, getVersionForKey(codeKey, hostPortInfo, conf)); } } + + private void updateTopologyResources(String topoId, Map> resourceOverrides, Subject subject) + throws AuthorizationException, IOException, KeyNotFoundException { + StormTopology topo = topoCache.readTopology(topoId, subject); + topo = topo.deepCopy(); + ResourceUtils.updateStormTopologyResources(topo, resourceOverrides); + topoCache.updateTopology(topoId, subject, topo); + } + + private void updateTopologyConf(String topoId, Map configOverride, Subject subject) + throws AuthorizationException, IOException, KeyNotFoundException { + Map topoConf = new HashMap<>(topoCache.readTopoConf(topoId, subject)); //Copy the data + topoConf.putAll(configOverride); + topoCache.updateTopoConf(topoId, subject, topoConf); + } + + private void updateBlobStore(String topoId, RebalanceOptions rbo, Subject subject) + throws AuthorizationException, IOException, KeyNotFoundException { + Map> resourceOverrides = rbo.get_topology_resources_overrides(); + if (resourceOverrides != null && !resourceOverrides.isEmpty()) { + updateTopologyResources(topoId, resourceOverrides, subject); + } + String confOverride = rbo.get_topology_conf_overrides(); + if (confOverride != null && !confOverride.isEmpty()) { + updateTopologyConf(topoId, Utils.parseJson(confOverride), subject); + } + } private Integer getBlobReplicationCount(String key) throws Exception { BlobStore store = blobStore; @@ -2763,12 +2791,32 @@ public void rebalance(String topoName, RebalanceOptions options) topoConf = Utils.merge(conf, topoConf); final String operation = "rebalance"; checkAuthorization(topoName, topoConf, operation); + // Set principal in RebalanceOptions to nil because users are not suppose to set this + options.set_principal(null); Map execOverrides = options.is_set_num_executors() ? options.get_num_executors() : Collections.emptyMap(); for (Integer value : execOverrides.values()) { if (value == null || value <= 0) { throw new InvalidTopologyException("Number of executors must be greater than 0"); } } + if (options.is_set_topology_conf_overrides()) { + Map topoConfigOverrides = Utils.parseJson(options.get_topology_conf_overrides()); + //Clean up some things the user should not set. (Not a security issue, just might confuse the topology) + topoConfigOverrides.remove(Config.TOPOLOGY_SUBMITTER_PRINCIPAL); + topoConfigOverrides.remove(Config.TOPOLOGY_SUBMITTER_USER); + topoConfigOverrides.remove(Config.STORM_ZOOKEEPER_SUPERACL); + topoConfigOverrides.remove(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME); + topoConfigOverrides.remove(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD); + if ((boolean) conf.getOrDefault(DaemonConfig.STORM_TOPOLOGY_CLASSPATH_BEGINNING_ENABLED, false)) { + topoConfigOverrides.remove(Config.TOPOLOGY_CLASSPATH_BEGINNING); + } + options.set_topology_conf_overrides(JSONValue.toJSONString(topoConfigOverrides)); + } + Subject subject = getSubject(); + if (subject != null) { + options.set_principal(subject.getPrincipals().iterator().next().getName()); + } + transitionName(topoName, TopologyActions.REBALANCE, options, true); notifyTopologyActionListener(topoName, operation); } catch (Exception e) { diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java index d68e512279a..6e92a9cb9a8 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java @@ -100,13 +100,6 @@ public ReadClusterState(Supervisor supervisor) throws Exception { } catch (Exception e) { LOG.warn("Error trying to clean up old workers", e); } - - //All the slots/assignments should be recovered now, so we can clean up anything that we don't expect to be here - try { - localizer.cleanupUnusedTopologies(); - } catch (Exception e) { - LOG.warn("Error trying to clean up old topologies", e); - } for (Slot slot: slots.values()) { slot.start(); 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 6533d15fb69..da6292534fe 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 @@ -19,7 +19,7 @@ package org.apache.storm.daemon.supervisor; import com.codahale.metrics.Meter; - +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.util.Collections; import java.util.HashMap; @@ -27,12 +27,13 @@ import java.util.Iterator; import java.util.Map; import java.util.Set; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; - import org.apache.storm.Config; import org.apache.storm.DaemonConfig; import org.apache.storm.cluster.IStormClusterState; @@ -44,6 +45,9 @@ import org.apache.storm.generated.ProfileAction; import org.apache.storm.generated.ProfileRequest; import org.apache.storm.localizer.AsyncLocalizer; +import org.apache.storm.localizer.BlobChangingCallback; +import org.apache.storm.localizer.GoodToGo; +import org.apache.storm.localizer.LocallyCachedBlob; import org.apache.storm.metric.StormMetricsRegistry; import org.apache.storm.scheduler.ISupervisor; import org.apache.storm.utils.LocalState; @@ -53,7 +57,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class Slot extends Thread implements AutoCloseable { +public class Slot extends Thread implements AutoCloseable, BlobChangingCallback { private static final Logger LOG = LoggerFactory.getLogger(Slot.class); private static final Meter numWorkersLaunched = StormMetricsRegistry.registerMeter("supervisor:num-workers-launched"); @@ -74,8 +78,9 @@ static enum MachineState { WAITING_FOR_WORKER_START, KILL_AND_RELAUNCH, KILL, - WAITING_FOR_BASIC_LOCALIZATION, - WAITING_FOR_BLOB_LOCALIZATION; + KILL_BLOB_UPDATE, + WAITING_FOR_BLOB_LOCALIZATION, + WAITING_FOR_BLOB_UPDATE; } static class StaticState { @@ -89,11 +94,13 @@ static class StaticState { public final String host; public final ISupervisor iSupervisor; public final LocalState localState; + public final BlobChangingCallback changingCallback; StaticState(AsyncLocalizer localizer, long hbTimeoutMs, long firstHbTimeoutMs, - long killSleepMs, long monitorFreqMs, - ContainerLauncher containerLauncher, String host, int port, - ISupervisor iSupervisor, LocalState localState) { + long killSleepMs, long monitorFreqMs, + ContainerLauncher containerLauncher, String host, int port, + ISupervisor iSupervisor, LocalState localState, + BlobChangingCallback changingCallback) { this.localizer = localizer; this.hbTimeoutMs = hbTimeoutMs; this.firstHbTimeoutMs = firstHbTimeoutMs; @@ -104,9 +111,12 @@ static class StaticState { this.port = port; this.iSupervisor = iSupervisor; this.localState = localState; + this.changingCallback = changingCallback; } } - + + //TODO go through all of the state transitions and make sure we handle changingBlobs + //TODO make sure to add in transition helpers that clean changingBlobs && pendingChangeingBlobs for not the current topology static class DynamicState { public final MachineState state; public final LocalAssignment newAssignment; @@ -116,6 +126,9 @@ static class DynamicState { public final Future pendingDownload; public final Set profileActions; public final Set pendingStopProfileActions; + public final Set changingBlobs; + public final LocalAssignment pendingChangingBlobsAssignment; + public final Set> pendingChangeingBlobs; /** * The last time that WAITING_FOR_WORKER_START, KILL, or KILL_AND_RELAUNCH were entered into. @@ -139,15 +152,22 @@ public DynamicState(final LocalAssignment currentAssignment, Container container this.newAssignment = newAssignment; this.pendingLocalization = null; this.pendingDownload = null; - this.profileActions = new HashSet<>(); - this.pendingStopProfileActions = new HashSet<>(); + this.profileActions = Collections.emptySet(); + this.pendingStopProfileActions = Collections.emptySet(); + this.changingBlobs = Collections.emptySet(); + this.pendingChangingBlobsAssignment = null; + this.pendingChangeingBlobs = Collections.emptySet(); } - + public DynamicState(final MachineState state, final LocalAssignment newAssignment, - final Container container, final LocalAssignment currentAssignment, - final LocalAssignment pendingLocalization, final long startTime, - final Future pendingDownload, final Set profileActions, - final Set pendingStopProfileActions) { + final Container container, final LocalAssignment currentAssignment, + final LocalAssignment pendingLocalization, final long startTime, + final Future pendingDownload, final Set profileActions, + final Set pendingStopProfileActions, + final Set changingBlobs, + final Set> pendingChangingBlobs, final LocalAssignment pendingChaningBlobsAssignment) { + assert pendingChangingBlobs != null; + assert !(pendingChangingBlobs.isEmpty() ^ (pendingChaningBlobsAssignment == null)); this.state = state; this.newAssignment = newAssignment; this.currentAssignment = currentAssignment; @@ -157,6 +177,9 @@ public DynamicState(final MachineState state, final LocalAssignment newAssignmen this.pendingDownload = pendingDownload; this.profileActions = profileActions; this.pendingStopProfileActions = pendingStopProfileActions; + this.changingBlobs = changingBlobs; + this.pendingChangeingBlobs = pendingChangingBlobs; + this.pendingChangingBlobsAssignment = pendingChaningBlobsAssignment; } public String toString() { @@ -179,47 +202,75 @@ public String toString() { */ public DynamicState withNewAssignment(LocalAssignment newAssignment) { return new DynamicState(this.state, newAssignment, - this.container, this.currentAssignment, - this.pendingLocalization, this.startTime, - this.pendingDownload, this.profileActions, - this.pendingStopProfileActions); + this.container, this.currentAssignment, + this.pendingLocalization, this.startTime, + this.pendingDownload, this.profileActions, + this.pendingStopProfileActions, this.changingBlobs, + this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); } - + public DynamicState withPendingLocalization(LocalAssignment pendingLocalization, Future pendingDownload) { return new DynamicState(this.state, this.newAssignment, - this.container, this.currentAssignment, - pendingLocalization, this.startTime, - pendingDownload, this.profileActions, - this.pendingStopProfileActions); + this.container, this.currentAssignment, + pendingLocalization, this.startTime, + pendingDownload, this.profileActions, + this.pendingStopProfileActions, this.changingBlobs, + this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); } public DynamicState withPendingLocalization(Future pendingDownload) { return withPendingLocalization(this.pendingLocalization, pendingDownload); } - + public DynamicState withState(final MachineState state) { long newStartTime = Time.currentTimeMillis(); return new DynamicState(state, this.newAssignment, - this.container, this.currentAssignment, - this.pendingLocalization, newStartTime, - this.pendingDownload, this.profileActions, - this.pendingStopProfileActions); + this.container, this.currentAssignment, + this.pendingLocalization, newStartTime, + this.pendingDownload, this.profileActions, + this.pendingStopProfileActions, this.changingBlobs, + this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); } public DynamicState withCurrentAssignment(final Container container, final LocalAssignment currentAssignment) { return new DynamicState(this.state, this.newAssignment, - container, currentAssignment, - this.pendingLocalization, this.startTime, - this.pendingDownload, this.profileActions, - this.pendingStopProfileActions); + container, currentAssignment, + this.pendingLocalization, this.startTime, + this.pendingDownload, this.profileActions, + this.pendingStopProfileActions, this.changingBlobs, + this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); } public DynamicState withProfileActions(Set profileActions, Set pendingStopProfileActions) { return new DynamicState(this.state, this.newAssignment, - this.container, this.currentAssignment, - this.pendingLocalization, this.startTime, - this.pendingDownload, profileActions, - pendingStopProfileActions); + this.container, this.currentAssignment, + this.pendingLocalization, this.startTime, + this.pendingDownload, profileActions, + pendingStopProfileActions, this.changingBlobs, + this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); + } + + public DynamicState withChangingBlobs(Set changingBlobs) { + if (changingBlobs == this.changingBlobs) { + return this; + } + return new DynamicState(this.state, this.newAssignment, + this.container, this.currentAssignment, + this.pendingLocalization, this.startTime, + this.pendingDownload, profileActions, + this.pendingStopProfileActions, changingBlobs, + this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); + } + + public DynamicState withPendingChangeingBlobs(Set> pendingChangeingBlobs, + LocalAssignment pendingChangeingBlobsAssignment) { + return new DynamicState(this.state, this.newAssignment, + this.container, this.currentAssignment, + this.pendingLocalization, this.startTime, + this.pendingDownload, profileActions, + this.pendingStopProfileActions, this.changingBlobs, + pendingChangeingBlobs, + pendingChangeingBlobsAssignment); } }; @@ -263,7 +314,41 @@ public String toString() { return "{ " + topoId + ": " + request + " }"; } } - + + /** + * Holds the information about a blob that is changing. + */ + static class BlobChangeing { + private final LocalAssignment assignment; + private final LocallyCachedBlob blob; + private final GoodToGo.GoodToGoLatch latch; + + public BlobChangeing(LocalAssignment assignment, LocallyCachedBlob blob, GoodToGo.GoodToGoLatch latch) { + this.assignment = assignment; + this.blob = blob; + this.latch = latch; + } + + @Override + public String toString() { + return "BLOB CHANGING " + blob + " " + assignment; + } + } + + @VisibleForTesting + static boolean forSameTopology(LocalAssignment a, LocalAssignment b) { + if (a == null && b == null) { + return true; + } + if (a != null && b != null) { + if (a.get_topology_id().equals(b.get_topology_id())) { + return true; + } + } + return false; + } + + @VisibleForTesting static boolean equivalent(LocalAssignment a, LocalAssignment b) { if (a == null && b == null) { return true; @@ -298,14 +383,16 @@ static DynamicState stateMachineStep(DynamicState dynamicState, StaticState stat return handleRunning(dynamicState, staticState); case WAITING_FOR_WORKER_START: return handleWaitingForWorkerStart(dynamicState, staticState); + case KILL_BLOB_UPDATE: + return handleKillBlobUpdate(dynamicState, staticState); case KILL_AND_RELAUNCH: return handleKillAndRelaunch(dynamicState, staticState); case KILL: return handleKill(dynamicState, staticState); - case WAITING_FOR_BASIC_LOCALIZATION: - return handleWaitingForBasicLocalization(dynamicState, staticState); case WAITING_FOR_BLOB_LOCALIZATION: return handleWaitingForBlobLocalization(dynamicState, staticState); + case WAITING_FOR_BLOB_UPDATE: + return handleWaitingForBlobUpdate(dynamicState, staticState); default: throw new IllegalStateException("Code not ready to handle a state of "+dynamicState.state); } @@ -325,12 +412,14 @@ static DynamicState prepareForNewAssignmentNoWorkersRunning(DynamicState dynamic if (dynamicState.newAssignment == null) { return dynamicState.withState(MachineState.EMPTY); } - Future pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment, staticState.port); - return dynamicState.withPendingLocalization(dynamicState.newAssignment, pendingDownload).withState(MachineState.WAITING_FOR_BASIC_LOCALIZATION); + Future pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.newAssignment, + staticState.port, staticState.changingCallback); + return dynamicState.withPendingLocalization(dynamicState.newAssignment, pendingDownload) + .withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION); } /** - * Kill the current container and start downloading what the new assignment needs, if there is a new assignment + * Kill the current container and start downloading what the new assignment needs, if there is a new assignment. * PRECONDITION: container != null * @param dynamicState current state * @param staticState static data @@ -339,17 +428,36 @@ static DynamicState prepareForNewAssignmentNoWorkersRunning(DynamicState dynamic */ static DynamicState killContainerForChangedAssignment(DynamicState dynamicState, StaticState staticState) throws Exception { assert(dynamicState.container != null); - + staticState.iSupervisor.killedWorker(staticState.port); dynamicState.container.kill(); Future pendingDownload = null; if (dynamicState.newAssignment != null) { - pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment, staticState.port); + pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.newAssignment, staticState.port, + staticState.changingCallback); } + dynamicState = drainAllChangingBlobs(dynamicState); Time.sleep(staticState.killSleepMs); return dynamicState.withPendingLocalization(dynamicState.newAssignment, pendingDownload).withState(MachineState.KILL); } - + + /** + * Kill the current container, and wait go to the state to inform the localizer that we are ready to go. + * PRECONDITION: container != null + * @param dynamicState current state + * @param staticState static data + * @return the next state + */ + private static DynamicState killContainerForChangedBlobs(DynamicState dynamicState, StaticState staticState) throws Exception { + assert(dynamicState.container != null); + + staticState.iSupervisor.killedWorker(staticState.port); + dynamicState.container.kill(); + + Time.sleep(staticState.killSleepMs); + return dynamicState.withState(MachineState.KILL_BLOB_UPDATE); + } + /** * Kill the current container and relaunch it. (Something odd happened) * PRECONDITION: container != null @@ -367,11 +475,11 @@ static DynamicState killAndRelaunchContainer(DynamicState dynamicState, StaticSt //any stop profile actions that hadn't timed out yet, we should restart after the worker is running again. HashSet mod = new HashSet<>(dynamicState.profileActions); mod.addAll(dynamicState.pendingStopProfileActions); - return dynamicState.withState(MachineState.KILL_AND_RELAUNCH).withProfileActions(mod, Collections. emptySet()); + return dynamicState.withState(MachineState.KILL_AND_RELAUNCH).withProfileActions(mod, Collections.emptySet()); } /** - * Clean up a container + * Clean up a container. * PRECONDITION: All of the processes have died. * @param dynamicState current state * @param staticState static data @@ -391,7 +499,79 @@ static DynamicState cleanupCurrentContainer(DynamicState dynamicState, StaticSta } return ret; } - + + /** + * Drop all of the changingBlobs and pendingChangeingBlobs. + * @param dynamicState current state. + * @return the next state. + */ + private static DynamicState drainAllChangingBlobs(DynamicState dynamicState) { + if (!dynamicState.changingBlobs.isEmpty()) { + for (BlobChangeing rc : dynamicState.changingBlobs) { + rc.latch.countDown(); + } + dynamicState = dynamicState.withChangingBlobs(Collections.emptySet()); + } + + if (!dynamicState.pendingChangeingBlobs.isEmpty()) { + dynamicState = dynamicState.withPendingChangeingBlobs(Collections.emptySet(), null); + } + + return dynamicState; + } + + /** + * Informs the async localizer for all of blobs that the worker is dead. + * + * PRECONDITION: container is null + * PRECONDITION: changingBlobs should only be for the given assignment. + * @param dynamicState the current state + * @return the futures for the current assignment. + */ + private static DynamicState informChangedBlobs(DynamicState dynamicState, LocalAssignment assignment) { + assert dynamicState.container == null; + assert dynamicState.changingBlobs.stream().allMatch((cr) -> forSameTopology(cr.assignment, assignment)); + + Set> futures = new HashSet<>(dynamicState.changingBlobs.size()); + if (forSameTopology(dynamicState.pendingChangingBlobsAssignment, assignment)) { + //We need to add the new futures to the existing ones + futures.addAll(dynamicState.pendingChangeingBlobs); + } + //Otherwise they will just be replaced + + for (BlobChangeing rc: dynamicState.changingBlobs) { + futures.add(rc.latch.countDown()); + } + + LOG.debug("found changeing blobs {} moving them to pending...", dynamicState.changingBlobs); + return dynamicState.withChangingBlobs(Collections.emptySet()) + .withPendingChangeingBlobs(futures, assignment); + } + + /** + * Filter all of the changing blobs to just be for those compatible with the given assignment. + * All others will be released appropriately. + * + * @param dynamicState the current state + * @param assignment the assignment to look for + * @return the updated dynamicState + */ + private static DynamicState filterChangingBlobsFor(DynamicState dynamicState, final LocalAssignment assignment) { + if (!dynamicState.changingBlobs.isEmpty()) { + return dynamicState; + } + + HashSet savedBlobs = new HashSet<>(dynamicState.changingBlobs.size()); + for (BlobChangeing rc: dynamicState.changingBlobs) { + if (forSameTopology(assignment, rc.assignment)) { + savedBlobs.add(rc); + } else { + rc.latch.countDown(); + } + } + return dynamicState.withChangingBlobs(savedBlobs); + } + /** * State Transitions for WAITING_FOR_BLOB_LOCALIZATION state. * PRECONDITION: neither pendingLocalization nor pendingDownload is null. @@ -410,17 +590,35 @@ static DynamicState handleWaitingForBlobLocalization(DynamicState dynamicState, // We don't support canceling the download through the future yet, // so to keep everything in sync, just wait try { + //Release things that don't need to wait for us to finish downloading. + dynamicState = filterChangingBlobsFor(dynamicState, dynamicState.pendingLocalization); + if (!dynamicState.changingBlobs.isEmpty()) { + //Unblock downloading by accepting the futures. + dynamicState = informChangedBlobs(dynamicState, dynamicState.pendingLocalization); + } + dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS); //Downloading of all blobs finished. if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) { //Scheduling changed staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port); - return prepareForNewAssignmentNoWorkersRunning(dynamicState, staticState); + return prepareForNewAssignmentNoWorkersRunning(dynamicState.withPendingChangeingBlobs(Collections.emptySet(), null), + staticState); + } + + if (!dynamicState.pendingChangeingBlobs.isEmpty()) { + LOG.info("There are pending changes, waiting for them to finish before launching container..."); + //We cannot launch the container yet the resources may still be updating + return dynamicState.withState(MachineState.WAITING_FOR_BLOB_UPDATE) + .withPendingLocalization(null, null); } + dynamicState = updateAssignmentIfNeeded(dynamicState); numWorkersLaunched.mark(); Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState); - return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null); + return dynamicState + .withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START) + .withPendingLocalization(null, null); } catch (TimeoutException e) { //We waited for 1 second loop around and try again.... return dynamicState; @@ -439,29 +637,64 @@ static DynamicState handleWaitingForBlobLocalization(DynamicState dynamicState, return dynamicState.withState(MachineState.EMPTY); } } - + + private static final long ONE_SEC_IN_NANO = TimeUnit.NANOSECONDS.convert(1, TimeUnit.SECONDS); + /** - * State Transitions for WAITING_FOR_BASIC_LOCALIZATION state. - * PRECONDITION: neither pendingLocalization nor pendingDownload is null. - * PRECONDITION: The slot should be empty + * State Transitions for WAITING_FOR_BLOB_UPDATE state. + * + * PRECONDITION: container is null + * PRECONDITION: pendingChangeingBlobs is not empty (otherwise why did we go to this state) + * PRECONDITION: pendingChangingBlobsAssignment is not null. + * * @param dynamicState current state * @param staticState static data * @return the next state * @throws Exception on any error */ - static DynamicState handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception { - assert(dynamicState.pendingLocalization != null); - assert(dynamicState.pendingDownload != null); - assert(dynamicState.container == null); - - //Ignore changes to scheduling while downloading the topology code - // We don't support canceling the download through the future yet, - // so to keep everything in sync, just wait + private static DynamicState handleWaitingForBlobUpdate(DynamicState dynamicState, StaticState staticState) + throws Exception { + assert dynamicState.container == null; + assert dynamicState.pendingChangingBlobsAssignment != null; + assert !dynamicState.pendingChangeingBlobs.isEmpty(); + + if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { + //We were rescheduled while waiting for the resources to be updated, + // but the container is already not running. + LOG.info("SLOT {}: Assignment Changed from {} to {}", staticState.port, + dynamicState.currentAssignment, dynamicState.newAssignment); + if (dynamicState.currentAssignment != null) { + staticState.localizer.releaseSlotFor(dynamicState.currentAssignment, staticState.port); + } + staticState.localizer.releaseSlotFor(dynamicState.pendingChangingBlobsAssignment, staticState.port); + return prepareForNewAssignmentNoWorkersRunning(dynamicState.withCurrentAssignment(null, null) + .withPendingChangeingBlobs(Collections.emptySet(), null), + staticState); + } + + dynamicState = filterChangingBlobsFor(dynamicState, dynamicState.pendingChangingBlobsAssignment); + if (!dynamicState.changingBlobs.isEmpty()) { + dynamicState = informChangedBlobs(dynamicState, dynamicState.pendingChangingBlobsAssignment); + } + + //We only have a set amount of time we can wait for before looping around again + long start = Time.nanoTime(); try { - dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS); - Future pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization, staticState.port); - return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION); - } catch (TimeoutException e) { + for (Future pending: dynamicState.pendingChangeingBlobs) { + long now = Time.nanoTime(); + long timeLeft = ONE_SEC_IN_NANO - (now - start); + if (timeLeft <= 0) { + throw new TimeoutException(); + } + pending.get(timeLeft, TimeUnit.NANOSECONDS); + } + //All done we can launch the worker now + Container c = staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingChangingBlobsAssignment, + staticState.localState); + return dynamicState + .withCurrentAssignment(c, dynamicState.pendingChangingBlobsAssignment).withState(MachineState.WAITING_FOR_WORKER_START) + .withPendingChangeingBlobs(Collections.emptySet(), null); + } catch (TimeoutException ex) { return dynamicState; } } @@ -478,11 +711,11 @@ static DynamicState handleWaitingForBasicLocalization(DynamicState dynamicState, static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception { assert(dynamicState.container != null); assert(dynamicState.currentAssignment != null); - + if (dynamicState.container.areAllProcessesDead()) { - LOG.warn("SLOT {} all processes are dead...", staticState.port); - return cleanupCurrentContainer(dynamicState, staticState, - dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BASIC_LOCALIZATION); + LOG.info("SLOT {} all processes are dead...", staticState.port); + return cleanupCurrentContainer(dynamicState, staticState, + dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BLOB_LOCALIZATION); } LOG.warn("SLOT {} force kill and wait...", staticState.port); @@ -524,6 +757,42 @@ static DynamicState handleKillAndRelaunch(DynamicState dynamicState, StaticState return dynamicState; } + /** + * State Transitions for KILL_BLOB_UPDATE state. + * PRECONDITION: container.kill() was called + * PRECONDITION: container != null && currentAssignment != null + * + * @param dynamicState current state + * @param staticState static data + * @return the next state + * @throws Exception on any error + */ + private static DynamicState handleKillBlobUpdate(DynamicState dynamicState, StaticState staticState) throws Exception { + assert(dynamicState.container != null); + assert(dynamicState.currentAssignment != null); + + //Release things that don't need to wait for us + dynamicState = filterChangingBlobsFor(dynamicState, dynamicState.currentAssignment); + + if (dynamicState.container.areAllProcessesDead()) { + if (equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { + dynamicState.container.cleanUp(); + dynamicState = dynamicState.withCurrentAssignment(null, dynamicState.currentAssignment); + return informChangedBlobs(dynamicState, dynamicState.currentAssignment) + .withState(MachineState.WAITING_FOR_BLOB_UPDATE); + } + //Scheduling changed after we killed all of the processes + return prepareForNewAssignmentNoWorkersRunning(cleanupCurrentContainer(dynamicState, staticState, null), staticState); + } + //The child processes typically exit in < 1 sec. If 2 mins later they are still around something is wrong + if ((Time.currentTimeMillis() - dynamicState.startTime) > 120_000) { + throw new RuntimeException("Not all processes in " + dynamicState.container + " exited after 120 seconds"); + } + dynamicState.container.forceKill(); + Time.sleep(staticState.killSleepMs); + return dynamicState; + } + /** * State Transitions for WAITING_FOR_WORKER_START state. * PRECONDITION: container != null && currentAssignment != null @@ -546,7 +815,8 @@ static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, Stati if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { //We were rescheduled while waiting for the worker to come up - LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment); + LOG.info("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, + dynamicState.newAssignment); return killContainerForChangedAssignment(dynamicState, staticState); } dynamicState = updateAssignmentIfNeeded(dynamicState); @@ -556,6 +826,12 @@ static DynamicState handleWaitingForWorkerStart(DynamicState dynamicState, Stati LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs); return killAndRelaunchContainer(dynamicState, staticState); } + + dynamicState = filterChangingBlobsFor(dynamicState, dynamicState.currentAssignment); + if (!dynamicState.changingBlobs.isEmpty()) { + //Kill the container and restart it + return killContainerForChangedBlobs(dynamicState, staticState); + } Time.sleep(1000); return dynamicState; } @@ -573,12 +849,19 @@ static DynamicState handleRunning(DynamicState dynamicState, StaticState staticS assert(dynamicState.currentAssignment != null); if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { - LOG.warn("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, dynamicState.newAssignment); + LOG.info("SLOT {}: Assignment Changed from {} to {}", staticState.port, dynamicState.currentAssignment, + dynamicState.newAssignment); //Scheduling changed while running... return killContainerForChangedAssignment(dynamicState, staticState); } dynamicState = updateAssignmentIfNeeded(dynamicState); + dynamicState = filterChangingBlobsFor(dynamicState, dynamicState.currentAssignment); + if (!dynamicState.changingBlobs.isEmpty()) { + //Kill the container and restart it + return killContainerForChangedBlobs(dynamicState, staticState); + } + if (dynamicState.container.didMainProcessExit()) { numWorkersKilledProcessExit.mark(); LOG.warn("SLOT {}: main process has exited", staticState.port); @@ -660,6 +943,8 @@ static DynamicState handleRunning(DynamicState dynamicState, StaticState staticS } static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException, IOException { + assert dynamicState.changingBlobs.isEmpty(); + assert dynamicState.pendingChangingBlobsAssignment == null; if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { return prepareForNewAssignmentNoWorkersRunning(dynamicState, staticState); } @@ -669,15 +954,17 @@ static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticSta if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) { //Nothing is scheduled here so throw away all of the profileActions LOG.warn("Dropping {} no topology is running", dynamicState.profileActions); - dynamicState = dynamicState.withProfileActions(Collections. emptySet(), Collections. emptySet()); + dynamicState = dynamicState.withProfileActions(Collections.emptySet(), Collections.emptySet()); } + //Drop the change notifications we are not running anything right now + dynamicState = drainAllChangingBlobs(dynamicState); Time.sleep(1000); return dynamicState; } private final AtomicReference newAssignment = new AtomicReference<>(); - private final AtomicReference> profiling = - new AtomicReference>(new HashSet()); + private final AtomicReference> profiling = new AtomicReference<>(new HashSet<>()); + private final BlockingQueue changingBlobs = new LinkedBlockingQueue<>(); private final StaticState staticState; private final IStormClusterState clusterState; private volatile boolean done = false; @@ -685,11 +972,11 @@ static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticSta private final AtomicReference> cachedCurrentAssignments; public Slot(AsyncLocalizer localizer, Map conf, - ContainerLauncher containerLauncher, String host, - int port, LocalState localState, - IStormClusterState clusterState, - ISupervisor iSupervisor, - AtomicReference> cachedCurrentAssignments) throws Exception { + ContainerLauncher containerLauncher, String host, + int port, LocalState localState, + IStormClusterState clusterState, + ISupervisor iSupervisor, + AtomicReference> cachedCurrentAssignments) throws Exception { super("SLOT_"+port); this.cachedCurrentAssignments = cachedCurrentAssignments; @@ -715,23 +1002,25 @@ public Slot(AsyncLocalizer localizer, Map conf, } dynamicState = new DynamicState(currentAssignment, container, newAssignment); - staticState = new StaticState(localizer, - ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000, - ObjectReader.getInt(conf.get(DaemonConfig.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000, - ObjectReader.getInt(conf.get(DaemonConfig.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000, - ObjectReader.getInt(conf.get(DaemonConfig.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000, - containerLauncher, - host, - port, - iSupervisor, - localState); + staticState = new StaticState(localizer, + ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000, + ObjectReader.getInt(conf.get(DaemonConfig.SUPERVISOR_WORKER_START_TIMEOUT_SECS)) * 1000, + ObjectReader.getInt(conf.get(DaemonConfig.SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS)) * 1000, + ObjectReader.getInt(conf.get(DaemonConfig.SUPERVISOR_MONITOR_FREQUENCY_SECS)) * 1000, + containerLauncher, + host, + port, + iSupervisor, + localState, + this); this.newAssignment.set(dynamicState.newAssignment); if (MachineState.RUNNING == dynamicState.state) { //We are running so we should recover the blobs. - staticState.localizer.recoverRunningTopology(currentAssignment, port); + staticState.localizer.recoverRunningTopology(currentAssignment, port, this); saveNewAssignment(currentAssignment); } - LOG.warn("SLOT {}:{} Starting in state {} - assignment {}", staticState.host, staticState.port, dynamicState.state, dynamicState.currentAssignment); + LOG.info("SLOT {}:{} Starting in state {} - assignment {}", staticState.host, staticState.port, dynamicState.state, + dynamicState.currentAssignment); } public MachineState getMachineState() { @@ -739,13 +1028,24 @@ public MachineState getMachineState() { } /** - * Set a new assignment asynchronously + * Set a new assignment asynchronously. * @param newAssignment the new assignment for this slot to run, null to run nothing */ public void setNewAssignment(LocalAssignment newAssignment) { this.newAssignment.set(newAssignment); } - + + @Override + public void blobChanging(LocalAssignment assignment, int port, LocallyCachedBlob blob, GoodToGo go) { + assert port == staticState.port : "got a callaback that is not for us " + port + " != " + staticState.port; + //This is called async so lets assume that it is something we care about + try { + changingBlobs.put(new BlobChangeing(assignment, blob, go.getLatch())); + } catch (InterruptedException e) { + throw new RuntimeException("This should not have happend, but it did (the queue is unbounded)", e); + } + } + public void addProfilerActions(Set actions) { if (actions != null) { while(true) { @@ -800,10 +1100,28 @@ public void run() { while(!done) { Set origProfileActions = new HashSet<>(profiling.get()); Set removed = new HashSet<>(origProfileActions); - - DynamicState nextState = - stateMachineStep(dynamicState.withNewAssignment(newAssignment.get()) - .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions), staticState); + + Set changingResourcesToHandle = dynamicState.changingBlobs; + if (!changingBlobs.isEmpty()) { + changingResourcesToHandle = new HashSet<>(changingResourcesToHandle); + changingBlobs.drainTo(changingResourcesToHandle); + Iterator it = changingResourcesToHandle.iterator(); + + //Remove/Clean up changed requests that are not for us + while(it.hasNext()) { + BlobChangeing rc = it.next(); + if (!forSameTopology(rc.assignment, dynamicState.currentAssignment) && + !forSameTopology(rc.assignment, dynamicState.newAssignment)) { + rc.latch.countDown(); //Ignore the future + it.remove(); + } + } + } + + DynamicState nextState = + stateMachineStep(dynamicState.withNewAssignment(newAssignment.get()) + .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions) + .withChangingBlobs(changingResourcesToHandle), staticState); if (LOG.isDebugEnabled() || dynamicState.state != nextState.state) { LOG.info("STATE {} -> {}", dynamicState, nextState); diff --git a/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java b/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java index 043de8a187c..4b475024626 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java @@ -24,13 +24,10 @@ import java.io.File; import java.io.FileOutputStream; import java.io.FileWriter; -import java.io.FilenameFilter; import java.io.IOException; -import java.io.OutputStream; import java.io.PrintWriter; -import java.net.JarURLConnection; -import java.net.URL; import java.net.URLDecoder; +import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -58,7 +55,6 @@ import org.apache.commons.io.FileUtils; import org.apache.storm.Config; import org.apache.storm.DaemonConfig; -import org.apache.storm.blobstore.BlobStore; import org.apache.storm.blobstore.ClientBlobStore; import org.apache.storm.blobstore.InputStreamWithMeta; import org.apache.storm.daemon.supervisor.AdvancedFSOps; @@ -71,7 +67,6 @@ import org.apache.storm.utils.ConfigUtils; import org.apache.storm.utils.NimbusLeaderNotFoundException; import org.apache.storm.utils.ObjectReader; -import org.apache.storm.utils.ServerConfigUtils; import org.apache.storm.utils.ServerUtils; import org.apache.storm.utils.ShellUtils; import org.apache.storm.utils.Utils; @@ -110,11 +105,12 @@ private static Set readDownloadedTopologyIds(Map conf) t private final AtomicReference> currAssignment; private final boolean isLocalMode; - private final Map basicPending; private final Map blobPending; private final Map conf; private final AdvancedFSOps fsOps; private final boolean symlinksDisabled; + private final ConcurrentHashMap topologyBlobs = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> topologyBasicDownloaded = new ConcurrentHashMap<>(); // track resources - user to resourceSet private final ConcurrentMap userRsrc = new ConcurrentHashMap<>(); @@ -126,16 +122,10 @@ private static Set readDownloadedTopologyIds(Map conf) t // cleanup private long cacheTargetSize; - private long cacheCleanupPeriod; - - - public AsyncLocalizer(Map conf, AtomicReference> currAssignment, - Map portToAssignments) throws IOException { - this(conf, ConfigUtils.supervisorLocalDir(conf), AdvancedFSOps.make(conf), currAssignment, portToAssignments); - } + private final long cacheCleanupPeriod; @VisibleForTesting - AsyncLocalizer(Map conf, String baseDir, AdvancedFSOps ops, + AsyncLocalizer(Map conf, AdvancedFSOps ops, String baseDir, AtomicReference> currAssignment, Map portToAssignments) throws IOException { @@ -146,7 +136,7 @@ public AsyncLocalizer(Map conf, AtomicReference conf, AtomicReference(); blobPending = new HashMap<>(); this.currAssignment = currAssignment; recoverBlobReferences(portToAssignments); } + public AsyncLocalizer(Map conf, AtomicReference> currAssignment, + Map portToAssignments) throws IOException { + this(conf, AdvancedFSOps.make(conf), ConfigUtils.supervisorLocalDir(conf), currAssignment, portToAssignments); + } + + @VisibleForTesting + LocallyCachedBlob getTopoJar(final String topologyId) throws IOException { + String topoJarKey = ConfigUtils.masterStormJarKey(topologyId); + LocallyCachedBlob topoJar = topologyBlobs.get(topoJarKey); + if (topoJar == null) { + topoJar = new LocallyCachedTopologyBlob(topologyId, isLocalMode, conf, fsOps, + LocallyCachedTopologyBlob.TopologyBlobType.TOPO_JAR); + topologyBlobs.put(topoJarKey, topoJar); + } + return topoJar; + } + + @VisibleForTesting + LocallyCachedBlob getTopoCode(final String topologyId) throws IOException { + String topoCodeKey = ConfigUtils.masterStormCodeKey(topologyId); + LocallyCachedBlob topoCode = topologyBlobs.get(topoCodeKey); + if (topoCode == null) { + topoCode = new LocallyCachedTopologyBlob(topologyId, isLocalMode, conf, fsOps, + LocallyCachedTopologyBlob.TopologyBlobType.TOPO_CODE); + topologyBlobs.put(topoCodeKey, topoCode); + } + return topoCode; + } + + @VisibleForTesting + LocallyCachedBlob getTopoConf(final String topologyId) throws IOException { + String topoConfKey = ConfigUtils.masterStormConfKey(topologyId); + LocallyCachedBlob topoConf = topologyBlobs.get(topoConfKey); + if (topoConf == null) { + topoConf = new LocallyCachedTopologyBlob(topologyId, isLocalMode, conf, fsOps, + LocallyCachedTopologyBlob.TopologyBlobType.TOPO_CONF); + topologyBlobs.put(topoConfKey, topoConf); + } + return topoConf; + } + + public synchronized CompletableFuture requestDownloadTopologyBlobs(final LocalAssignment assignment, final int port, + final BlobChangingCallback cb) throws IOException { + final String topologyId = assignment.get_topology_id(); + + CompletableFuture baseBlobs = requestDownloadBaseTopologyBlobs(assignment, port, cb); + return baseBlobs.thenComposeAsync((v) -> { + LocalDownloadedResource localResource = blobPending.get(topologyId); + if (localResource == null) { + Supplier supplier = new DownloadBlobs(topologyId, assignment.get_owner()); + localResource = new LocalDownloadedResource(CompletableFuture.supplyAsync(supplier, execService)); + blobPending.put(topologyId, localResource); + } + CompletableFuture r = localResource.reserve(port, assignment); + LOG.debug("Reserved blobs {} {}", topologyId, localResource); + return r; + }); + } + + @VisibleForTesting + synchronized CompletableFuture requestDownloadBaseTopologyBlobs(final LocalAssignment assignment, final int port, + BlobChangingCallback cb) throws IOException { + PortAndAssignment pna = new PortAndAssignment(port, assignment); + final String topologyId = assignment.get_topology_id(); + + LocallyCachedBlob topoJar = getTopoJar(topologyId); + topoJar.addReference(pna, cb); + + LocallyCachedBlob topoCode = getTopoCode(topologyId); + topoCode.addReference(pna, cb); + + LocallyCachedBlob topoConf = getTopoConf(topologyId); + topoConf.addReference(pna, cb); + + CompletableFuture ret = topologyBasicDownloaded.get(topologyId); + if (ret == null) { + ret = downloadOrUpdate(topoJar, topoCode, topoConf); + } + return ret; + } + + private static final int ATTEMPTS_INTERVAL_TIME = 100; + + private CompletableFuture downloadOrUpdate(LocallyCachedBlob ... blobs) { + CompletableFuture [] all = new CompletableFuture[blobs.length]; + for (int i = 0; i < blobs.length; i++) { + final LocallyCachedBlob blob = blobs[i]; + all[i] = CompletableFuture.runAsync(() -> { + LOG.debug("STARTING download of {}", blob); + try (ClientBlobStore blobStore = ServerUtils.getClientBlobStoreForSupervisor(conf)) { + boolean done = false; + long failures = 0; + while (!done) { + try { + synchronized (blob) { + long localVersion = blob.getLocalVersion(); + long remoteVersion = blob.getRemoteVersion(blobStore); + if (localVersion != remoteVersion) { + try { + long newVersion = blob.downloadToTempLocation(blobStore); + blob.informAllOfChangeAndWaitForConsensus(); + blob.commitNewVersion(newVersion); + blob.informAllChangeComplete(); + } finally { + blob.cleanupOrphanedData(); + } + } + } + done = true; + } catch (Exception e) { + failures++; + if (failures > blobDownloadRetries) { + throw new RuntimeException("Could not download...", e); + } + LOG.warn("Failed to download blob {} will try again in {} ms", blob, ATTEMPTS_INTERVAL_TIME, e); + Utils.sleep(ATTEMPTS_INTERVAL_TIME); + } + } + } + LOG.debug("FINISHED download of {}", blob); + }, execService); + } + return CompletableFuture.allOf(all); + } + /** * For each of the downloaded topologies, adds references to the blobs that the topologies are using. This is used to reconstruct the * cache on restart. @@ -211,11 +325,20 @@ private void recoverBlobReferences(Map portToAssignmen * with a suffix. The runnable is intended to be run periodically by a timer, created elsewhere. */ private void updateBlobs() { + for (LocallyCachedBlob blob : topologyBlobs.values()) { + if (blob.isUsed()) { + try { + downloadOrUpdate(blob); + } catch (Exception e) { + LOG.error("Could not update {}", blob, e); + } + } + } try { Map topoIdToOwner = currAssignment.get().values().stream() .map((la) -> Pair.of(la.get_topology_id(), la.get_owner())) .distinct() - .collect(Collectors.toMap((p) -> p.getFirst(), (p) -> p.getSecond())); + .collect(Collectors.toMap(Pair::getFirst, Pair::getSecond)); for (String topoId : readDownloadedTopologyIds(conf)) { String owner = topoIdToOwner.get(topoId); if (owner == null) { @@ -272,129 +395,6 @@ public void close() throws InterruptedException { } //ILocalizer - private class DownloadBaseBlobsDistributed implements Supplier { - protected final String topologyId; - protected final File stormRoot; - protected final LocalAssignment assignment; - protected final String owner; - - public DownloadBaseBlobsDistributed(String topologyId, LocalAssignment assignment) throws IOException { - this.topologyId = topologyId; - stormRoot = new File(ConfigUtils.supervisorStormDistRoot(conf, this.topologyId)); - this.assignment = assignment; - owner = assignment.get_owner(); - } - - protected void downloadBaseBlobs(File tmproot) throws Exception { - String stormJarKey = ConfigUtils.masterStormJarKey(topologyId); - String stormCodeKey = ConfigUtils.masterStormCodeKey(topologyId); - String topoConfKey = ConfigUtils.masterStormConfKey(topologyId); - String jarPath = ConfigUtils.supervisorStormJarPath(tmproot.getAbsolutePath()); - String codePath = ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath()); - String confPath = ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath()); - fsOps.forceMkdir(tmproot); - fsOps.restrictDirectoryPermissions(tmproot); - ClientBlobStore blobStore = ServerUtils.getClientBlobStoreForSupervisor(conf); - try { - ServerUtils.downloadResourcesAsSupervisor(stormJarKey, jarPath, blobStore); - ServerUtils.downloadResourcesAsSupervisor(stormCodeKey, codePath, blobStore); - ServerUtils.downloadResourcesAsSupervisor(topoConfKey, confPath, blobStore); - } finally { - blobStore.shutdown(); - } - ServerUtils.extractDirFromJar(jarPath, ServerConfigUtils.RESOURCES_SUBDIR, tmproot); - } - - @Override - public Void get() { - try { - if (fsOps.fileExists(stormRoot)) { - if (!fsOps.supportsAtomicDirectoryMove()) { - LOG.warn("{} may have partially downloaded blobs, recovering", topologyId); - fsOps.deleteIfExists(stormRoot); - } else { - LOG.warn("{} already downloaded blobs, skipping", topologyId); - return null; - } - } - boolean deleteAll = true; - String tmproot = ServerConfigUtils.supervisorTmpDir(conf) + File.separator + Utils.uuid(); - File tr = new File(tmproot); - try { - downloadBaseBlobs(tr); - if (assignment.is_set_total_node_shared()) { - File sharedMemoryDirTmpLocation = new File(tr, "shared_by_topology"); - //We need to create a directory for shared memory to write to (we should not encourage this though) - Path path = sharedMemoryDirTmpLocation.toPath(); - Files.createDirectories(path); - } - fsOps.moveDirectoryPreferAtomic(tr, stormRoot); - fsOps.setupStormCodeDir(owner, stormRoot); - if (assignment.is_set_total_node_shared()) { - File sharedMemoryDir = new File(stormRoot, "shared_by_topology"); - fsOps.setupWorkerArtifactsDir(owner, sharedMemoryDir); - } - deleteAll = false; - } finally { - if (deleteAll) { - LOG.warn("Failed to download basic resources for topology-id {}", topologyId); - fsOps.deleteIfExists(tr); - fsOps.deleteIfExists(stormRoot); - } - } - return null; - } catch (Exception e) { - LOG.warn("Caught Exception While Downloading (rethrowing)... ", e); - throw new RuntimeException(e); - } - } - } - - private class DownloadBaseBlobsLocal extends DownloadBaseBlobsDistributed { - - public DownloadBaseBlobsLocal(String topologyId, LocalAssignment assignment) throws IOException { - super(topologyId, assignment); - } - - @Override - protected void downloadBaseBlobs(File tmproot) throws Exception { - fsOps.forceMkdir(tmproot); - String stormCodeKey = ConfigUtils.masterStormCodeKey(topologyId); - String topoConfKey = ConfigUtils.masterStormConfKey(topologyId); - File codePath = new File(ConfigUtils.supervisorStormCodePath(tmproot.getAbsolutePath())); - File confPath = new File(ConfigUtils.supervisorStormConfPath(tmproot.getAbsolutePath())); - BlobStore blobStore = ServerUtils.getNimbusBlobStore(conf, null); - try { - try (OutputStream codeOutStream = fsOps.getOutputStream(codePath)){ - blobStore.readBlobTo(stormCodeKey, codeOutStream, null); - } - try (OutputStream confOutStream = fsOps.getOutputStream(confPath)) { - blobStore.readBlobTo(topoConfKey, confOutStream, null); - } - } finally { - blobStore.shutdown(); - } - - ClassLoader classloader = Thread.currentThread().getContextClassLoader(); - String resourcesJar = resourcesJar(); - URL url = classloader.getResource(ServerConfigUtils.RESOURCES_SUBDIR); - - String targetDir = tmproot + File.separator; - if (resourcesJar != null) { - LOG.info("Extracting resources from jar at {} to {}", resourcesJar, targetDir); - ServerUtils.extractDirFromJar(resourcesJar, ServerConfigUtils.RESOURCES_SUBDIR, new File(targetDir)); - } else if (url != null) { - LOG.info("Copying resources at {} to {}", url, targetDir); - if ("jar".equals(url.getProtocol())) { - JarURLConnection urlConnection = (JarURLConnection) url.openConnection(); - ServerUtils.extractDirFromJar(urlConnection.getJarFileURL().getFile(), ServerConfigUtils.RESOURCES_SUBDIR, new File(targetDir)); - } else { - fsOps.copyDirectory(new File(url.getFile()), new File(targetDir, ConfigUtils.RESOURCES_SUBDIR)); - } - } - } - } - private class DownloadBlobs implements Supplier { private final String topologyId; private final String topoOwner; @@ -472,51 +472,21 @@ public Void get() { } } - public synchronized CompletableFuture requestDownloadBaseTopologyBlobs(final LocalAssignment assignment, final int port) throws IOException { + public synchronized void recoverRunningTopology(final LocalAssignment assignment, final int port, + final BlobChangingCallback cb) throws IOException { + PortAndAssignment pna = new PortAndAssignment(port, assignment); final String topologyId = assignment.get_topology_id(); - LocalDownloadedResource localResource = basicPending.get(topologyId); - if (localResource == null) { - Supplier supplier; - if (isLocalMode) { - supplier = new DownloadBaseBlobsLocal(topologyId, assignment); - } else { - supplier = new DownloadBaseBlobsDistributed(topologyId, assignment); - } - localResource = new LocalDownloadedResource(CompletableFuture.supplyAsync(supplier, execService)); - basicPending.put(topologyId, localResource); - } - CompletableFuture ret = localResource.reserve(port, assignment); - LOG.debug("Reserved basic {} {}", topologyId, localResource); - return ret; - } - private static String resourcesJar() throws IOException { - String path = ServerUtils.currentClasspath(); - if (path == null) { - return null; - } + LocallyCachedBlob topoJar = getTopoJar(topologyId); + topoJar.addReference(pna, cb); - for (String jpath : path.split(File.pathSeparator)) { - if (jpath.endsWith(".jar")) { - if (ServerUtils.zipDoesContainDir(jpath, ServerConfigUtils.RESOURCES_SUBDIR)) { - return jpath; - } - } - } - return null; - } + LocallyCachedBlob topoCode = getTopoCode(topologyId); + topoCode.addReference(pna, cb); - public synchronized void recoverRunningTopology(LocalAssignment assignment, int port) { - final String topologyId = assignment.get_topology_id(); - LocalDownloadedResource localResource = basicPending.get(topologyId); - if (localResource == null) { - localResource = new LocalDownloadedResource(ALL_DONE_FUTURE); - basicPending.put(topologyId, localResource); - } - localResource.reserve(port, assignment); - LOG.debug("Recovered basic {} {}", topologyId, localResource); + LocallyCachedBlob topoConf = getTopoConf(topologyId); + topoConf.addReference(pna, cb); - localResource = blobPending.get(topologyId); + LocalDownloadedResource localResource = blobPending.get(topologyId); if (localResource == null) { localResource = new LocalDownloadedResource(ALL_DONE_FUTURE); blobPending.put(topologyId, localResource); @@ -525,19 +495,6 @@ public synchronized void recoverRunningTopology(LocalAssignment assignment, int LOG.debug("Recovered blobs {} {}", topologyId, localResource); } - public synchronized CompletableFuture requestDownloadTopologyBlobs(LocalAssignment assignment, int port) { - final String topologyId = assignment.get_topology_id(); - LocalDownloadedResource localResource = blobPending.get(topologyId); - if (localResource == null) { - Supplier supplier = new DownloadBlobs(topologyId, assignment.get_owner()); - localResource = new LocalDownloadedResource(CompletableFuture.supplyAsync(supplier, execService)); - blobPending.put(topologyId, localResource); - } - CompletableFuture ret = localResource.reserve(port, assignment); - LOG.debug("Reserved blobs {} {}", topologyId, localResource); - return ret; - } - /** * Remove this assignment/port as blocking resources from being cleaned up. * @@ -546,8 +503,29 @@ public synchronized CompletableFuture requestDownloadTopologyBlobs(LocalAs * @throws IOException on any error */ public synchronized void releaseSlotFor(LocalAssignment assignment, int port) throws IOException { + PortAndAssignment pna = new PortAndAssignment(port, assignment); final String topologyId = assignment.get_topology_id(); LOG.debug("Releasing slot for {} {}", topologyId, port); + + String topoJarKey = ConfigUtils.masterStormJarKey(topologyId); + String topoCodeKey = ConfigUtils.masterStormCodeKey(topologyId); + String topoConfKey = ConfigUtils.masterStormConfKey(topologyId); + + LocallyCachedBlob topoJar = topologyBlobs.get(topoJarKey); + if (topoJar != null) { + topoJar.removeReference(pna); + } + + LocallyCachedBlob topoCode = topologyBlobs.get(topoCodeKey); + if (topoCode != null) { + topoCode.removeReference(pna); + } + + LocallyCachedBlob topoConfBlob = topologyBlobs.get(topoConfKey); + if (topoConfBlob != null) { + topoConfBlob.removeReference(pna); + } + LocalDownloadedResource localResource = blobPending.get(topologyId); if (localResource == null || !localResource.release(port, assignment)) { LOG.warn("Released blob reference {} {} for something that we didn't have {}", topologyId, port, localResource); @@ -574,38 +552,12 @@ public synchronized void releaseSlotFor(LocalAssignment assignment, int port) th } else { LOG.debug("Released blob reference {} {} still waiting on {}", topologyId, port, localResource); } - - localResource = basicPending.get(topologyId); - if (localResource == null || !localResource.release(port, assignment)) { - LOG.warn("Released basic reference {} {} for something that we didn't have {}", topologyId, port, localResource); - } else if (localResource.isDone()){ - LOG.info("Released blob reference {} {} Cleaning up basic files...", topologyId, port); - basicPending.remove(topologyId); - String path = ConfigUtils.supervisorStormDistRoot(conf, topologyId); - fsOps.deleteIfExists(new File(path), null, "rmr "+topologyId); - } else { - LOG.debug("Released basic reference {} {} still waiting on {}", topologyId, port, localResource); - } - } - - public synchronized void cleanupUnusedTopologies() throws IOException { - File distRoot = new File(ConfigUtils.supervisorStormDistRoot(conf)); - LOG.info("Cleaning up unused topologies in {}", distRoot); - File[] children = distRoot.listFiles(); - if (children != null) { - for (File topoDir : children) { - String topoId = URLDecoder.decode(topoDir.getName(), "UTF-8"); - if (basicPending.get(topoId) == null && blobPending.get(topoId) == null) { - fsOps.deleteIfExists(topoDir, null, "rmr " + topoId); - } - } - } } //From Localizer - // For testing, it allows setting size in bytes - protected void setTargetCacheSize(long size) { + @VisibleForTesting + void setTargetCacheSize(long size) { cacheTargetSize = size; } @@ -615,33 +567,33 @@ ConcurrentMap getUserResources() { } // baseDir/supervisor/usercache/ - protected File getUserCacheDir() { + private File getUserCacheDir() { return new File(localBaseDir, USERCACHE); } // baseDir/supervisor/usercache/user1/ - protected File getLocalUserDir(String userName) { + File getLocalUserDir(String userName) { return new File(getUserCacheDir(), userName); } // baseDir/supervisor/usercache/user1/filecache - public File getLocalUserFileCacheDir(String userName) { + File getLocalUserFileCacheDir(String userName) { return new File(getLocalUserDir(userName), FILECACHE); } // baseDir/supervisor/usercache/user1/filecache/files - protected File getCacheDirForFiles(File dir) { + private File getCacheDirForFiles(File dir) { return new File(dir, FILESDIR); } // get the directory to put uncompressed archives in // baseDir/supervisor/usercache/user1/filecache/archives - protected File getCacheDirForArchives(File dir) { + private File getCacheDirForArchives(File dir) { return new File(dir, ARCHIVESDIR); } - protected void addLocalizedResourceInDir(String dir, LocalizedResourceSet lrsrcSet, - boolean uncompress) { + private void addLocalizedResourceInDir(String dir, LocalizedResourceSet lrsrcSet, + boolean uncompress) { File[] lrsrcs = readCurrentBlobs(dir); if (lrsrcs != null) { @@ -662,22 +614,17 @@ protected void addLocalizedResourceInDir(String dir, LocalizedResourceSet lrsrcS } // Looks for files in the directory with .current suffix - protected File[] readCurrentBlobs(String location) { + private File[] readCurrentBlobs(String location) { File dir = new File(location); File[] files = null; if (dir.exists()) { - files = dir.listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - return name.toLowerCase().endsWith(ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX); - } - }); + files = dir.listFiles((d, name) -> name.toLowerCase().endsWith(ServerUtils.DEFAULT_CURRENT_BLOB_SUFFIX)); } return files; } // Check to see if there are any existing files already localized. - protected void reconstructLocalizedResources() { + private void reconstructLocalizedResources() { try { LOG.info("Reconstruct localized resource: " + getUserCacheDir().getPath()); Collection users = ConfigUtils.readDirFiles(getUserCacheDir().getPath()); @@ -705,8 +652,8 @@ protected void reconstructLocalizedResources() { } // ignores invalid user/topo/key - public synchronized void removeBlobReference(String key, String user, String topo, - boolean uncompress) throws AuthorizationException, KeyNotFoundException { + synchronized void removeBlobReference(String key, String user, String topo, + boolean uncompress) throws AuthorizationException, KeyNotFoundException { LocalizedResourceSet lrsrcSet = userRsrc.get(user); if (lrsrcSet != null) { LocalizedResource lrsrc = lrsrcSet.get(key, uncompress); @@ -723,8 +670,8 @@ public synchronized void removeBlobReference(String key, String user, String top } } - public synchronized void addReferences(List localresource, String user, - String topo) { + synchronized void addReferences(List localresource, String user, + String topo) { LocalizedResourceSet lrsrcSet = userRsrc.get(user); if (lrsrcSet != null) { for (LocalResource blob : localresource) { @@ -746,9 +693,9 @@ public synchronized void addReferences(List localresource, String * This function either returns the blob in the existing cache or if it doesn't exist in the * cache, it will download the blob and will block until the download is complete. */ - public LocalizedResource getBlob(LocalResource localResource, String user, String topo, + LocalizedResource getBlob(LocalResource localResource, String user, String topo, File userFileDir) throws AuthorizationException, KeyNotFoundException, IOException { - ArrayList arr = new ArrayList(); + ArrayList arr = new ArrayList<>(); arr.add(localResource); List results = getBlobs(arr, user, topo, userFileDir); if (results.isEmpty() || results.size() != 1) { @@ -758,15 +705,15 @@ public LocalizedResource getBlob(LocalResource localResource, String user, Strin return results.get(0); } - protected boolean isLocalizedResourceDownloaded(LocalizedResource lrsrc) { + private boolean isLocalizedResourceDownloaded(LocalizedResource lrsrc) { File rsrcFileCurrent = new File(lrsrc.getCurrentSymlinkPath()); File rsrcFileWithVersion = new File(lrsrc.getFilePathWithVersion()); File versionFile = new File(lrsrc.getVersionFilePath()); return (rsrcFileWithVersion.exists() && rsrcFileCurrent.exists() && versionFile.exists()); } - protected boolean isLocalizedResourceUpToDate(LocalizedResource lrsrc, - ClientBlobStore blobstore) throws AuthorizationException, KeyNotFoundException { + private boolean isLocalizedResourceUpToDate(LocalizedResource lrsrc, + ClientBlobStore blobstore) throws AuthorizationException, KeyNotFoundException { String localFile = lrsrc.getFilePath(); long nimbusBlobVersion = ServerUtils.nimbusVersionOfBlob(lrsrc.getKey(), blobstore); long currentBlobVersion = ServerUtils.localVersionOfBlob(localFile); @@ -781,8 +728,8 @@ protected ClientBlobStore getClientBlobStore() { * This function updates blobs on the supervisor. It uses a separate thread pool and runs * asynchronously of the download and delete. */ - public List updateBlobs(List localResources, - String user) throws AuthorizationException, KeyNotFoundException, IOException { + List updateBlobs(List localResources, + String user) throws AuthorizationException, KeyNotFoundException, IOException { LocalizedResourceSet lrsrcSet = userRsrc.get(user); ArrayList results = new ArrayList<>(); ArrayList> updates = new ArrayList<>(); @@ -799,10 +746,8 @@ public List updateBlobs(List localResources, LocalizedResource lrsrc = lrsrcSet.get(key, localResource.shouldUncompress()); if (lrsrc == null) { LOG.warn("blob requested for update doesn't exist: {}", key); - continue; } else if ((boolean) conf.getOrDefault(Config.DISABLE_SYMLINKS, false)) { LOG.warn("symlinks are disabled so blobs cannot be downloaded."); - continue; } else { // update it if either the version isn't the latest or if any local blob files are missing if (!isLocalizedResourceUpToDate(lrsrc, blobstore) || @@ -855,8 +800,8 @@ public List updateBlobs(List localResources, * cache, it downloads them in parallel (up to SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT) * and will block until all of them have been downloaded */ - public synchronized List getBlobs(List localResources, - String user, String topo, File userFileDir) + synchronized List getBlobs(List localResources, + String user, String topo, File userFileDir) throws AuthorizationException, KeyNotFoundException, IOException { if ((boolean) conf.getOrDefault(Config.DISABLE_SYMLINKS, false)) { throw new KeyNotFoundException("symlinks are disabled so blobs cannot be downloaded."); @@ -942,8 +887,8 @@ static class DownloadBlob implements Callable { private boolean uncompress; private boolean isUpdate; - public DownloadBlob(AsyncLocalizer localizer, Map conf, String key, File localFile, - String user, boolean uncompress, boolean update) { + DownloadBlob(AsyncLocalizer localizer, Map conf, String key, File localFile, + String user, boolean uncompress, boolean update) { this.localizer = localizer; this.conf = conf; this.key = key; @@ -1098,7 +1043,7 @@ private LocalizedResource downloadBlob(Map conf, String key, Fil } } - public void setBlobPermissions(Map conf, String user, String path) + private void setBlobPermissions(Map conf, String user, String path) throws IOException { if (!ObjectReader.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) { @@ -1109,7 +1054,7 @@ public void setBlobPermissions(Map conf, String user, String pat String stormHome = System.getProperty("storm.home"); wlCommand = stormHome + "/bin/worker-launcher"; } - List command = new ArrayList(Arrays.asList(wlCommand, user, "blob", path)); + List command = new ArrayList<>(Arrays.asList(wlCommand, user, "blob", path)); String[] commandArray = command.toArray(new String[command.size()]); ShellUtils.ShellCommandExecutor shExec = new ShellUtils.ShellCommandExecutor(commandArray); @@ -1127,15 +1072,54 @@ public void setBlobPermissions(Map conf, String user, String pat } } + private interface ConsumePathAndId { + void accept(Path path, String topologyId) throws IOException; + } - public synchronized void cleanup() { + private void forEachTopologyDistDir(ConsumePathAndId consumer) throws IOException { + Path stormCodeRoot = Paths.get(ConfigUtils.supervisorStormDistRoot(conf)); + if (Files.exists(stormCodeRoot) && Files.isDirectory(stormCodeRoot)) { + try (DirectoryStream children = Files.newDirectoryStream(stormCodeRoot)) { + for (Path child : children) { + if (Files.isDirectory(child)) { + String topologyId = child.getFileName().toString(); + consumer.accept(child, topologyId); + } + } + } + } + } + + @VisibleForTesting + synchronized void cleanup() { LocalizedResourceRetentionSet toClean = new LocalizedResourceRetentionSet(cacheTargetSize); // need one large set of all and then clean via LRU for (LocalizedResourceSet t : userRsrc.values()) { toClean.addResources(t); LOG.debug("Resources to be cleaned after adding {} : {}", t.getUser(), toClean); } + + toClean.addResources(topologyBlobs); toClean.cleanup(); + + HashSet safeTopologyIds = new HashSet<>(); + for (String blobKey : topologyBlobs.keySet()) { + safeTopologyIds.add(ConfigUtils.getIdFromBlobKey(blobKey)); + } + + //Deleting this early does not hurt anything + topologyBasicDownloaded.keySet().removeIf(topoId -> !safeTopologyIds.contains(topoId)); + + try { + forEachTopologyDistDir((p, topologyId) -> { + if (!safeTopologyIds.contains(topologyId)) { + fsOps.deleteIfExists(p.toFile()); + } + }); + } catch (Exception e) { + LOG.error("Could not read topology directories for cleanup", e); + } + LOG.debug("Resource cleanup: {}", toClean); for (LocalizedResourceSet t : userRsrc.values()) { if (t.getSize() == 0) { @@ -1154,5 +1138,4 @@ public synchronized void cleanup() { } } } - } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/BlobChangingCallback.java b/storm-server/src/main/java/org/apache/storm/localizer/BlobChangingCallback.java new file mode 100644 index 00000000000..e99f95b28e0 --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/localizer/BlobChangingCallback.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.localizer; + +import org.apache.storm.generated.LocalAssignment; + +/** + * Callback for when a localized blob is going to change. + */ +public interface BlobChangingCallback { + + /** + * Informs the listener that a blob has changed and is ready to update and replace a localized blob that has been marked as + * tied to the life cycle of the worker process. + * + * If `go.getLatch()` is never called before the method completes it is assumed that + * the listener is good with the blob changing. + * @param assignment the assignment this resource and callback are registered with. + * @param port the port that this resource and callback are registered with. + * @param blob the blob that is going to change. + * @param go a way to indicate if the listener is ready for the resource to change. + */ + void blobChanging(LocalAssignment assignment, int port, LocallyCachedBlob blob, GoodToGo go); +} diff --git a/storm-server/src/main/java/org/apache/storm/localizer/GoodToGo.java b/storm-server/src/main/java/org/apache/storm/localizer/GoodToGo.java new file mode 100644 index 00000000000..04c7a06568f --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/localizer/GoodToGo.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.localizer; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Future; + +/** + * Used as a way to give feedback that the listener is ready for the caller to change the blob. + * By calling @{link GoodToGo#getLatch()} the listener indicates that it wants to block + * changing the blob until the CountDownLatch is triggered with a call to @{link CountDownLatch#countDown()}. + */ +public class GoodToGo { + public static class GoodToGoLatch { + private final CountDownLatch latch; + private final Future doneChangeing; + private boolean wasCounted = false; + + public GoodToGoLatch(CountDownLatch latch, Future doneChangeing) { + this.latch = latch; + this.doneChangeing = doneChangeing; + } + + public synchronized Future countDown() { + if (!wasCounted) { + latch.countDown(); + wasCounted = true; + } + return doneChangeing; + } + } + + private final GoodToGoLatch latch; + private boolean gotLatch = false; + + public GoodToGo(CountDownLatch latch, Future doneChangeing) { + this.latch = new GoodToGoLatch(latch, doneChangeing); + } + + /** + * Get the latch and indicate that you want to block the blob being changed. + * @return the latch to use when you are ready. + */ + public synchronized GoodToGoLatch getLatch() { + gotLatch = true; + return latch; + } + + synchronized void countDownIfLatchWasNotGotten() { + if (!gotLatch) { + latch.countDown(); + } + } +} diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocalDownloadedResource.java b/storm-server/src/main/java/org/apache/storm/localizer/LocalDownloadedResource.java index f019374d9f5..2d2e87983b0 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocalDownloadedResource.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocalDownloadedResource.java @@ -15,59 +15,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.localizer; import java.util.HashSet; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import org.apache.storm.generated.LocalAssignment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Used for accounting to keep track of who is waiting for specific resources to be downloaded. + */ public class LocalDownloadedResource { private static final Logger LOG = LoggerFactory.getLogger(LocalDownloadedResource.class); - private static class PortNAssignment { - private final int _port; - private final LocalAssignment _assignment; - - public PortNAssignment(int port, LocalAssignment assignment) { - _port = port; - _assignment = assignment; - } - - @Override - public boolean equals(Object other) { - if (!(other instanceof PortNAssignment)) { - return false; - } - PortNAssignment pna = (PortNAssignment) other; - return pna._port == _port && _assignment.equals(pna._assignment); - } - - @Override - public int hashCode() { - return (17 * _port) + _assignment.hashCode(); - } - - @Override - public String toString() { - return "{"+ _port + " " + _assignment +"}"; - } - } - private final CompletableFuture _pending; - private final Set _references; - private boolean _isDone; + private final CompletableFuture pending; + private final Set references; + private boolean isDone; public LocalDownloadedResource(CompletableFuture pending) { - _pending = pending; - _references = new HashSet<>(); - _isDone = false; + this.pending = pending; + references = new HashSet<>(); + isDone = false; } /** @@ -77,11 +49,11 @@ public LocalDownloadedResource(CompletableFuture pending) { * @return a future that can be used to track it being downloaded. */ public synchronized CompletableFuture reserve(int port, LocalAssignment la) { - PortNAssignment pna = new PortNAssignment(port, la); - if (!_references.add(pna)) { - LOG.warn("Resources {} already reserved {} for this topology", pna, _references); + PortAndAssignment pna = new PortAndAssignment(port, la); + if (!references.add(pna)) { + LOG.warn("Resources {} already reserved {} for this topology", pna, references); } - return _pending; + return pending; } /** @@ -91,10 +63,10 @@ public synchronized CompletableFuture reserve(int port, LocalAssignment la * @return true if the port was being counted else false */ public synchronized boolean release(int port, LocalAssignment la) { - PortNAssignment pna = new PortNAssignment(port, la); - boolean ret = _references.remove(pna); - if (ret && _references.isEmpty()) { - _isDone = true; + PortAndAssignment pna = new PortAndAssignment(port, la); + boolean ret = references.remove(pna); + if (ret && references.isEmpty()) { + isDone = true; } return ret; } @@ -104,11 +76,11 @@ public synchronized boolean release(int port, LocalAssignment la) { * @return true if it is done else false */ public synchronized boolean isDone() { - return _isDone; + return isDone; } @Override public String toString() { - return _references.toString(); + return references.toString(); } } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java index b6b5cd18e2d..7241976b24e 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java @@ -127,4 +127,17 @@ private long currentTime() { return System.nanoTime(); } + @Override + public boolean equals(Object other) { + if (other instanceof LocalizedResource) { + LocalizedResource l = (LocalizedResource)other; + return _key.equals(l._key) && _uncompressed == l._uncompressed && _localPath.equals(l._localPath); + } + return false; + } + + @Override + public int hashCode() { + return _key.hashCode() + Boolean.hashCode(_uncompressed) + _localPath.hashCode(); + } } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java index 9f42b47b416..826bf9869ce 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java @@ -17,6 +17,9 @@ */ package org.apache.storm.localizer; +import com.google.common.annotations.VisibleForTesting; +import java.nio.file.Path; +import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.io.FileUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,112 +32,325 @@ import java.util.Map; import java.util.SortedMap; import java.util.TreeMap; + /** * A set of resources that we can look at to see which ones we retain and which ones should be * removed. */ public class LocalizedResourceRetentionSet { - public static final Logger LOG = LoggerFactory.getLogger(LocalizedResourceRetentionSet.class); - private long _delSize; - private long _currentSize; - // targetSize in Bytes - private long _targetSize; - private final SortedMap _noReferences; - - LocalizedResourceRetentionSet(long targetSize) { - this(targetSize, new LRUComparator()); - } - - LocalizedResourceRetentionSet(long targetSize, Comparator cmp) { - this(targetSize, new TreeMap(cmp)); - } - - LocalizedResourceRetentionSet(long targetSize, - SortedMap retain) { - this._noReferences = retain; - this._targetSize = targetSize; - } - - // for testing - protected int getSizeWithNoReferences() { - return _noReferences.size(); - } - - protected void addResourcesForSet(Iterator setIter, LocalizedResourceSet set) { - for (Iterator iter = setIter; setIter.hasNext(); ) { - LocalizedResource lrsrc = iter.next(); - _currentSize += lrsrc.getSize(); - if (lrsrc.getRefCount() > 0) { - // always retain resources in use - continue; - } - LOG.debug("adding {} to be checked for cleaning", lrsrc.getKey()); - _noReferences.put(lrsrc, set); - } - } - - public void addResources(LocalizedResourceSet set) { - addResourcesForSet(set.getLocalFilesIterator(), set); - addResourcesForSet(set.getLocalArchivesIterator(), set); - } - - public void cleanup() { - LOG.debug("cleanup target size: {} current size is: {}", _targetSize, _currentSize); - for (Iterator> i = - _noReferences.entrySet().iterator(); - _currentSize - _delSize > _targetSize && i.hasNext();) { - Map.Entry rsrc = i.next(); - LocalizedResource resource = rsrc.getKey(); - LocalizedResourceSet set = rsrc.getValue(); - if (resource != null && set.remove(resource)) { - if (deleteResource(resource)) { - _delSize += resource.getSize(); - LOG.info("deleting: " + resource.getFilePath() + " size of: " + resource.getSize()); - i.remove(); - } else { - // since it failed to delete add it back so it gets retried - set.add(resource.getKey(), resource, resource.isUncompressed()); - } - } - } - } - - protected boolean deleteResource(LocalizedResource resource){ - try { - String fileWithVersion = resource.getFilePathWithVersion(); - String currentSymlinkName = resource.getCurrentSymlinkPath(); - String versionFile = resource.getVersionFilePath(); - File deletePath = new File(fileWithVersion); - if (resource.isUncompressed()) { - // this doesn't follow symlinks, which is what we want - FileUtils.deleteDirectory(deletePath); - } else { - Files.delete(deletePath.toPath()); - } - Files.delete(new File(currentSymlinkName).toPath()); - Files.delete(new File(versionFile).toPath()); - return true; - } catch (IOException e) { - LOG.warn("Could not delete: {}", resource.getFilePath()); - } - return false; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("Cache: ").append(_currentSize).append(", "); - sb.append("Deleted: ").append(_delSize); - return sb.toString(); - } - - static class LRUComparator implements Comparator { - public int compare(LocalizedResource r1, LocalizedResource r2) { - long ret = r1.getLastAccessTime() - r2.getLastAccessTime(); - if (0 == ret) { - return System.identityHashCode(r1) - System.identityHashCode(r2); - } - return ret > 0 ? 1 : -1; - } - } + public static final Logger LOG = LoggerFactory.getLogger(LocalizedResourceRetentionSet.class); + private long delSize; + private long currentSize; + // targetSize in Bytes + private long targetSize; + @VisibleForTesting + final SortedMap noReferences; + private int resourceCount = 0; + + LocalizedResourceRetentionSet(long targetSize) { + this(targetSize, new LRUComparator()); + } + + LocalizedResourceRetentionSet(long targetSize, Comparator cmp) { + this(targetSize, new TreeMap<>(cmp)); + } + + LocalizedResourceRetentionSet(long targetSize, + SortedMap retain) { + this.noReferences = retain; + this.targetSize = targetSize; + } + + // for testing + protected int getSizeWithNoReferences() { + return noReferences.size(); + } + + protected void addResourcesForSet(Iterator setIter, LocalizedResourceSet set) { + CleanableLocalizedResourceSet cleanset = new CleanableLocalizedResourceSet(set); + for (Iterator iter = setIter; setIter.hasNext(); ) { + LocalizedResource lrsrc = iter.next(); + currentSize += lrsrc.getSize(); + resourceCount ++; + if (lrsrc.getRefCount() > 0) { + // always retain resources in use + continue; + } + noReferences.put(new LocalizedBlobComparableResource(lrsrc), cleanset); + } + } + + public void addResources(LocalizedResourceSet set) { + addResourcesForSet(set.getLocalFilesIterator(), set); + addResourcesForSet(set.getLocalArchivesIterator(), set); + } + + public void addResources(ConcurrentHashMap blobs) { + CleanableLocalizedLocallyCachedBlob set = new CleanableLocalizedLocallyCachedBlob(blobs); + for (LocallyCachedBlob b: blobs.values()) { + currentSize += b.getSizeOnDisk(); + resourceCount ++; + if (b.isUsed()) { + // always retain resources in use + continue; + } + LocallyCachedBlobComparableResource cb = new LocallyCachedBlobComparableResource(b); + noReferences.put(cb, set); + } + } + + public void cleanup() { + LOG.debug("cleanup target size: {} current size is: {}", targetSize, currentSize); + for (Iterator> i = + noReferences.entrySet().iterator(); + currentSize - delSize > targetSize && i.hasNext();) { + Map.Entry rsrc = i.next(); + ComparableResource resource = rsrc.getKey(); + CleanableResourceSet set = rsrc.getValue(); + if (resource != null && set.remove(resource)) { + if (set.deleteUnderlyingResource(resource)) { + delSize += resource.getSize(); + LOG.info("deleting: {} with size of: {}", resource.getNameForDebug(), resource.getSize()); + i.remove(); + } else { + // since it failed to delete add it back so it gets retried + set.add(resource.getKey(), resource); + } + } + } + } + + @VisibleForTesting + public boolean deleteResource(CleanableResourceSet set, ComparableResource resource) { + return set.deleteUnderlyingResource(resource); + } + + public long getCurrentSize() { + return currentSize; + } + + public int getResourceCount() { + return resourceCount; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("Cache: ").append(currentSize).append(", "); + sb.append("Deleted: ").append(delSize); + return sb.toString(); + } + + interface ComparableResource { + long getLastAccessTime(); + + long getSize(); + + String getNameForDebug(); + + String getKey(); + } + + interface CleanableResourceSet { + boolean remove(ComparableResource resource); + + void add(String key, ComparableResource resource); + + boolean deleteUnderlyingResource(ComparableResource resource); + } + + public static class LocallyCachedBlobComparableResource implements ComparableResource { + private final LocallyCachedBlob blob; + + public LocallyCachedBlobComparableResource(LocallyCachedBlob blob) { + this.blob = blob; + } + + @Override + public long getLastAccessTime() { + return blob.getLastUsed(); + } + + @Override + public long getSize() { + return blob.getSizeOnDisk(); + } + + @Override + public String getNameForDebug() { + return blob.getKey(); + } + + @Override + public String getKey() { + return blob.getKey(); + } + + @Override + public String toString() { + return blob.toString(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof LocallyCachedBlobComparableResource) { + return blob.equals(((LocallyCachedBlobComparableResource) other).blob); + } + return false; + } + + @Override + public int hashCode() { + return blob.hashCode(); + } + } + + private static class CleanableLocalizedLocallyCachedBlob implements CleanableResourceSet { + private final ConcurrentHashMap blobs; + + public CleanableLocalizedLocallyCachedBlob(ConcurrentHashMap blobs) { + this.blobs = blobs; + } + + @Override + public boolean remove(ComparableResource resource) { + if (!(resource instanceof LocallyCachedBlobComparableResource)) { + throw new IllegalStateException(resource + " must be a LocallyCachedBlobComparableResource"); + } + LocallyCachedBlob blob = ((LocallyCachedBlobComparableResource)resource).blob; + synchronized (blob) { + if (!blob.isUsed()) { + try { + blob.completelyRemove(); + } catch (Exception e) { + LOG.warn("Tried to remove {} but failed with", blob, e); + } + blobs.remove(blob.getKey()); + return true; + } + return false; + } + } + + @Override + public void add(String key, ComparableResource resource) { + ///NOOP not used + } + + @Override + public boolean deleteUnderlyingResource(ComparableResource resource) { + //NOOP not used + return true; + } + } + + private static class LocalizedBlobComparableResource implements ComparableResource { + private final LocalizedResource resource; + + private LocalizedBlobComparableResource(LocalizedResource resource) { + this.resource = resource; + } + + @Override + public long getLastAccessTime() { + return resource.getLastAccessTime(); + } + + @Override + public long getSize() { + return resource.getSize(); + } + + @Override + public String getNameForDebug() { + return resource.getFilePath(); + } + + @Override + public String getKey() { + return resource.getKey(); + } + + @Override + public String toString() { + return resource.getKey() + " at " + resource.getFilePathWithVersion(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof LocalizedBlobComparableResource) { + return resource.equals(((LocalizedBlobComparableResource) other).resource); + } + return false; + } + + @Override + public int hashCode() { + return resource.hashCode(); + } + } + + private static class CleanableLocalizedResourceSet implements CleanableResourceSet { + private final LocalizedResourceSet set; + + public CleanableLocalizedResourceSet(LocalizedResourceSet set) { + this.set = set; + } + + @Override + public boolean remove(ComparableResource resource) { + if (!(resource instanceof LocalizedBlobComparableResource)) { + throw new IllegalStateException(resource + " must be a LocalizedBlobComparableResource"); + } + return set.remove(((LocalizedBlobComparableResource)resource).resource); + } + + @Override + public void add(String key, ComparableResource resource) { + if (!(resource instanceof LocalizedBlobComparableResource)) { + throw new IllegalStateException(resource + " must be a LocalizedBlobComparableResource"); + } + LocalizedResource r = ((LocalizedBlobComparableResource)resource).resource; + set.add(key, r, r.isUncompressed()); + } + + @Override + public boolean deleteUnderlyingResource(ComparableResource resource) { + if (resource instanceof LocalizedBlobComparableResource) { + LocalizedResource lr = ((LocalizedBlobComparableResource) resource).resource; + try { + Path fileWithVersion = new File(lr.getFilePathWithVersion()).toPath(); + Path currentSymLink = new File(lr.getCurrentSymlinkPath()).toPath(); + Path versionFile = new File(lr.getVersionFilePath()).toPath(); + + if (lr.isUncompressed()) { + if (Files.exists(fileWithVersion)) { + // this doesn't follow symlinks, which is what we want + FileUtils.deleteDirectory(fileWithVersion.toFile()); + } + } else { + Files.deleteIfExists(fileWithVersion); + } + Files.deleteIfExists(currentSymLink); + Files.deleteIfExists(versionFile); + return true; + } catch (IOException e) { + LOG.warn("Could not delete: {}", resource.getNameForDebug(), e); + } + return false; + } else { + throw new IllegalArgumentException("Don't know how to handle a " + resource.getClass()); + } + } + } + + static class LRUComparator implements Comparator { + public int compare(ComparableResource r1, ComparableResource r2) { + long ret = r1.getLastAccessTime() - r2.getLastAccessTime(); + if (0 == ret) { + return System.identityHashCode(r1) - System.identityHashCode(r2); + } + return ret > 0 ? 1 : -1; + } + } } diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java new file mode 100644 index 00000000000..c09108da1cf --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java @@ -0,0 +1,220 @@ +/** + * 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.localizer; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.LinkOption; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.storm.blobstore.BlobStore; +import org.apache.storm.blobstore.ClientBlobStore; +import org.apache.storm.generated.AuthorizationException; +import org.apache.storm.generated.KeyNotFoundException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Represents a blob that is cached locally on disk by the supervisor. + */ +public abstract class LocallyCachedBlob { + private static final Logger LOG = LoggerFactory.getLogger(LocallyCachedBlob.class); + public static final long NOT_DOWNLOADED_VERSION = -1; + // A callback that does nothing. + private static final BlobChangingCallback NOOP_CB = (assignment, port, resource, go) -> {}; + + private long lastUsed = System.currentTimeMillis(); + private final Map references = new HashMap<>(); + private final String blobDescription; + private final String blobKey; + private CompletableFuture doneUpdating = null; + + /** + * Create a new LocallyCachedBlob. + * @param blobDescription a description of the blob this represents. Typically it should at least be the blob key, but ideally also + * include if it is an archive or not, what user or topology it is for, or if it is a storm.jar etc. + */ + protected LocallyCachedBlob(String blobDescription, String blobKey) { + this.blobDescription = blobDescription; + this.blobKey = blobKey; + } + + /** + * Get the version of the blob cached locally. If the version is unknown or it has not been downloaded NOT_DOWNLOADED_VERSION + * should be returned. + * PRECONDITION: this can only be called with a lock on this instance held. + */ + public abstract long getLocalVersion(); + + /** + * Get the version of the blob in the blob store. + * PRECONDITION: this can only be called with a lock on this instance held. + */ + public abstract long getRemoteVersion(ClientBlobStore store) throws KeyNotFoundException, AuthorizationException; + + /** + * Download the latest version to a temp location. This may also include unzipping some or all of the data to a temp location. + * PRECONDITION: this can only be called with a lock on this instance held. + * @param store the store to us to download the data. + * @return the version that was downloaded. + */ + public abstract long downloadToTempLocation(ClientBlobStore store) throws IOException, KeyNotFoundException, AuthorizationException; + + /** + * Commit the new version and make it available for the end user. + * PRECONDITION: uncompressToTempLocationIfNeeded will have been called. + * PRECONDITION: this can only be called with a lock on this instance held. + * @param version the version of the blob to commit. + */ + public abstract void commitNewVersion(long version) throws IOException; + + /** + * Clean up any temporary files. This will be called after updating a blob, either successfully or if an error has occured. + * The goal is to find any files that may be left over and remove them so space is not leaked. + * PRECONDITION: this can only be called with a lock on this instance held. + */ + public abstract void cleanupOrphanedData() throws IOException; + + /** + * Completely remove anything that is cached locally for this blob and all tracking files also stored for it. + * This will be called after the blob was determined to no longer be needed in the cache. + * PRECONDITION: this can only be called with a lock on this instance held. + */ + public abstract void completelyRemove() throws IOException; + + /** + * Get the amount of disk space that is used by this blob. If the blob is uncompressed it should be the sum of the space used by all + * of the uncompressed files. In general this will not be called with any locks held so it is a good idea to cache it and updated it + * when committing a new version. + */ + public abstract long getSizeOnDisk(); + + /** + * Updates the last updated time. This should be called when references are added or removed. + */ + private synchronized void touch() { + lastUsed = System.currentTimeMillis(); + } + + /** + * Get the last time that this used for LRU calculations. + */ + public synchronized long getLastUsed() { + return lastUsed; + } + + /** + * Return true if this blob is actively being used, else false (meaning it can be deleted, but might not be). + */ + public synchronized boolean isUsed() { + return !references.isEmpty(); + } + + /** + * Get the size of p in bytes. + * @param p the path to read. + * @return the size of p in bytes. + */ + protected long getSizeOnDisk(Path p) throws IOException { + if (!Files.exists(p)) { + return 0; + } else if (Files.isRegularFile(p)) { + return Files.size(p); + } else { + //We will not follow sym links + return Files.walk(p) + .filter((subp) -> Files.isRegularFile(subp, LinkOption.NOFOLLOW_LINKS)) + .mapToLong((subp) -> { + try { + return Files.size(subp); + } catch (IOException e) { + LOG.warn("Could not get the size of "); + } + return 0; + }).sum(); + } + } + + /** + * Mark that a given port and assignemnt are using this. + * @param pna the slot and assignment that are using this blob. + * @param cb an optional callback indicating that they want to know/synchronize when a blob is updated. + */ + public void addReference(final PortAndAssignment pna, BlobChangingCallback cb) { + if (cb == null) { + cb = NOOP_CB; + } + if (references.put(pna, cb) != null) { + LOG.warn("{} already has a reservation for {}", pna, blobDescription); + } + } + + /** + * Removes a reservation for this blob from a given slot and assignemnt. + * @param pna the slot + assignment that no longer needs this blob. + */ + public void removeReference(final PortAndAssignment pna) { + if (references.remove(pna) == null) { + LOG.warn("{} had no reservation for {}", pna, blobDescription); + } + } + + /** + * Inform all of the callbacks that a change is going to happen and then wait for + * them to all get back that it is OK to make that change. + */ + public synchronized void informAllOfChangeAndWaitForConsensus() { + CountDownLatch cdl = new CountDownLatch(references.size()); + doneUpdating = new CompletableFuture<>(); + for (Map.Entry entry : references.entrySet()) { + GoodToGo gtg = new GoodToGo(cdl, doneUpdating); + try { + PortAndAssignment pna = entry.getKey(); + BlobChangingCallback cb = entry.getValue(); + //TODO we probably want to not use this, or make it just return something that has less power to modify things + cb.blobChanging(pna.getAssignment(), pna.getPort(), this, gtg); + } finally { + gtg.countDownIfLatchWasNotGotten(); + } + } + try { + cdl.await(3, TimeUnit.MINUTES); + } catch (InterruptedException e) { + //TODO need to think about error handling here in general. + } + } + + /** + * Inform all of the callbacks that the change to the blob is complete. + */ + public synchronized void informAllChangeComplete() { + doneUpdating.complete(null); + } + + /** + * Get the key for this blob. + */ + public String getKey() { + return blobKey; + } +} diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java new file mode 100644 index 00000000000..35371b53797 --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java @@ -0,0 +1,364 @@ +/** + * 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.localizer; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.JarURLConnection; +import java.net.URL; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Enumeration; +import java.util.Map; +import java.util.jar.JarEntry; +import java.util.jar.JarFile; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.storm.blobstore.ClientBlobStore; +import org.apache.storm.blobstore.InputStreamWithMeta; +import org.apache.storm.daemon.supervisor.AdvancedFSOps; +import org.apache.storm.generated.AuthorizationException; +import org.apache.storm.generated.KeyNotFoundException; +import org.apache.storm.utils.ConfigUtils; +import org.apache.storm.utils.ServerConfigUtils; +import org.apache.storm.utils.ServerUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A locally cached blob for the topology. storm.jar, stormcode.ser, or stormconf.ser. + * The version number of the blob's file will be stored in `${basename}.version` + */ +public class LocallyCachedTopologyBlob extends LocallyCachedBlob { + private static final Logger LOG = LoggerFactory.getLogger(LocallyCachedTopologyBlob.class); + public static final long LOCAL_MODE_JAR_VERSION = 1; + + private static String resourcesJar() throws IOException { + String path = ServerUtils.currentClasspath(); + if (path == null) { + return null; + } + + for (String jpath : path.split(File.pathSeparator)) { + if (jpath.endsWith(".jar")) { + if (ServerUtils.zipDoesContainDir(jpath, ServerConfigUtils.RESOURCES_SUBDIR)) { + return jpath; + } + } + } + return null; + } + + public enum TopologyBlobType { + TOPO_JAR("stormjar.jar", "-stormjar.jar", "resources"), + TOPO_CODE("stormcode.ser", "-stormcode.ser", null), + TOPO_CONF("stormconf.ser", "-stormconf.ser", null); + + private final String fileName; + private final String keySuffix; + private final String extractionDir; + + TopologyBlobType(String fileName, String keySuffix, String extractionDir) { + this.fileName = fileName; + this.keySuffix = keySuffix; + this.extractionDir = extractionDir; + } + + public String getFileName() { + return fileName; + } + + public String getTempFileName(long version) { + return fileName + "." + version; + } + + public String getVersionFileName() { + return fileName + ".version"; + } + + public String getKey(String topologyId) { + return topologyId + keySuffix; + } + + public boolean needsExtraction() { + return extractionDir != null; + } + + public String getExtractionDir() { + return extractionDir; + } + + public String getTempExtractionDir(long version) { + return extractionDir + "." + version; + } + }; + + private final TopologyBlobType type; + private final String topologyId; + private final boolean isLocalMode; + private final Path topologyBasicBlobsRootDir; + private final AdvancedFSOps fsOps; + private final Map conf; + private volatile long version = NOT_DOWNLOADED_VERSION; + private volatile long size = 0; + + /** + * Create a new LocallyCachedBlob. + * + * @param topologyId the ID of the topology. + * @param type the type of the blob. + */ + protected LocallyCachedTopologyBlob(final String topologyId, final boolean isLocalMode, final Map conf, + final AdvancedFSOps fsOps, final TopologyBlobType type) throws IOException { + super(topologyId + " " + type.getFileName(), type.getKey(topologyId)); + this.topologyId = topologyId; + this.type = type; + this.isLocalMode = isLocalMode; + this.fsOps = fsOps; + this.conf = conf; + topologyBasicBlobsRootDir = Paths.get(ConfigUtils.supervisorStormDistRoot(conf, topologyId)); + readVersion(); + updateSizeOnDisk(); + } + + private void updateSizeOnDisk() throws IOException { + long total = getSizeOnDisk(topologyBasicBlobsRootDir.resolve(type.getFileName())); + if (type.needsExtraction()) { + total += getSizeOnDisk(topologyBasicBlobsRootDir.resolve(type.getExtractionDir())); + } + size = total; + } + + private void readVersion() throws IOException { + Path versionFile = topologyBasicBlobsRootDir.resolve(type.getVersionFileName()); + if (!fsOps.fileExists(versionFile)) { + version = NOT_DOWNLOADED_VERSION; + } else { + String ver = FileUtils.readFileToString(versionFile.toFile(), "UTF8").trim(); + version = Long.parseLong(ver); + } + } + + @Override + public long getLocalVersion() { + LOG.debug("LOCAL VERSION {}/{} is {}", type, topologyId, version); + return version; + } + + @Override + public long getRemoteVersion(ClientBlobStore store) throws KeyNotFoundException, AuthorizationException { + if (isLocalMode && type == TopologyBlobType.TOPO_JAR) { + LOG.debug("REMOTE VERSION LOCAL JAR {}", LOCAL_MODE_JAR_VERSION); + return LOCAL_MODE_JAR_VERSION; + } + return store.getBlobMeta(type.getKey(topologyId)).get_version(); + } + + @Override + public long downloadToTempLocation(ClientBlobStore store) + throws IOException, KeyNotFoundException, AuthorizationException { + if (isLocalMode && type == TopologyBlobType.TOPO_JAR) { + LOG.debug("DOWNLOADING LOCAL JAR to TEMP LOCATION... {}", topologyId); + //This is a special case where the jar was not uploaded so we will not download it (it is already on the classpath) + ClassLoader classloader = Thread.currentThread().getContextClassLoader(); + String resourcesJar = resourcesJar(); + URL url = classloader.getResource(ServerConfigUtils.RESOURCES_SUBDIR); + Path extractionDest = topologyBasicBlobsRootDir.resolve(type.getTempExtractionDir(LOCAL_MODE_JAR_VERSION)); + if (resourcesJar != null) { + LOG.info("Extracting resources from jar at {} to {}", resourcesJar, extractionDest); + extractDirFromJar(resourcesJar, ServerConfigUtils.RESOURCES_SUBDIR, extractionDest); + } else if (url != null) { + LOG.info("Copying resources at {} to {}", url, extractionDest); + if ("jar".equals(url.getProtocol())) { + JarURLConnection urlConnection = (JarURLConnection) url.openConnection(); + extractDirFromJar(urlConnection.getJarFileURL().getFile(), ServerConfigUtils.RESOURCES_SUBDIR, extractionDest); + } else { + fsOps.copyDirectory(new File(url.getFile()), extractionDest.toFile()); + } + } + return LOCAL_MODE_JAR_VERSION; + } + + long newVersion; + Path tmpLocation; + String key = type.getKey(topologyId); + try (InputStreamWithMeta in = store.getBlob(key)) { + newVersion = in.getVersion(); + long expectedSize = in.getFileLength(); + if (newVersion == version) { + throw new RuntimeException("The version did not change, but we tried to download it. " + version + " " + key); + } + tmpLocation = topologyBasicBlobsRootDir.resolve(type.getTempFileName(newVersion)); + long totalRead = 0; + //Make sure the parent directory is there and ready to go + fsOps.forceMkdir(tmpLocation.getParent()); + try (OutputStream outStream = fsOps.getOutputStream(tmpLocation.toFile())) { + byte [] buffer = new byte[4096]; + int read = 0; + while ((read = in.read(buffer)) > 0) { + outStream.write(buffer, 0, read); + totalRead += read; + } + } + if (totalRead != expectedSize) { + throw new IOException("We expected to download " + expectedSize + " bytes but found we got " + totalRead); + } + } + + if (type.needsExtraction()) { + Path extractionDest = topologyBasicBlobsRootDir.resolve(type.getTempExtractionDir(newVersion)); + extractDirFromJar(tmpLocation.toAbsolutePath().toString(), ServerConfigUtils.RESOURCES_SUBDIR, + extractionDest); + } + return newVersion; + } + + protected void extractDirFromJar(String jarpath, String dir, Path dest) throws IOException { + LOG.debug("EXTRACTING {} from {} and placing it at {}", dir, jarpath, dest); + try (JarFile jarFile = new JarFile(jarpath)) { + String toRemove = dir + '/'; + Enumeration jarEnums = jarFile.entries(); + while (jarEnums.hasMoreElements()) { + JarEntry entry = jarEnums.nextElement(); + String name = entry.getName(); + if (!entry.isDirectory() && name.startsWith(toRemove)) { + String shortenedName = name.replace(toRemove, ""); + Path aFile = dest.resolve(shortenedName); + LOG.debug("EXTRACTING {} SHORTENED to {} into {}", name, shortenedName, aFile); + fsOps.forceMkdir(aFile.getParent()); + try (FileOutputStream out = new FileOutputStream(aFile.toFile()); + InputStream in = jarFile.getInputStream(entry)) { + IOUtils.copy(in, out); + } + } + } + } + } + + @Override + public void commitNewVersion(long newVersion) throws IOException { + //This is not atomic (so if something bad happens in the middle we need to be able to recover + Path tempLoc = topologyBasicBlobsRootDir.resolve(type.getTempFileName(newVersion)); + Path dest = topologyBasicBlobsRootDir.resolve(type.getFileName()); + Path versionFile = topologyBasicBlobsRootDir.resolve(type.getVersionFileName()); + + LOG.debug("Removing version file {} to force download on failure", versionFile); + fsOps.deleteIfExists(versionFile.toFile()); //So if we fail we are forced to try again + LOG.debug("Removing destination file {} in preparation for move", dest); + fsOps.deleteIfExists(dest.toFile()); + if (type.needsExtraction()) { + Path extractionTemp = topologyBasicBlobsRootDir.resolve(type.getTempExtractionDir(newVersion)); + Path extractionDest = topologyBasicBlobsRootDir.resolve(type.getExtractionDir()); + LOG.debug("Removing extraction dest {} in preparation for extraction", extractionDest); + fsOps.deleteIfExists(extractionDest.toFile()); + if (fsOps.fileExists(extractionTemp)) { + fsOps.moveDirectoryPreferAtomic(extractionTemp.toFile(), extractionDest.toFile()); + } + } + if (!(isLocalMode && type == TopologyBlobType.TOPO_JAR)) { + //Don't try to move the JAR file in local mode, it does not exist because it was not uploaded + Files.move(tempLoc, dest); + } + LOG.debug("Writing out version file {} with version {}", versionFile, newVersion); + FileUtils.write(versionFile.toFile(), Long.toString(newVersion), "UTF8"); + this.version = newVersion; + updateSizeOnDisk(); + LOG.debug("New version of {} - {} committed {}", topologyId, type, newVersion); + } + + @Override + public void cleanupOrphanedData() throws IOException { + cleanUpTemp(type.getFileName()); + if (type.needsExtraction()) { + cleanUpTemp(type.getExtractionDir()); + } + } + + private static final Pattern EXTRACT_BASE_NAME_AND_VERSION = Pattern.compile("^(.*)\\.([0-9]+)$"); + + private void cleanUpTemp(String baseName) throws IOException { + LOG.debug("Cleaning up temporary data in {}", topologyBasicBlobsRootDir); + try (DirectoryStream children = fsOps.newDirectoryStream(topologyBasicBlobsRootDir, + (p) -> { + String fileName = p.getFileName().toString(); + Matcher m = EXTRACT_BASE_NAME_AND_VERSION.matcher(fileName); + return m.matches() && baseName.equals(m.group(1)); + })) { + //children is only ever null if topologyBasicBlobsRootDir does not exist. This happens during unit tests + // And because a non-existant directory is by definition clean we are ignoring it. + if (children != null) { + for (Path p : children) { + LOG.debug("Cleaning up {}", p); + fsOps.deleteIfExists(p.toFile()); + } + } + } + } + + @Override + public void completelyRemove() throws IOException { + removeAll(type.getFileName()); + if (type.needsExtraction()) { + removeAll(type.getExtractionDir()); + } + } + + private void removeAll(String baseName) throws IOException { + try (DirectoryStream children = fsOps.newDirectoryStream(topologyBasicBlobsRootDir)) { + for (Path p : children) { + String fileName = p.getFileName().toString(); + if (fileName.startsWith(baseName)) { + fsOps.deleteIfExists(p.toFile()); + } + } + } + } + + @Override + public long getSizeOnDisk() { + return size; + } + + @Override + public boolean equals(Object other) { + if (other instanceof LocallyCachedTopologyBlob) { + LocallyCachedTopologyBlob o = (LocallyCachedTopologyBlob)other; + return topologyId.equals(o.topologyId) && type == o.type && topologyBasicBlobsRootDir.equals(o.topologyBasicBlobsRootDir); + } + return false; + } + + @Override + public int hashCode() { + return topologyId.hashCode() + type.hashCode() + topologyBasicBlobsRootDir.hashCode(); + } + + @Override + public String toString() { + return "LOCAL TOPO BLOB " + type + " " + topologyId; + } +} diff --git a/storm-server/src/main/java/org/apache/storm/localizer/PortAndAssignment.java b/storm-server/src/main/java/org/apache/storm/localizer/PortAndAssignment.java new file mode 100644 index 00000000000..081c8117493 --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/localizer/PortAndAssignment.java @@ -0,0 +1,71 @@ +/** + * 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.localizer; + +import org.apache.storm.generated.LocalAssignment; + +/** + * A Port and a LocalAssignment used to reference count Resources + */ +class PortAndAssignment { + private final int port; + private final LocalAssignment assignment; + + public PortAndAssignment(int port, LocalAssignment assignment) { + this.port = port; + this.assignment = assignment; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof PortAndAssignment)) { + return false; + } + PortAndAssignment pna = (PortAndAssignment) other; + return pna.port == port && assignment.equals(pna.assignment); + } + + public String getToplogyId() { + return assignment.get_topology_id(); + } + + @Override + public int hashCode() { + return (17 * port) + assignment.hashCode(); + } + + @Override + public String toString() { + return "{" + port + " " + assignment + "}"; + } + + /** + * Return the port associated with this. + */ + public int getPort() { + return port; + } + + /** + * return the assigment for this. + */ + public LocalAssignment getAssignment() { + return assignment; + } +} diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/ResourceUtils.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/ResourceUtils.java index a50dcedb050..ab8eb14de9f 100644 --- a/storm-server/src/main/java/org/apache/storm/scheduler/resource/ResourceUtils.java +++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/ResourceUtils.java @@ -24,6 +24,7 @@ import org.apache.storm.Config; import org.apache.storm.generated.Bolt; +import org.apache.storm.generated.ComponentCommon; import org.apache.storm.generated.SpoutSpec; import org.apache.storm.generated.StormTopology; import org.apache.storm.utils.ObjectReader; @@ -63,6 +64,71 @@ public static Map> getSpoutsResources(StormTopology return spoutResources; } + public static void updateStormTopologyResources(StormTopology topology, Map> resourceUpdatesMap) { + Map> componentsUpdated = new HashMap<>(); + if (topology.get_spouts() != null) { + for (Map.Entry spout : topology.get_spouts().entrySet()) { + SpoutSpec spoutSpec = spout.getValue(); + String spoutName = spout.getKey(); + + if (resourceUpdatesMap.containsKey(spoutName)) { + ComponentCommon spoutCommon = spoutSpec.get_common(); + Map resourcesUpdate = resourceUpdatesMap.get(spoutName); + String newJsonConf = getJsonWithUpdatedResources(spoutCommon.get_json_conf(), resourcesUpdate); + spoutCommon.set_json_conf(newJsonConf); + componentsUpdated.put(spoutName, resourcesUpdate); + } + } + } + + if (topology.get_bolts() != null) { + for (Map.Entry bolt : topology.get_bolts().entrySet()) { + Bolt boltObj = bolt.getValue(); + String boltName = bolt.getKey(); + + if(resourceUpdatesMap.containsKey(boltName)) { + ComponentCommon boltCommon = boltObj.get_common(); + Map resourcesUpdate = resourceUpdatesMap.get(boltName); + String newJsonConf = getJsonWithUpdatedResources(boltCommon.get_json_conf(), resourceUpdatesMap.get(boltName)); + boltCommon.set_json_conf(newJsonConf); + componentsUpdated.put(boltName, resourcesUpdate); + } + } + } + LOG.info("Component resources updated: {}", componentsUpdated); + Map> notUpdated = new HashMap>(); + for (String component : resourceUpdatesMap.keySet()) { + if (!componentsUpdated.containsKey(component)) { + notUpdated.put(component, resourceUpdatesMap.get(component)); + } + } + LOG.info("Component resource updates ignored: {}", notUpdated); + } + + public static String getJsonWithUpdatedResources(String jsonConf, Map resourceUpdates) { + try { + JSONParser parser = new JSONParser(); + Object obj = parser.parse(jsonConf); + JSONObject jsonObject = (JSONObject) obj; + + if (resourceUpdates.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB)) { + Double topoMemOnHeap = resourceUpdates.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB); + jsonObject.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, topoMemOnHeap); + } + if (resourceUpdates.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)) { + Double topoMemOffHeap = resourceUpdates.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB); + jsonObject.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, topoMemOffHeap); + } + if (resourceUpdates.containsKey(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT)) { + Double topoCPU = resourceUpdates.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT); + jsonObject.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, topoCPU); + } + return jsonObject.toJSONString(); + } catch (ParseException ex) { + throw new RuntimeException("Failed to parse component resources with json: " + jsonConf); + } + } + public static void checkIntialization(Map topologyResources, String com, Map topologyConf) { checkInitMem(topologyResources, com, topologyConf); diff --git a/storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java b/storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java index 9f2b95b5ab0..6a4454a0a85 100644 --- a/storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java +++ b/storm-server/src/main/java/org/apache/storm/utils/ServerUtils.java @@ -18,6 +18,7 @@ package org.apache.storm.utils; +import javax.security.auth.Subject; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; import org.apache.commons.exec.CommandLine; @@ -32,6 +33,7 @@ import org.apache.storm.blobstore.ClientBlobStore; import org.apache.storm.blobstore.InputStreamWithMeta; import org.apache.storm.blobstore.LocalFsBlobStore; +import org.apache.storm.blobstore.LocalModeClientBlobStore; import org.apache.storm.daemon.StormCommon; import org.apache.storm.generated.AccessControl; import org.apache.storm.generated.AccessControlType; @@ -43,6 +45,7 @@ import org.apache.storm.generated.StormTopology; import org.apache.storm.nimbus.NimbusInfo; import org.apache.storm.scheduler.resource.ResourceUtils; +import org.apache.storm.security.auth.SingleUserPrincipal; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -212,8 +215,13 @@ public static String constructBlobWithVersionFileName(String fileName, long vers } public static ClientBlobStore getClientBlobStoreForSupervisor(Map conf) { - ClientBlobStore store = (ClientBlobStore) ReflectionUtils.newInstance( + ClientBlobStore store; + if (ConfigUtils.isLocalMode(conf)) { + store = new LocalModeClientBlobStore(getNimbusBlobStore(conf, null)); + } else { + store = (ClientBlobStore) ReflectionUtils.newInstance( (String) conf.get(DaemonConfig.SUPERVISOR_BLOBSTORE)); + } store.prepare(conf); return store; } @@ -767,4 +775,11 @@ public static int getComponentParallelism(Map topoConf, Object c } return ret; } + + public static Subject principalNameToSubject(String name) { + SingleUserPrincipal principal = new SingleUserPrincipal(name); + Subject sub = new Subject(); + sub.getPrincipals().add(principal); + return sub; + } } diff --git a/storm-server/src/test/java/org/apache/storm/TestRebalance.java b/storm-server/src/test/java/org/apache/storm/TestRebalance.java new file mode 100644 index 00000000000..bd72b3a81ec --- /dev/null +++ b/storm-server/src/test/java/org/apache/storm/TestRebalance.java @@ -0,0 +1,170 @@ +/** + * 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; + +import org.apache.storm.generated.ClusterSummary; +import org.apache.storm.generated.RebalanceOptions; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.generated.TopologySummary; +import org.apache.storm.scheduler.resource.ResourceAwareScheduler; +import org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler; +import org.apache.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy; +import org.apache.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy; +import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy; +import org.apache.storm.topology.BoltDeclarer; +import org.apache.storm.topology.SpoutDeclarer; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.utils.Utils; +import org.apache.thrift.TException; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class TestRebalance { + + static final int SLEEP_TIME_BETWEEN_RETRY = 1000; + + private static final Logger LOG = LoggerFactory.getLogger(TestRebalance.class); + + @Test + public void testRebalanceTopologyResourcesAndConfigs() + throws Exception { + + LOG.info("Starting local cluster..."); + + Config conf = new Config(); + conf.put(DaemonConfig.STORM_SCHEDULER, ResourceAwareScheduler.class.getName()); + conf.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY, DefaultEvictionStrategy.class.getName()); + conf.put(DaemonConfig.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY, DefaultSchedulingPriorityStrategy.class.getName()); + conf.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, DefaultResourceAwareStrategy.class.getName()); + conf.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 10.0); + conf.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 10.0); + conf.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 100.0); + conf.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, Double.MAX_VALUE); + + try (ILocalCluster cluster = new LocalCluster.Builder().withDaemonConf(conf).build()) { + + TopologyBuilder builder = new TopologyBuilder(); + SpoutDeclarer s1 = builder.setSpout("spout-1", new TestUtilsForResourceAwareScheduler.TestSpout(), + 2); + BoltDeclarer b1 = builder.setBolt("bolt-1", new TestUtilsForResourceAwareScheduler.TestBolt(), + 2).shuffleGrouping("spout-1"); + BoltDeclarer b2 = builder.setBolt("bolt-2", new TestUtilsForResourceAwareScheduler.TestBolt(), + 2).shuffleGrouping("bolt-1"); + + StormTopology stormTopology = builder.createTopology(); + + LOG.info("submitting topologies..."); + String topoName = "topo1"; + cluster.submitTopology(topoName, new HashMap<>(), stormTopology); + + waitTopologyScheduled(topoName, cluster, 20); + + RebalanceOptions opts = new RebalanceOptions(); + + Map> resources = new HashMap>(); + resources.put("spout-1", new HashMap()); + resources.get("spout-1").put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, 120.0); + resources.get("spout-1").put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, 25.0); + + opts.set_topology_resources_overrides(resources); + opts.set_wait_secs(0); + + JSONObject jsonObject = new JSONObject(); + jsonObject.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, 768.0); + + opts.set_topology_conf_overrides(jsonObject.toJSONString()); + + LOG.info("rebalancing..."); + cluster.rebalance("topo1", opts); + + waitTopologyScheduled(topoName, cluster, 10); + + String confRaw = cluster.getTopologyConf(topoNameToId(topoName, cluster)); + + JSONParser parser = new JSONParser(); + + JSONObject readConf = (JSONObject) parser.parse(confRaw); + assertEquals("updated conf correct", 768.0, (double) readConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB), 0.001); + + StormTopology readStormTopology = cluster.getTopology(topoNameToId(topoName, cluster)); + String componentConfRaw = readStormTopology.get_spouts().get("spout-1").get_common().get_json_conf(); + + JSONObject readTopologyConf = (JSONObject) parser.parse(componentConfRaw); + + assertEquals("Updated CPU correct", 25.0, (double) readTopologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), 0.001); + assertEquals("Updated Memory correct", 120.0, (double) readTopologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), 0.001); + } + } + + public void waitTopologyScheduled(String topoName, ILocalCluster cluster, int retryAttempts) throws TException { + for (int i = 0; i < retryAttempts; i++) { + if (checkTopologyScheduled(topoName, cluster)) { + //sleep to prevent race conditions + Utils.sleep(SLEEP_TIME_BETWEEN_RETRY); + return; + } + Utils.sleep(SLEEP_TIME_BETWEEN_RETRY); + } + throw new RuntimeException("Error: Wait for topology " + topoName + " to be ACTIVE has timed out!"); + } + + public boolean checkTopologyScheduled(String topoName, ILocalCluster cluster) throws TException { + if (checkTopologyUp(topoName, cluster)) { + ClusterSummary sum = cluster.getClusterInfo(); + for (TopologySummary topoSum : sum.get_topologies()) { + if (topoSum.get_name().equals(topoName)) { + String status = topoSum.get_status(); + String sched_status = topoSum.get_sched_status(); + if (status.equals("ACTIVE") && (sched_status != null && !sched_status.equals(""))) { + return true; + } + } + } + } + return false; + } + + public boolean checkTopologyUp(String topoName, ILocalCluster cluster) throws TException { + ClusterSummary sum = cluster.getClusterInfo(); + + for (TopologySummary topoSum : sum.get_topologies()) { + if (topoSum.get_name().equals(topoName)) { + return true; + } + } + return false; + } + + public static String topoNameToId(String topoName, ILocalCluster cluster) throws TException { + for (TopologySummary topoSum : cluster.getClusterInfo().get_topologies()) { + if (topoSum.get_name().equals(topoName)) { + return topoSum.get_id(); + } + } + return null; + } +} \ No newline at end of file diff --git a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java index 60b628e41ea..927225d4c26 100644 --- a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java +++ b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import org.apache.storm.daemon.supervisor.Slot.StaticState; @@ -40,13 +41,20 @@ import org.apache.storm.generated.ProfileRequest; import org.apache.storm.generated.WorkerResources; import org.apache.storm.localizer.AsyncLocalizer; +import org.apache.storm.localizer.BlobChangingCallback; +import org.apache.storm.localizer.GoodToGo; +import org.apache.storm.localizer.LocallyCachedBlob; import org.apache.storm.scheduler.ISupervisor; import org.apache.storm.utils.LocalState; import org.apache.storm.utils.Time; import org.apache.storm.utils.Time.SimulatedTime; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SlotTest { + private static final Logger LOG = LoggerFactory.getLogger(SlotTest.class); + static WorkerResources mkWorkerResources(Double cpu, Double mem_on_heap, Double mem_off_heap) { WorkerResources resources = new WorkerResources(); if (cpu != null) { @@ -111,16 +119,36 @@ public void testEquivilant() { assertFalse(Slot.equivalent(null, b)); assertFalse(Slot.equivalent(a, b)); } - + + @Test + public void testForSameTopology() { + LocalAssignment a = mkLocalAssignment("A", mkExecutorInfoList(1,2,3,4,5), mkWorkerResources(100.0, 100.0, 100.0)); + LocalAssignment aResized = mkLocalAssignment("A", mkExecutorInfoList(1,2,3,4,5), mkWorkerResources(100.0, 200.0, 100.0)); + LocalAssignment b = mkLocalAssignment("B", mkExecutorInfoList(1,2,3,4,5,6), mkWorkerResources(100.0, 100.0, 100.0)); + LocalAssignment bReordered = mkLocalAssignment("B", mkExecutorInfoList(6,5,4,3,2,1), mkWorkerResources(100.0, 100.0, 100.0)); + + assertTrue(Slot.forSameTopology(null, null)); + assertTrue(Slot.forSameTopology(a, a)); + assertTrue(Slot.forSameTopology(a, aResized)); + assertTrue(Slot.forSameTopology(aResized, a)); + assertTrue(Slot.forSameTopology(b, bReordered)); + assertTrue(Slot.forSameTopology(bReordered, b)); + + assertFalse(Slot.forSameTopology(a, null)); + assertFalse(Slot.forSameTopology(null, b)); + assertFalse(Slot.forSameTopology(a, b)); + } + @Test public void testEmptyToEmpty() throws Exception { try (SimulatedTime t = new SimulatedTime(1010)){ AsyncLocalizer localizer = mock(AsyncLocalizer.class); LocalState state = mock(LocalState.class); + BlobChangingCallback cb = mock(BlobChangingCallback.class); ContainerLauncher containerLauncher = mock(ContainerLauncher.class); ISupervisor iSuper = mock(ISupervisor.class); StaticState staticState = new StaticState(localizer, 1000, 1000, 1000, 1000, - containerLauncher, "localhost", 8080, iSuper, state); + containerLauncher, "localhost", 8080, iSuper, state, cb); DynamicState dynamicState = new DynamicState(null, null, null); DynamicState nextState = Slot.handleEmpty(dynamicState, staticState); assertEquals(MachineState.EMPTY, nextState.state); @@ -136,39 +164,28 @@ public void testLaunchContainerFromEmpty() throws Exception { List execList = mkExecutorInfoList(1,2,3,4,5); LocalAssignment newAssignment = mkLocalAssignment(topoId, execList, mkWorkerResources(100.0, 100.0, 100.0)); - + AsyncLocalizer localizer = mock(AsyncLocalizer.class); + BlobChangingCallback cb = mock(BlobChangingCallback.class); Container container = mock(Container.class); LocalState state = mock(LocalState.class); ContainerLauncher containerLauncher = mock(ContainerLauncher.class); when(containerLauncher.launchContainer(port, newAssignment, state)).thenReturn(container); LSWorkerHeartbeat hb = mkWorkerHB(topoId, port, execList, Time.currentTimeSecs()); when(container.readHeartbeat()).thenReturn(hb, hb); - - @SuppressWarnings("unchecked") - CompletableFuture baseFuture = mock(CompletableFuture.class); - when(localizer.requestDownloadBaseTopologyBlobs(newAssignment, port)).thenReturn(baseFuture); - + @SuppressWarnings("unchecked") CompletableFuture blobFuture = mock(CompletableFuture.class); - when(localizer.requestDownloadTopologyBlobs(newAssignment, port)).thenReturn(blobFuture); + when(localizer.requestDownloadTopologyBlobs(newAssignment, port, cb)).thenReturn(blobFuture); ISupervisor iSuper = mock(ISupervisor.class); StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000, - containerLauncher, "localhost", port, iSuper, state); + containerLauncher, "localhost", port, iSuper, state, cb); DynamicState dynamicState = new DynamicState(null, null, null) .withNewAssignment(newAssignment); - + DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState); - verify(localizer).requestDownloadBaseTopologyBlobs(newAssignment, port); - assertEquals(MachineState.WAITING_FOR_BASIC_LOCALIZATION, nextState.state); - assertSame("pendingDownload not set properly", baseFuture, nextState.pendingDownload); - assertEquals(newAssignment, nextState.pendingLocalization); - assertEquals(0, Time.currentTimeMillis()); - - nextState = Slot.stateMachineStep(nextState, staticState); - verify(baseFuture).get(1000, TimeUnit.MILLISECONDS); - verify(localizer).requestDownloadTopologyBlobs(newAssignment, port); + verify(localizer).requestDownloadTopologyBlobs(newAssignment, port, cb); assertEquals(MachineState.WAITING_FOR_BLOB_LOCALIZATION, nextState.state); assertSame("pendingDownload not set properly", blobFuture, nextState.pendingDownload); assertEquals(newAssignment, nextState.pendingLocalization); @@ -221,6 +238,7 @@ public void testRelaunch() throws Exception { mkLocalAssignment(topoId, execList, mkWorkerResources(100.0, 100.0, 100.0)); AsyncLocalizer localizer = mock(AsyncLocalizer.class); + BlobChangingCallback cb = mock(BlobChangingCallback.class); Container container = mock(Container.class); ContainerLauncher containerLauncher = mock(ContainerLauncher.class); LSWorkerHeartbeat oldhb = mkWorkerHB(topoId, port, execList, Time.currentTimeSecs()-10); @@ -231,7 +249,7 @@ public void testRelaunch() throws Exception { ISupervisor iSuper = mock(ISupervisor.class); LocalState state = mock(LocalState.class); StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000, - containerLauncher, "localhost", port, iSuper, state); + containerLauncher, "localhost", port, iSuper, state, cb); DynamicState dynamicState = new DynamicState(assignment, container, assignment); DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState); @@ -265,7 +283,9 @@ public void testReschedule() throws Exception { List cExecList = mkExecutorInfoList(1,2,3,4,5); LocalAssignment cAssignment = mkLocalAssignment(cTopoId, cExecList, mkWorkerResources(100.0, 100.0, 100.0)); - + + BlobChangingCallback cb = mock(BlobChangingCallback.class); + Container cContainer = mock(Container.class); LSWorkerHeartbeat chb = mkWorkerHB(cTopoId, port, cExecList, Time.currentTimeSecs()); when(cContainer.readHeartbeat()).thenReturn(chb); @@ -284,48 +304,36 @@ public void testReschedule() throws Exception { LSWorkerHeartbeat nhb = mkWorkerHB(nTopoId, 100, nExecList, Time.currentTimeSecs()); when(nContainer.readHeartbeat()).thenReturn(nhb, nhb); - @SuppressWarnings("unchecked") - CompletableFuture baseFuture = mock(CompletableFuture.class); - when(localizer.requestDownloadBaseTopologyBlobs(nAssignment, port)).thenReturn(baseFuture); - @SuppressWarnings("unchecked") CompletableFuture blobFuture = mock(CompletableFuture.class); - when(localizer.requestDownloadTopologyBlobs(nAssignment, port)).thenReturn(blobFuture); + when(localizer.requestDownloadTopologyBlobs(nAssignment, port, cb)).thenReturn(blobFuture); ISupervisor iSuper = mock(ISupervisor.class); StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000, - containerLauncher, "localhost", port, iSuper, state); + containerLauncher, "localhost", port, iSuper, state, cb); DynamicState dynamicState = new DynamicState(cAssignment, cContainer, nAssignment); DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState); assertEquals(MachineState.KILL, nextState.state); verify(cContainer).kill(); - verify(localizer).requestDownloadBaseTopologyBlobs(nAssignment, port); - assertSame("pendingDownload not set properly", baseFuture, nextState.pendingDownload); + verify(localizer).requestDownloadTopologyBlobs(nAssignment, port, cb); + assertSame("pendingDownload not set properly", blobFuture, nextState.pendingDownload); assertEquals(nAssignment, nextState.pendingLocalization); assertTrue(Time.currentTimeMillis() > 1000); nextState = Slot.stateMachineStep(nextState, staticState); assertEquals(MachineState.KILL, nextState.state); verify(cContainer).forceKill(); - assertSame("pendingDownload not set properly", baseFuture, nextState.pendingDownload); + assertSame("pendingDownload not set properly", blobFuture, nextState.pendingDownload); assertEquals(nAssignment, nextState.pendingLocalization); assertTrue(Time.currentTimeMillis() > 2000); nextState = Slot.stateMachineStep(nextState, staticState); - assertEquals(MachineState.WAITING_FOR_BASIC_LOCALIZATION, nextState.state); + assertEquals(MachineState.WAITING_FOR_BLOB_LOCALIZATION, nextState.state); verify(cContainer).cleanUp(); verify(localizer).releaseSlotFor(cAssignment, port); assertTrue(Time.currentTimeMillis() > 2000); - nextState = Slot.stateMachineStep(nextState, staticState); - assertEquals(MachineState.WAITING_FOR_BLOB_LOCALIZATION, nextState.state); - verify(baseFuture).get(1000, TimeUnit.MILLISECONDS); - verify(localizer).requestDownloadTopologyBlobs(nAssignment, port); - assertSame("pendingDownload not set properly", blobFuture, nextState.pendingDownload); - assertEquals(nAssignment, nextState.pendingLocalization); - assertTrue(Time.currentTimeMillis() > 2000); - nextState = Slot.stateMachineStep(nextState, staticState); verify(blobFuture).get(1000, TimeUnit.MILLISECONDS); verify(containerLauncher).launchContainer(port, nAssignment, state); @@ -361,7 +369,6 @@ public void testReschedule() throws Exception { assertTrue(Time.currentTimeMillis() > 4000); } } - @Test public void testRunningToEmpty() throws Exception { @@ -378,18 +385,19 @@ public void testRunningToEmpty() throws Exception { when(cContainer.areAllProcessesDead()).thenReturn(false, true); AsyncLocalizer localizer = mock(AsyncLocalizer.class); + BlobChangingCallback cb = mock(BlobChangingCallback.class); ContainerLauncher containerLauncher = mock(ContainerLauncher.class); ISupervisor iSuper = mock(ISupervisor.class); LocalState state = mock(LocalState.class); StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000, - containerLauncher, "localhost", port, iSuper, state); + containerLauncher, "localhost", port, iSuper, state, cb); DynamicState dynamicState = new DynamicState(cAssignment, cContainer, null); DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState); assertEquals(MachineState.KILL, nextState.state); verify(cContainer).kill(); - verify(localizer, never()).requestDownloadBaseTopologyBlobs(null, port); + verify(localizer, never()).requestDownloadTopologyBlobs(null, port, cb); assertSame("pendingDownload not set properly", null, nextState.pendingDownload); assertEquals(null, nextState.pendingLocalization); assertTrue(Time.currentTimeMillis() > 1000); @@ -438,12 +446,13 @@ public void testRunWithProfileActions() throws Exception { when(cContainer.runProfiling(any(ProfileRequest.class), anyBoolean())).thenReturn(true); AsyncLocalizer localizer = mock(AsyncLocalizer.class); + BlobChangingCallback cb = mock(BlobChangingCallback.class); ContainerLauncher containerLauncher = mock(ContainerLauncher.class); ISupervisor iSuper = mock(ISupervisor.class); LocalState state = mock(LocalState.class); StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000, - containerLauncher, "localhost", port, iSuper, state); + containerLauncher, "localhost", port, iSuper, state, cb); Set profileActions = new HashSet<>(); ProfileRequest request = new ProfileRequest(); request.set_action(ProfileAction.JPROFILE_STOP); @@ -495,4 +504,103 @@ public void testRunWithProfileActions() throws Exception { assertTrue(Time.currentTimeMillis() > 5000); } } + + @Test + public void testResourcesChanged() throws Exception { + try (SimulatedTime t = new SimulatedTime(1010)){ + int port = 8080; + String cTopoId = "CURRENT"; + List cExecList = mkExecutorInfoList(1,2,3,4,5); + LocalAssignment cAssignment = + mkLocalAssignment(cTopoId, cExecList, mkWorkerResources(100.0, 100.0, 100.0)); + + BlobChangingCallback cb = mock(BlobChangingCallback.class); + + Container cContainer = mock(Container.class); + LSWorkerHeartbeat chb = mkWorkerHB(cTopoId, port, cExecList, Time.currentTimeSecs()); + when(cContainer.readHeartbeat()).thenReturn(chb); + when(cContainer.areAllProcessesDead()).thenReturn(false, true); + + AsyncLocalizer localizer = mock(AsyncLocalizer.class); + Container nContainer = mock(Container.class); + LocalState state = mock(LocalState.class); + ContainerLauncher containerLauncher = mock(ContainerLauncher.class); + when(containerLauncher.launchContainer(port, cAssignment, state)).thenReturn(nContainer); + when(nContainer.readHeartbeat()).thenReturn(chb, chb); + + ISupervisor iSuper = mock(ISupervisor.class); + StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000, + containerLauncher, "localhost", port, iSuper, state, cb); + + Set changing = new HashSet<>(); + LocallyCachedBlob stormJar = mock(LocallyCachedBlob.class); + GoodToGo.GoodToGoLatch stormJarLatch = mock(GoodToGo.GoodToGoLatch.class); + CompletableFuture stormJarLatchFuture = mock(CompletableFuture.class); + when(stormJarLatch.countDown()).thenReturn(stormJarLatchFuture); + changing.add(new Slot.BlobChangeing(cAssignment, stormJar, stormJarLatch)); + + DynamicState dynamicState = new DynamicState(cAssignment, cContainer, cAssignment).withChangingBlobs(changing); + + DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState); + assertEquals(MachineState.KILL_BLOB_UPDATE, nextState.state); + verify(iSuper).killedWorker(port); + verify(cContainer).kill(); + verify(localizer, never()).requestDownloadTopologyBlobs(any(), anyInt(), any()); + assertNull(nextState.pendingDownload); + assertNull(nextState.pendingLocalization); + assertEquals(changing, nextState.changingBlobs); + assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertNull(nextState.pendingChangingBlobsAssignment); + assertTrue(Time.currentTimeMillis() > 1000); + + nextState = Slot.stateMachineStep(nextState, staticState); + assertEquals(MachineState.KILL_BLOB_UPDATE, nextState.state); + verify(cContainer).forceKill(); + assertNull(nextState.pendingDownload); + assertNull(nextState.pendingLocalization); + assertEquals(changing, nextState.changingBlobs); + assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertNull(nextState.pendingChangingBlobsAssignment); + assertTrue(Time.currentTimeMillis() > 2000); + + nextState = Slot.stateMachineStep(nextState, staticState); + assertEquals(MachineState.WAITING_FOR_BLOB_UPDATE, nextState.state); + verify(cContainer).cleanUp(); + assertTrue(Time.currentTimeMillis() > 2000); + + nextState = Slot.stateMachineStep(nextState, staticState); + verify(stormJarLatchFuture).get(anyLong(), any()); + verify(containerLauncher).launchContainer(port, cAssignment, state); + assertEquals(MachineState.WAITING_FOR_WORKER_START, nextState.state); + assertNull(nextState.pendingChangingBlobsAssignment); + assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertSame(cAssignment, nextState.currentAssignment); + assertSame(nContainer, nextState.container); + assertTrue(Time.currentTimeMillis() > 2000); + + nextState = Slot.stateMachineStep(nextState, staticState); + assertEquals(MachineState.RUNNING, nextState.state); + assertNull(nextState.pendingChangingBlobsAssignment); + assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertSame(cAssignment, nextState.currentAssignment); + assertSame(nContainer, nextState.container); + assertTrue(Time.currentTimeMillis() > 2000); + + nextState = Slot.stateMachineStep(nextState, staticState); + assertEquals(MachineState.RUNNING, nextState.state); + assertNull(nextState.pendingChangingBlobsAssignment); + assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertSame(cAssignment, nextState.currentAssignment); + assertSame(nContainer, nextState.container); + assertTrue(Time.currentTimeMillis() > 3000); + + nextState = Slot.stateMachineStep(nextState, staticState); + assertEquals(MachineState.RUNNING, nextState.state); + assertNull(nextState.pendingChangingBlobsAssignment); + assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertSame(cAssignment, nextState.currentAssignment); + assertSame(nContainer, nextState.container); + assertTrue(Time.currentTimeMillis() > 4000); + } + } } diff --git a/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java b/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java index 34e9e8ee68a..65f22def116 100644 --- a/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java +++ b/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java @@ -18,6 +18,7 @@ package org.apache.storm.localizer; +import static org.apache.storm.blobstore.BlobStoreAclHandler.WORLD_EVERYTHING; import static org.apache.storm.localizer.AsyncLocalizer.USERCACHE; import static org.junit.Assert.*; import static org.junit.Assert.assertEquals; @@ -46,6 +47,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -87,6 +89,36 @@ private static String getTestLocalizerRoot() { return f.getPath(); } + private class MockInputStreamWithMeta extends InputStreamWithMeta { + private int at = 0; + private final int len; + private final int version; + + public MockInputStreamWithMeta(int len, int version) { + this.len = len; + this.version = version; + } + + @Override + public long getVersion() throws IOException { + return version; + } + + @Override + public long getFileLength() throws IOException { + return len; + } + + @Override + public int read() throws IOException { + at++; + if (at > len) { + return -1; + } + return 0; + } + } + @Test public void testRequestDownloadBaseTopologyBlobs() throws Exception { final String topoId = "TOPO"; @@ -99,12 +131,7 @@ public void testRequestDownloadBaseTopologyBlobs() throws Exception { ei.set_task_end(1); la.add_to_executors(ei); final int port = 8080; - final String jarKey = topoId + "-stormjar.jar"; - final String codeKey = topoId + "-stormcode.ser"; - final String confKey = topoId + "-stormconf.ser"; - final String stormLocal = "/tmp/storm-local/"; - final String stormRoot = stormLocal+topoId+"/"; - final File fStormRoot = new File(stormRoot); + final String stormLocal = "./target/DOWNLOAD-TEST/storm-local/"; ClientBlobStore blobStore = mock(ClientBlobStore.class); Map conf = new HashMap<>(); conf.put(DaemonConfig.SUPERVISOR_BLOBSTORE, ClientBlobStore.class.getName()); @@ -112,40 +139,55 @@ public void testRequestDownloadBaseTopologyBlobs() throws Exception { conf.put(Config.STORM_CLUSTER_MODE, "distributed"); conf.put(Config.STORM_LOCAL_DIR, stormLocal); AdvancedFSOps ops = mock(AdvancedFSOps.class); - ConfigUtils mockedCU = mock(ConfigUtils.class); ReflectionUtils mockedRU = mock(ReflectionUtils.class); ServerUtils mockedU = mock(ServerUtils.class); - - Map topoConf = new HashMap<>(conf); - - AsyncLocalizer bl = new AsyncLocalizer(conf, getTestLocalizerRoot(), ops, new AtomicReference<>(new HashMap<>()), null); - ConfigUtils orig = ConfigUtils.setInstance(mockedCU); + + AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, getTestLocalizerRoot(), new AtomicReference<>(new HashMap<>()), null)); + LocallyCachedTopologyBlob jarBlob = mock(LocallyCachedTopologyBlob.class); + doReturn(jarBlob).when(bl).getTopoJar(topoId); + when(jarBlob.getLocalVersion()).thenReturn(-1L); + when(jarBlob.getRemoteVersion(any())).thenReturn(100L); + when(jarBlob.downloadToTempLocation(any())).thenReturn(100L); + + LocallyCachedTopologyBlob codeBlob = mock(LocallyCachedTopologyBlob.class); + doReturn(codeBlob).when(bl).getTopoCode(topoId); + when(codeBlob.getLocalVersion()).thenReturn(-1L); + when(codeBlob.getRemoteVersion(any())).thenReturn(200L); + when(codeBlob.downloadToTempLocation(any())).thenReturn(200L); + + LocallyCachedTopologyBlob confBlob = mock(LocallyCachedTopologyBlob.class); + doReturn(confBlob).when(bl).getTopoConf(topoId); + when(confBlob.getLocalVersion()).thenReturn(-1L); + when(confBlob.getRemoteVersion(any())).thenReturn(300L); + when(confBlob.downloadToTempLocation(any())).thenReturn(300L); + ReflectionUtils origRU = ReflectionUtils.setInstance(mockedRU); ServerUtils origUtils = ServerUtils.setInstance(mockedU); try { - when(mockedCU.supervisorStormDistRootImpl(conf, topoId)).thenReturn(stormRoot); - when(mockedCU.supervisorLocalDirImpl(conf)).thenReturn(stormLocal); when(mockedRU.newInstanceImpl(ClientBlobStore.class)).thenReturn(blobStore); - when(mockedCU.readSupervisorStormConfImpl(conf, topoId)).thenReturn(topoConf); - Future f = bl.requestDownloadBaseTopologyBlobs(la, port); + Future f = bl.requestDownloadBaseTopologyBlobs(la, port, null); f.get(20, TimeUnit.SECONDS); - // We should be done now... - - verify(blobStore).prepare(conf); - verify(mockedU).downloadResourcesAsSupervisorImpl(eq(jarKey), startsWith(stormLocal), eq(blobStore)); - verify(mockedU).downloadResourcesAsSupervisorImpl(eq(codeKey), startsWith(stormLocal), eq(blobStore)); - verify(mockedU).downloadResourcesAsSupervisorImpl(eq(confKey), startsWith(stormLocal), eq(blobStore)); - verify(blobStore).shutdown(); - //Extracting the dir from the jar - verify(mockedU).extractDirFromJarImpl(endsWith("stormjar.jar"), eq("resources"), any(File.class)); - verify(ops).moveDirectoryPreferAtomic(any(File.class), eq(fStormRoot)); - verify(ops).setupStormCodeDir(user, fStormRoot); - - verify(ops, never()).deleteIfExists(any(File.class)); + + verify(jarBlob).downloadToTempLocation(any()); + verify(jarBlob).informAllOfChangeAndWaitForConsensus(); + verify(jarBlob).commitNewVersion(100L); + verify(jarBlob).informAllChangeComplete(); + verify(jarBlob).cleanupOrphanedData(); + + verify(codeBlob).downloadToTempLocation(any()); + verify(codeBlob).informAllOfChangeAndWaitForConsensus(); + verify(codeBlob).commitNewVersion(200L); + verify(codeBlob).informAllChangeComplete(); + verify(codeBlob).cleanupOrphanedData(); + + verify(confBlob).downloadToTempLocation(any()); + verify(confBlob).informAllOfChangeAndWaitForConsensus(); + verify(confBlob).commitNewVersion(300L); + verify(confBlob).informAllChangeComplete(); + verify(confBlob).cleanupOrphanedData(); } finally { bl.close(); - ConfigUtils.setInstance(orig); ReflectionUtils.setInstance(origRU); ServerUtils.setInstance(origUtils); } @@ -199,7 +241,7 @@ public void testRequestDownloadTopologyBlobs() throws Exception { LocalizedResource simpleLocal = new LocalizedResource(simpleKey, simpleLocalFile, false); localizedList.add(simpleLocal); - AsyncLocalizer bl = spy(new AsyncLocalizer(conf, localizerRoot, ops, new AtomicReference<>(new HashMap<>()), null)); + AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, localizerRoot, new AtomicReference<>(new HashMap<>()), null)); ConfigUtils orig = ConfigUtils.setInstance(mockedCU); try { when(mockedCU.supervisorStormDistRootImpl(conf, topoId)).thenReturn(stormRoot); @@ -207,10 +249,12 @@ public void testRequestDownloadTopologyBlobs() throws Exception { when(mockedCU.readSupervisorTopologyImpl(conf, topoId, ops)).thenReturn(st); //Write the mocking backwards so the actual method is not called on the spy object + doReturn(CompletableFuture.supplyAsync(() -> null)).when(bl) + .requestDownloadBaseTopologyBlobs(la, port, null); doReturn(userDir).when(bl).getLocalUserFileCacheDir(user); doReturn(localizedList).when(bl).getBlobs(any(List.class), eq(user), eq(topoName), eq(userDir)); - Future f = bl.requestDownloadTopologyBlobs(la, port); + Future f = bl.requestDownloadTopologyBlobs(la, port, null); f.get(20, TimeUnit.SECONDS); // We should be done now... @@ -242,7 +286,7 @@ public void testRequestDownloadTopologyBlobs() throws Exception { class TestLocalizer extends AsyncLocalizer { TestLocalizer(Map conf, String baseDir) throws IOException { - super(conf, baseDir, AdvancedFSOps.make(conf), new AtomicReference<>(new HashMap<>()), null); + super(conf, AdvancedFSOps.make(conf), baseDir, new AtomicReference<>(new HashMap<>()), null); } @Override @@ -460,7 +504,7 @@ public void testArchives(File archiveFile, boolean supportSymlinks, int size) th localizer.setTargetCacheSize(1); ReadableBlobMeta rbm = new ReadableBlobMeta(); - rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING)); + rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm); when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(new @@ -537,7 +581,7 @@ public void testBasic() throws Exception { localizer.setTargetCacheSize(1); ReadableBlobMeta rbm = new ReadableBlobMeta(); - rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING)); + rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm); when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta()); @@ -606,7 +650,7 @@ public void testMultipleKeysOneUser() throws Exception { localizer.setTargetCacheSize(68); ReadableBlobMeta rbm = new ReadableBlobMeta(); - rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING)); + rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); when(mockblobstore.getBlobMeta(anyString())).thenReturn(rbm); when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta()); when(mockblobstore.getBlob(key2)).thenReturn(new TestInputStreamWithMeta()); @@ -736,7 +780,7 @@ public void testMultipleUsers() throws Exception { localizer.setTargetCacheSize(68); ReadableBlobMeta rbm = new ReadableBlobMeta(); - rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING)); + rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); when(mockblobstore.getBlobMeta(anyString())).thenReturn(rbm); when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta()); when(mockblobstore.getBlob(key2)).thenReturn(new TestInputStreamWithMeta()); @@ -816,7 +860,7 @@ public void testUpdate() throws Exception { ReadableBlobMeta rbm = new ReadableBlobMeta(); rbm.set_version(1); - rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING)); + rbm.set_settable(new SettableBlobMeta(WORLD_EVERYTHING)); when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm); when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta()); diff --git a/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java b/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java index 04b5ab2b0b9..150bea579ec 100644 --- a/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java +++ b/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java @@ -17,9 +17,12 @@ */ package org.apache.storm.localizer; + import org.junit.Test; import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -37,16 +40,21 @@ public void testAddResources() throws Exception { lrset.add("key1", localresource1, false); lrset.add("key2", localresource2, false); lrretset.addResources(lrset); - assertEquals("number to clean is not 0", 0, lrretset.getSizeWithNoReferences()); + assertEquals("number to clean is not 0 " + lrretset.noReferences, 0, lrretset.getSizeWithNoReferences()); localresource1.removeReference(("topo1")); + lrretset = new LocalizedResourceRetentionSet(10); lrretset.addResources(lrset); - assertEquals("number to clean is not 1", 1, lrretset.getSizeWithNoReferences()); + assertEquals("number to clean is not 1 " + lrretset.noReferences, 1, lrretset.getSizeWithNoReferences()); + localresource2.removeReference(("topo1")); + lrretset = new LocalizedResourceRetentionSet(10); lrretset.addResources(lrset); - assertEquals("number to clean is not 1", 1, lrretset.getSizeWithNoReferences()); + assertEquals("number to clean is not 1 " + lrretset.noReferences, 1, lrretset.getSizeWithNoReferences()); + localresource2.removeReference(("topo2")); + lrretset = new LocalizedResourceRetentionSet(10); lrretset.addResources(lrset); - assertEquals("number to clean is not 2", 2, lrretset.getSizeWithNoReferences()); + assertEquals("number to clean is not 2 " + lrretset.noReferences, 2, lrretset.getSizeWithNoReferences()); } @Test @@ -54,13 +62,13 @@ public void testCleanup() throws Exception { LocalizedResourceRetentionSet lrretset = spy(new LocalizedResourceRetentionSet(10)); LocalizedResourceSet lrset = new LocalizedResourceSet("user1"); // no reference to key1 - LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false); + LocalizedResource localresource1 = new LocalizedResource("key1", "./target/TESTING/testfile1", false); localresource1.setSize(10); // no reference to archive1 - LocalizedResource archiveresource1 = new LocalizedResource("archive1", "testarchive1", true); + LocalizedResource archiveresource1 = new LocalizedResource("archive1", "./target/TESTING/testarchive1", true); archiveresource1.setSize(20); // reference to key2 - LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", false, "topo1"); + LocalizedResource localresource2 = new LocalizedResource("key2", "./target/TESTING/testfile2", false, "topo1"); // check adding reference to local resource with topology of same name localresource2.addReference(("topo1")); localresource2.setSize(10); @@ -71,14 +79,9 @@ public void testCleanup() throws Exception { lrretset.addResources(lrset); assertEquals("number to clean is not 2", 2, lrretset.getSizeWithNoReferences()); - // shouldn't change number since file doesn't exist and delete fails - lrretset.cleanup(); - assertEquals("resource cleaned up", 2, lrretset.getSizeWithNoReferences()); + // make deleteUnderlyingResource return true even though file doesn't exist + doReturn(true).when(lrretset).deleteResource(any(), any()); - // make deleteResource return true even though file doesn't exist - when(lrretset.deleteResource(localresource1)).thenReturn(true); - when(lrretset.deleteResource(localresource2)).thenReturn(true); - when(lrretset.deleteResource(archiveresource1)).thenReturn(true); lrretset.cleanup(); assertEquals("resource not cleaned up", 0, lrretset.getSizeWithNoReferences()); } From d1e6bbb50ee5dbfb595d27e3630205d29ff975c2 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 9 Oct 2017 16:27:36 -0500 Subject: [PATCH 2/2] STORM-2483: Addressed review comments --- .../apache/storm/daemon/supervisor/Slot.java | 82 +++++++++---------- .../org/apache/storm/localizer/GoodToGo.java | 12 +-- .../storm/localizer/LocallyCachedBlob.java | 6 +- .../storm/daemon/supervisor/SlotTest.java | 17 ++-- 4 files changed, 57 insertions(+), 60 deletions(-) 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 da6292534fe..4baff663617 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 @@ -115,8 +115,6 @@ static class StaticState { } } - //TODO go through all of the state transitions and make sure we handle changingBlobs - //TODO make sure to add in transition helpers that clean changingBlobs && pendingChangeingBlobs for not the current topology static class DynamicState { public final MachineState state; public final LocalAssignment newAssignment; @@ -126,9 +124,9 @@ static class DynamicState { public final Future pendingDownload; public final Set profileActions; public final Set pendingStopProfileActions; - public final Set changingBlobs; + public final Set changingBlobs; public final LocalAssignment pendingChangingBlobsAssignment; - public final Set> pendingChangeingBlobs; + public final Set> pendingChangingBlobs; /** * The last time that WAITING_FOR_WORKER_START, KILL, or KILL_AND_RELAUNCH were entered into. @@ -156,7 +154,7 @@ public DynamicState(final LocalAssignment currentAssignment, Container container this.pendingStopProfileActions = Collections.emptySet(); this.changingBlobs = Collections.emptySet(); this.pendingChangingBlobsAssignment = null; - this.pendingChangeingBlobs = Collections.emptySet(); + this.pendingChangingBlobs = Collections.emptySet(); } public DynamicState(final MachineState state, final LocalAssignment newAssignment, @@ -164,7 +162,7 @@ public DynamicState(final MachineState state, final LocalAssignment newAssignmen final LocalAssignment pendingLocalization, final long startTime, final Future pendingDownload, final Set profileActions, final Set pendingStopProfileActions, - final Set changingBlobs, + final Set changingBlobs, final Set> pendingChangingBlobs, final LocalAssignment pendingChaningBlobsAssignment) { assert pendingChangingBlobs != null; assert !(pendingChangingBlobs.isEmpty() ^ (pendingChaningBlobsAssignment == null)); @@ -178,7 +176,7 @@ public DynamicState(final MachineState state, final LocalAssignment newAssignmen this.profileActions = profileActions; this.pendingStopProfileActions = pendingStopProfileActions; this.changingBlobs = changingBlobs; - this.pendingChangeingBlobs = pendingChangingBlobs; + this.pendingChangingBlobs = pendingChangingBlobs; this.pendingChangingBlobsAssignment = pendingChaningBlobsAssignment; } @@ -206,7 +204,7 @@ public DynamicState withNewAssignment(LocalAssignment newAssignment) { this.pendingLocalization, this.startTime, this.pendingDownload, this.profileActions, this.pendingStopProfileActions, this.changingBlobs, - this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); + this.pendingChangingBlobs, this.pendingChangingBlobsAssignment); } public DynamicState withPendingLocalization(LocalAssignment pendingLocalization, Future pendingDownload) { @@ -215,7 +213,7 @@ public DynamicState withPendingLocalization(LocalAssignment pendingLocalization, pendingLocalization, this.startTime, pendingDownload, this.profileActions, this.pendingStopProfileActions, this.changingBlobs, - this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); + this.pendingChangingBlobs, this.pendingChangingBlobsAssignment); } public DynamicState withPendingLocalization(Future pendingDownload) { @@ -229,7 +227,7 @@ public DynamicState withState(final MachineState state) { this.pendingLocalization, newStartTime, this.pendingDownload, this.profileActions, this.pendingStopProfileActions, this.changingBlobs, - this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); + this.pendingChangingBlobs, this.pendingChangingBlobsAssignment); } public DynamicState withCurrentAssignment(final Container container, final LocalAssignment currentAssignment) { @@ -238,7 +236,7 @@ public DynamicState withCurrentAssignment(final Container container, final Local this.pendingLocalization, this.startTime, this.pendingDownload, this.profileActions, this.pendingStopProfileActions, this.changingBlobs, - this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); + this.pendingChangingBlobs, this.pendingChangingBlobsAssignment); } public DynamicState withProfileActions(Set profileActions, Set pendingStopProfileActions) { @@ -247,10 +245,10 @@ public DynamicState withProfileActions(Set profileActions, Se this.pendingLocalization, this.startTime, this.pendingDownload, profileActions, pendingStopProfileActions, this.changingBlobs, - this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); + this.pendingChangingBlobs, this.pendingChangingBlobsAssignment); } - public DynamicState withChangingBlobs(Set changingBlobs) { + public DynamicState withChangingBlobs(Set changingBlobs) { if (changingBlobs == this.changingBlobs) { return this; } @@ -259,18 +257,18 @@ public DynamicState withChangingBlobs(Set changingBlobs) { this.pendingLocalization, this.startTime, this.pendingDownload, profileActions, this.pendingStopProfileActions, changingBlobs, - this.pendingChangeingBlobs, this.pendingChangingBlobsAssignment); + this.pendingChangingBlobs, this.pendingChangingBlobsAssignment); } - public DynamicState withPendingChangeingBlobs(Set> pendingChangeingBlobs, - LocalAssignment pendingChangeingBlobsAssignment) { + public DynamicState withPendingChangingBlobs(Set> pendingChangingBlobs, + LocalAssignment pendingChangingBlobsAssignment) { return new DynamicState(this.state, this.newAssignment, this.container, this.currentAssignment, this.pendingLocalization, this.startTime, this.pendingDownload, profileActions, this.pendingStopProfileActions, this.changingBlobs, - pendingChangeingBlobs, - pendingChangeingBlobsAssignment); + pendingChangingBlobs, + pendingChangingBlobsAssignment); } }; @@ -318,12 +316,12 @@ public String toString() { /** * Holds the information about a blob that is changing. */ - static class BlobChangeing { + static class BlobChanging { private final LocalAssignment assignment; private final LocallyCachedBlob blob; private final GoodToGo.GoodToGoLatch latch; - public BlobChangeing(LocalAssignment assignment, LocallyCachedBlob blob, GoodToGo.GoodToGoLatch latch) { + public BlobChanging(LocalAssignment assignment, LocallyCachedBlob blob, GoodToGo.GoodToGoLatch latch) { this.assignment = assignment; this.blob = blob; this.latch = latch; @@ -501,20 +499,20 @@ static DynamicState cleanupCurrentContainer(DynamicState dynamicState, StaticSta } /** - * Drop all of the changingBlobs and pendingChangeingBlobs. + * Drop all of the changingBlobs and pendingChangingBlobs. * @param dynamicState current state. * @return the next state. */ private static DynamicState drainAllChangingBlobs(DynamicState dynamicState) { if (!dynamicState.changingBlobs.isEmpty()) { - for (BlobChangeing rc : dynamicState.changingBlobs) { + for (BlobChanging rc : dynamicState.changingBlobs) { rc.latch.countDown(); } dynamicState = dynamicState.withChangingBlobs(Collections.emptySet()); } - if (!dynamicState.pendingChangeingBlobs.isEmpty()) { - dynamicState = dynamicState.withPendingChangeingBlobs(Collections.emptySet(), null); + if (!dynamicState.pendingChangingBlobs.isEmpty()) { + dynamicState = dynamicState.withPendingChangingBlobs(Collections.emptySet(), null); } return dynamicState; @@ -535,17 +533,17 @@ private static DynamicState informChangedBlobs(DynamicState dynamicState, LocalA Set> futures = new HashSet<>(dynamicState.changingBlobs.size()); if (forSameTopology(dynamicState.pendingChangingBlobsAssignment, assignment)) { //We need to add the new futures to the existing ones - futures.addAll(dynamicState.pendingChangeingBlobs); + futures.addAll(dynamicState.pendingChangingBlobs); } //Otherwise they will just be replaced - for (BlobChangeing rc: dynamicState.changingBlobs) { + for (BlobChanging rc: dynamicState.changingBlobs) { futures.add(rc.latch.countDown()); } - LOG.debug("found changeing blobs {} moving them to pending...", dynamicState.changingBlobs); + LOG.debug("found changing blobs {} moving them to pending...", dynamicState.changingBlobs); return dynamicState.withChangingBlobs(Collections.emptySet()) - .withPendingChangeingBlobs(futures, assignment); + .withPendingChangingBlobs(futures, assignment); } /** @@ -561,8 +559,8 @@ private static DynamicState filterChangingBlobsFor(DynamicState dynamicState, fi return dynamicState; } - HashSet savedBlobs = new HashSet<>(dynamicState.changingBlobs.size()); - for (BlobChangeing rc: dynamicState.changingBlobs) { + HashSet savedBlobs = new HashSet<>(dynamicState.changingBlobs.size()); + for (BlobChanging rc: dynamicState.changingBlobs) { if (forSameTopology(assignment, rc.assignment)) { savedBlobs.add(rc); } else { @@ -602,11 +600,11 @@ static DynamicState handleWaitingForBlobLocalization(DynamicState dynamicState, if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) { //Scheduling changed staticState.localizer.releaseSlotFor(dynamicState.pendingLocalization, staticState.port); - return prepareForNewAssignmentNoWorkersRunning(dynamicState.withPendingChangeingBlobs(Collections.emptySet(), null), + return prepareForNewAssignmentNoWorkersRunning(dynamicState.withPendingChangingBlobs(Collections.emptySet(), null), staticState); } - if (!dynamicState.pendingChangeingBlobs.isEmpty()) { + if (!dynamicState.pendingChangingBlobs.isEmpty()) { LOG.info("There are pending changes, waiting for them to finish before launching container..."); //We cannot launch the container yet the resources may still be updating return dynamicState.withState(MachineState.WAITING_FOR_BLOB_UPDATE) @@ -644,7 +642,7 @@ static DynamicState handleWaitingForBlobLocalization(DynamicState dynamicState, * State Transitions for WAITING_FOR_BLOB_UPDATE state. * * PRECONDITION: container is null - * PRECONDITION: pendingChangeingBlobs is not empty (otherwise why did we go to this state) + * PRECONDITION: pendingChangingBlobs is not empty (otherwise why did we go to this state) * PRECONDITION: pendingChangingBlobsAssignment is not null. * * @param dynamicState current state @@ -656,7 +654,7 @@ private static DynamicState handleWaitingForBlobUpdate(DynamicState dynamicState throws Exception { assert dynamicState.container == null; assert dynamicState.pendingChangingBlobsAssignment != null; - assert !dynamicState.pendingChangeingBlobs.isEmpty(); + assert !dynamicState.pendingChangingBlobs.isEmpty(); if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { //We were rescheduled while waiting for the resources to be updated, @@ -668,7 +666,7 @@ private static DynamicState handleWaitingForBlobUpdate(DynamicState dynamicState } staticState.localizer.releaseSlotFor(dynamicState.pendingChangingBlobsAssignment, staticState.port); return prepareForNewAssignmentNoWorkersRunning(dynamicState.withCurrentAssignment(null, null) - .withPendingChangeingBlobs(Collections.emptySet(), null), + .withPendingChangingBlobs(Collections.emptySet(), null), staticState); } @@ -680,7 +678,7 @@ private static DynamicState handleWaitingForBlobUpdate(DynamicState dynamicState //We only have a set amount of time we can wait for before looping around again long start = Time.nanoTime(); try { - for (Future pending: dynamicState.pendingChangeingBlobs) { + for (Future pending: dynamicState.pendingChangingBlobs) { long now = Time.nanoTime(); long timeLeft = ONE_SEC_IN_NANO - (now - start); if (timeLeft <= 0) { @@ -693,7 +691,7 @@ private static DynamicState handleWaitingForBlobUpdate(DynamicState dynamicState staticState.localState); return dynamicState .withCurrentAssignment(c, dynamicState.pendingChangingBlobsAssignment).withState(MachineState.WAITING_FOR_WORKER_START) - .withPendingChangeingBlobs(Collections.emptySet(), null); + .withPendingChangingBlobs(Collections.emptySet(), null); } catch (TimeoutException ex) { return dynamicState; } @@ -964,7 +962,7 @@ static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticSta private final AtomicReference newAssignment = new AtomicReference<>(); private final AtomicReference> profiling = new AtomicReference<>(new HashSet<>()); - private final BlockingQueue changingBlobs = new LinkedBlockingQueue<>(); + private final BlockingQueue changingBlobs = new LinkedBlockingQueue<>(); private final StaticState staticState; private final IStormClusterState clusterState; private volatile boolean done = false; @@ -1040,7 +1038,7 @@ public void blobChanging(LocalAssignment assignment, int port, LocallyCachedBlob assert port == staticState.port : "got a callaback that is not for us " + port + " != " + staticState.port; //This is called async so lets assume that it is something we care about try { - changingBlobs.put(new BlobChangeing(assignment, blob, go.getLatch())); + changingBlobs.put(new BlobChanging(assignment, blob, go.getLatch())); } catch (InterruptedException e) { throw new RuntimeException("This should not have happend, but it did (the queue is unbounded)", e); } @@ -1101,15 +1099,15 @@ public void run() { Set origProfileActions = new HashSet<>(profiling.get()); Set removed = new HashSet<>(origProfileActions); - Set changingResourcesToHandle = dynamicState.changingBlobs; + Set changingResourcesToHandle = dynamicState.changingBlobs; if (!changingBlobs.isEmpty()) { changingResourcesToHandle = new HashSet<>(changingResourcesToHandle); changingBlobs.drainTo(changingResourcesToHandle); - Iterator it = changingResourcesToHandle.iterator(); + Iterator it = changingResourcesToHandle.iterator(); //Remove/Clean up changed requests that are not for us while(it.hasNext()) { - BlobChangeing rc = it.next(); + BlobChanging rc = it.next(); if (!forSameTopology(rc.assignment, dynamicState.currentAssignment) && !forSameTopology(rc.assignment, dynamicState.newAssignment)) { rc.latch.countDown(); //Ignore the future diff --git a/storm-server/src/main/java/org/apache/storm/localizer/GoodToGo.java b/storm-server/src/main/java/org/apache/storm/localizer/GoodToGo.java index 04c7a06568f..2fffe215d7a 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/GoodToGo.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/GoodToGo.java @@ -29,12 +29,12 @@ public class GoodToGo { public static class GoodToGoLatch { private final CountDownLatch latch; - private final Future doneChangeing; + private final Future doneChanging; private boolean wasCounted = false; - public GoodToGoLatch(CountDownLatch latch, Future doneChangeing) { + public GoodToGoLatch(CountDownLatch latch, Future doneChanging) { this.latch = latch; - this.doneChangeing = doneChangeing; + this.doneChanging = doneChanging; } public synchronized Future countDown() { @@ -42,15 +42,15 @@ public synchronized Future countDown() { latch.countDown(); wasCounted = true; } - return doneChangeing; + return doneChanging; } } private final GoodToGoLatch latch; private boolean gotLatch = false; - public GoodToGo(CountDownLatch latch, Future doneChangeing) { - this.latch = new GoodToGoLatch(latch, doneChangeing); + public GoodToGo(CountDownLatch latch, Future doneChanging) { + this.latch = new GoodToGoLatch(latch, doneChanging); } /** diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java index c09108da1cf..a287e959d3c 100644 --- a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java +++ b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java @@ -156,7 +156,7 @@ protected long getSizeOnDisk(Path p) throws IOException { } /** - * Mark that a given port and assignemnt are using this. + * Mark that a given port and assignment are using this. * @param pna the slot and assignment that are using this blob. * @param cb an optional callback indicating that they want to know/synchronize when a blob is updated. */ @@ -191,7 +191,6 @@ public synchronized void informAllOfChangeAndWaitForConsensus() { try { PortAndAssignment pna = entry.getKey(); BlobChangingCallback cb = entry.getValue(); - //TODO we probably want to not use this, or make it just return something that has less power to modify things cb.blobChanging(pna.getAssignment(), pna.getPort(), this, gtg); } finally { gtg.countDownIfLatchWasNotGotten(); @@ -200,7 +199,8 @@ public synchronized void informAllOfChangeAndWaitForConsensus() { try { cdl.await(3, TimeUnit.MINUTES); } catch (InterruptedException e) { - //TODO need to think about error handling here in general. + //Interrupted is thrown when we are shutting down. + // So just ignore it for now... } } diff --git a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java index 927225d4c26..4d5691b6cc8 100644 --- a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java +++ b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import org.apache.storm.daemon.supervisor.Slot.StaticState; @@ -532,12 +531,12 @@ public void testResourcesChanged() throws Exception { StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000, containerLauncher, "localhost", port, iSuper, state, cb); - Set changing = new HashSet<>(); + Set changing = new HashSet<>(); LocallyCachedBlob stormJar = mock(LocallyCachedBlob.class); GoodToGo.GoodToGoLatch stormJarLatch = mock(GoodToGo.GoodToGoLatch.class); CompletableFuture stormJarLatchFuture = mock(CompletableFuture.class); when(stormJarLatch.countDown()).thenReturn(stormJarLatchFuture); - changing.add(new Slot.BlobChangeing(cAssignment, stormJar, stormJarLatch)); + changing.add(new Slot.BlobChanging(cAssignment, stormJar, stormJarLatch)); DynamicState dynamicState = new DynamicState(cAssignment, cContainer, cAssignment).withChangingBlobs(changing); @@ -549,7 +548,7 @@ public void testResourcesChanged() throws Exception { assertNull(nextState.pendingDownload); assertNull(nextState.pendingLocalization); assertEquals(changing, nextState.changingBlobs); - assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertTrue(nextState.pendingChangingBlobs.isEmpty()); assertNull(nextState.pendingChangingBlobsAssignment); assertTrue(Time.currentTimeMillis() > 1000); @@ -559,7 +558,7 @@ public void testResourcesChanged() throws Exception { assertNull(nextState.pendingDownload); assertNull(nextState.pendingLocalization); assertEquals(changing, nextState.changingBlobs); - assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertTrue(nextState.pendingChangingBlobs.isEmpty()); assertNull(nextState.pendingChangingBlobsAssignment); assertTrue(Time.currentTimeMillis() > 2000); @@ -573,7 +572,7 @@ public void testResourcesChanged() throws Exception { verify(containerLauncher).launchContainer(port, cAssignment, state); assertEquals(MachineState.WAITING_FOR_WORKER_START, nextState.state); assertNull(nextState.pendingChangingBlobsAssignment); - assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertTrue(nextState.pendingChangingBlobs.isEmpty()); assertSame(cAssignment, nextState.currentAssignment); assertSame(nContainer, nextState.container); assertTrue(Time.currentTimeMillis() > 2000); @@ -581,7 +580,7 @@ public void testResourcesChanged() throws Exception { nextState = Slot.stateMachineStep(nextState, staticState); assertEquals(MachineState.RUNNING, nextState.state); assertNull(nextState.pendingChangingBlobsAssignment); - assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertTrue(nextState.pendingChangingBlobs.isEmpty()); assertSame(cAssignment, nextState.currentAssignment); assertSame(nContainer, nextState.container); assertTrue(Time.currentTimeMillis() > 2000); @@ -589,7 +588,7 @@ public void testResourcesChanged() throws Exception { nextState = Slot.stateMachineStep(nextState, staticState); assertEquals(MachineState.RUNNING, nextState.state); assertNull(nextState.pendingChangingBlobsAssignment); - assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertTrue(nextState.pendingChangingBlobs.isEmpty()); assertSame(cAssignment, nextState.currentAssignment); assertSame(nContainer, nextState.container); assertTrue(Time.currentTimeMillis() > 3000); @@ -597,7 +596,7 @@ public void testResourcesChanged() throws Exception { nextState = Slot.stateMachineStep(nextState, staticState); assertEquals(MachineState.RUNNING, nextState.state); assertNull(nextState.pendingChangingBlobsAssignment); - assertTrue(nextState.pendingChangeingBlobs.isEmpty()); + assertTrue(nextState.pendingChangingBlobs.isEmpty()); assertSame(cAssignment, nextState.currentAssignment); assertSame(nContainer, nextState.container); assertTrue(Time.currentTimeMillis() > 4000);