From 0dac58b0aa82133df242b3b2ebeb65bfea7d63cc Mon Sep 17 00:00:00 2001 From: chenyuzhao Date: Wed, 28 Mar 2018 23:22:03 +0800 Subject: [PATCH] [STORM-2693] Heartbeats and assignments promotion for storm2.0 --- conf/defaults.yaml | 16 + .../src/jvm/org/apache/storm/Config.java | 72 +- .../src/jvm/org/apache/storm/StormTimer.java | 24 +- .../src/jvm/org/apache/storm/Thrift.java | 42 +- .../assignments/ILocalAssignmentsBackend.java | 116 + .../InMemoryAssignmentBackend.java | 158 + .../LocalAssignmentsBackendFactory.java | 48 + .../apache/storm/cluster/ClusterUtils.java | 19 +- .../org/apache/storm/cluster/DaemonType.java | 2 + .../storm/cluster/IStormClusterState.java | 78 +- .../storm/cluster/StormClusterStateImpl.java | 124 +- .../apache/storm/cluster/ZKStateStorage.java | 15 +- .../storm/cluster/ZKStateStorageFactory.java | 7 +- .../apache/storm/daemon/worker/Worker.java | 57 +- .../storm/daemon/worker/WorkerState.java | 49 +- .../org/apache/storm/generated/HBNodes.java | 32 +- .../org/apache/storm/generated/HBRecords.java | 36 +- .../org/apache/storm/generated/Nimbus.java | 5135 +++++++++++++---- .../apache/storm/generated/Supervisor.java | 2968 ++++++++++ .../generated/SupervisorAssignments.java | 478 ++ .../storm/generated/SupervisorInfo.java | 177 +- .../generated/SupervisorWorkerHeartbeat.java | 660 +++ .../generated/SupervisorWorkerHeartbeats.java | 561 ++ .../storm/generated/WorkerMetricList.java | 36 +- .../generated/WorkerTokenServiceType.java | 5 +- .../grouping/LoadAwareShuffleGrouping.java | 8 +- .../storm/security/auth/IAuthorizer.java | 6 +- .../security/auth/ThriftConnectionType.java | 5 +- .../auth/authorizer/SimpleACLAuthorizer.java | 7 +- .../SupervisorSimpleACLAuthorizer.java | 157 + .../auth/kerberos/ServerCallbackHandler.java | 3 +- .../jvm/org/apache/storm/stats/StatsUtil.java | 141 +- .../storm/topology/TopologyBuilder.java | 21 +- .../apache/storm/utils/SupervisorClient.java | 88 + storm-client/src/py/storm/Nimbus-remote | 21 + storm-client/src/py/storm/Nimbus.py | 633 +- storm-client/src/py/storm/Supervisor-remote | 126 + storm-client/src/py/storm/Supervisor.py | 694 +++ storm-client/src/py/storm/__init__.py | 2 +- storm-client/src/py/storm/ttypes.py | 369 +- storm-client/src/storm.thrift | 46 +- .../LocalAssignmentsBackendTest.java | 102 + .../cluster/StormClusterStateImplTest.java | 5 +- .../apache/storm/command/shell_submission.clj | 2 +- .../test/clj/org/apache/storm/nimbus_test.clj | 57 +- .../storm/security/auth/nimbus_auth_test.clj | 4 + .../org/apache/storm/stats/TestStatsUtil.java | 10 +- .../java/org/apache/storm/DaemonConfig.java | 26 + .../java/org/apache/storm/LocalCluster.java | 19 + .../apache/storm/daemon/nimbus/Nimbus.java | 527 +- .../daemon/supervisor/BasicContainer.java | 40 +- .../supervisor/BasicContainerLauncher.java | 13 +- .../storm/daemon/supervisor/Container.java | 5 +- .../daemon/supervisor/ContainerLauncher.java | 10 +- .../daemon/supervisor/LocalContainer.java | 7 +- .../supervisor/LocalContainerLauncher.java | 7 +- .../daemon/supervisor/ReadClusterState.java | 56 +- .../daemon/supervisor/RunAsUserContainer.java | 19 +- .../RunAsUserContainerLauncher.java | 15 +- .../storm/daemon/supervisor/Supervisor.java | 224 +- .../daemon/supervisor/SupervisorUtils.java | 17 +- .../timer/ReportWorkerHeartbeats.java | 112 + .../supervisor/timer/SupervisorHeartbeat.java | 17 +- .../timer/SynchronizeAssignments.java | 133 + .../nimbus/AssignmentDistributionService.java | 312 + .../apache/storm/nimbus/ILeaderElector.java | 6 +- .../IWorkerHeartbeatsRecoveryStrategy.java | 48 + .../storm/nimbus/LeaderListenerCallback.java | 231 + .../nimbus/NimbusHeartbeatsPressureTest.java | 241 + ...meOutWorkerHeartbeatsRecoveryStrategy.java | 80 + ...rkerHeartbeatsRecoveryStrategyFactory.java | 53 + .../storm/scheduler/SupervisorDetails.java | 27 +- .../storm/zookeeper/LeaderElectorImp.java | 12 +- .../org/apache/storm/zookeeper/Zookeeper.java | 145 +- .../daemon/supervisor/BasicContainerTest.java | 244 +- .../daemon/supervisor/ContainerTest.java | 16 +- 76 files changed, 14135 insertions(+), 1949 deletions(-) create mode 100644 storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java create mode 100644 storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java create mode 100644 storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java create mode 100644 storm-client/src/jvm/org/apache/storm/generated/Supervisor.java create mode 100644 storm-client/src/jvm/org/apache/storm/generated/SupervisorAssignments.java create mode 100644 storm-client/src/jvm/org/apache/storm/generated/SupervisorWorkerHeartbeat.java create mode 100644 storm-client/src/jvm/org/apache/storm/generated/SupervisorWorkerHeartbeats.java create mode 100644 storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SupervisorSimpleACLAuthorizer.java create mode 100644 storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java create mode 100644 storm-client/src/py/storm/Supervisor-remote create mode 100644 storm-client/src/py/storm/Supervisor.py create mode 100644 storm-client/test/jvm/org/apache/storm/assignments/LocalAssignmentsBackendTest.java create mode 100644 storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/ReportWorkerHeartbeats.java create mode 100644 storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SynchronizeAssignments.java create mode 100644 storm-server/src/main/java/org/apache/storm/nimbus/AssignmentDistributionService.java create mode 100644 storm-server/src/main/java/org/apache/storm/nimbus/IWorkerHeartbeatsRecoveryStrategy.java create mode 100644 storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java create mode 100644 storm-server/src/main/java/org/apache/storm/nimbus/NimbusHeartbeatsPressureTest.java create mode 100644 storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java create mode 100644 storm-server/src/main/java/org/apache/storm/nimbus/WorkerHeartbeatsRecoveryStrategyFactory.java diff --git a/conf/defaults.yaml b/conf/defaults.yaml index d64623ee69a..2ed45991c2a 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -77,6 +77,10 @@ topology.max.replication.wait.time.sec: 60 nimbus.credential.renewers.freq.secs: 600 nimbus.queue.size: 100000 scheduler.display.resource: false +nimbus.local.assignments.backend.class: "org.apache.storm.assignments.InMemoryAssignmentBackend" +nimbus.assignments.service.threads: 10 +nimbus.assignments.service.thread.queue.size: 100 +nimbus.worker.heartbeats.recovery.strategy.class: "org.apache.storm.nimbus.TimeOutWorkerHeartbeatsRecoveryStrategy" ### ui.* configs are for the master ui.host: 0.0.0.0 @@ -154,6 +158,8 @@ supervisor.worker.shutdown.sleep.secs: 3 supervisor.monitor.frequency.secs: 3 #how frequently the supervisor heartbeats to the cluster state (for nimbus) supervisor.heartbeat.frequency.secs: 5 +#max timeout for a node worker heartbeats when master gains leadership +supervisor.worker.heartbeats.max.timeout.secs: 600 supervisor.enable: true supervisor.supervisors: [] supervisor.supervisors.commands: [] @@ -163,6 +169,13 @@ supervisor.memory.capacity.mb: 4096.0 # for single threaded bolts supervisor.cpu.capacity: 400.0 +#Supervisor thrift config +supervisor.thrift.port: 6628 +supervisor.queue.size: 128 +supervisor.thrift.threads: 16 +supervisor.thrift.max_buffer_size: 1048576 +supervisor.thrift.socket.timeout.ms: 5000 + ### worker.* configs are for task workers worker.heap.memory.mb: 768 worker.childopts: "-Xmx%HEAP-MEM%m -XX:+PrintGCDetails -Xloggc:artifacts/gc.log -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=1M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=artifacts/heapdump" @@ -182,6 +195,9 @@ worker.log.level.reset.poll.secs: 30 # control how many worker receiver threads we need per worker topology.worker.receiver.thread.count: 1 +# Executor metrics reporting interval. +executor.metrics.frequency.secs: 60 + task.heartbeat.frequency.secs: 3 task.refresh.poll.secs: 10 task.credentials.poll.secs: 30 diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java index ed6068c4b6c..b2759854f52 100644 --- a/storm-client/src/jvm/org/apache/storm/Config.java +++ b/storm-client/src/jvm/org/apache/storm/Config.java @@ -1101,6 +1101,13 @@ public class Config extends HashMap { @isPositiveNumber public static final String NIMBUS_QUEUE_SIZE = "nimbus.queue.size"; + /** + * Nimbus assignments backend for storing local assignments. We will use it to store physical plan and runtime storm ids. + */ + @isString + @isImplementationOfClass(implementsClass = org.apache.storm.assignments.ILocalAssignmentsBackend.class) + public static final String NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS = "nimbus.local.assignments.backend.class"; + /** * The number of threads that should be used by the nimbus thrift server. */ @@ -1411,6 +1418,44 @@ public class Config extends HashMap { @isPositiveNumber public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity"; + @isInteger + @isPositiveNumber + /** + * Port used for supervisor thrift server. + */ + public static final String SUPERVISOR_THRIFT_PORT = "supervisor.thrift.port"; + + @isString + /** + * The Supervisor invocations transport plug-in for Thrift client/server communication. + */ + public static final String SUPERVISOR_THRIFT_TRANSPORT_PLUGIN = "supervisor.thrift.transport"; + + @isInteger + @isPositiveNumber + /** + * Supervisor thrift server queue size. + */ + public static final String SUPERVISOR_QUEUE_SIZE = "supervisor.queue.size"; + + @isInteger + @isPositiveNumber + /** + * The number of threads that should be used by the supervisor thrift server. + */ + public static final String SUPERVISOR_THRIFT_THREADS = "supervisor.thrift.threads"; + + @isNumber + @isPositiveNumber + public static final String SUPERVISOR_THRIFT_MAX_BUFFER_SIZE = "supervisor.thrift.max_buffer_size"; + + /** + * How long before a supervisor Thrift Client socket hangs before timeout + * and restart the socket. + */ + @isInteger + public static final String SUPERVISOR_THRIFT_SOCKET_TIMEOUT_MS = "supervisor.thrift.socket.timeout.ms"; + /** * A map of resources the Supervisor has e.g {"cpu.pcore.percent" : 200.0. "onheap.memory.mb": 256.0, "gpu.count" : 2.0 } */ @@ -1523,6 +1568,12 @@ public class Config extends HashMap { @isBoolean public static final String SUPERVISOR_RUN_WORKER_AS_USER = "supervisor.run.worker.as.user"; + /** + * max timeout for supervisor reported heartbeats when master gains leadership + */ + @isInteger + public static final String SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS = "supervisor.worker.heartbeats.max.timeout.secs"; + /** * On some systems (windows for example) symlinks require special privileges that not everyone wants to * grant a headless user. You can completely disable the use of symlinks by setting this config to true, but @@ -1572,11 +1623,19 @@ public class Config extends HashMap { /** * A list of users that run the supervisors and should be authorized to interact with * nimbus as a supervisor would. To use this set - * nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer + * nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer. */ @isStringList public static final String NIMBUS_SUPERVISOR_USERS = "nimbus.supervisor.users"; + /** + * A list of users that nimbus runs as and should be authorized to interact with + * the supervisor as nimbus would. To use this set supervisor.authorizer to + * org.apache.storm.security.auth.authorizer.SupervisorSimpleACLAuthorizer. + */ + @isStringList + public static final String NIMBUS_DAEMON_USERS = "nimbus.daemon.users"; + /** * A list of users that are cluster admins and can run any command. To use this set * nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer @@ -1781,10 +1840,19 @@ public class Config extends HashMap { public static final String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs"; /** - * How often a task should heartbeat its status to the master. + * How often executor metrics should report to master, used for RPC heartbeat mode. */ @isInteger @isPositiveNumber + public static final String EXECUTOR_METRICS_FREQUENCY_SECS = "executor.metrics.frequency.secs"; + + /** + * How often a task should heartbeat its status to the master, + * deprecated for 2.0 RPC heartbeat reporting, see {@code EXECUTOR_METRICS_FREQUENCY_SECS }. + */ + @Deprecated + @isInteger + @isPositiveNumber public static final String TASK_HEARTBEAT_FREQUENCY_SECS = "task.heartbeat.frequency.secs"; /** diff --git a/storm-client/src/jvm/org/apache/storm/StormTimer.java b/storm-client/src/jvm/org/apache/storm/StormTimer.java index 0f54ce187f4..72b20d38879 100644 --- a/storm-client/src/jvm/org/apache/storm/StormTimer.java +++ b/storm-client/src/jvm/org/apache/storm/StormTimer.java @@ -18,19 +18,19 @@ package org.apache.storm; -import org.apache.storm.utils.Time; -import org.apache.storm.utils.Utils; - import java.nio.channels.ClosedByInterruptException; import java.util.Comparator; import java.util.Random; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.storm.utils.Time; +import org.apache.storm.utils.Utils; + /** * The timer defined in this file is very similar to java.util.Timer, except * it integrates with Storm's time simulation capabilities. This lets us test - * code that does asynchronous work on the timer thread + * code that does asynchronous work on the timer thread. */ public class StormTimer implements AutoCloseable { @@ -131,7 +131,7 @@ public void add(QueueEntry queueEntry) { private StormTimerTask task = new StormTimerTask(); /** - * Makes a Timer in the form of a StormTimerTask Object + * Makes a Timer in the form of a StormTimerTask Object. * @param name name of the timer * @param onKill function to call when timer is killed unexpectedly * @return StormTimerTask object that was initialized @@ -154,7 +154,7 @@ public StormTimer (String name, Thread.UncaughtExceptionHandler onKill) { } /** - * Schedule a function to be executed in the timer + * Schedule a function to be executed in the timer. * @param delaySecs the number of seconds to delay before running the function * @param func the function to run * @param checkActive whether to check is the timer is active @@ -195,7 +195,7 @@ public void scheduleMs(long delayMs, Runnable func) { } /** - * Schedule a function to run recurrently + * Schedule a function to run recurrently. * @param delaySecs the number of seconds to delay before running the function * @param recurSecs the time between each invocation * @param func the function to run @@ -212,7 +212,7 @@ public void run() { } /** - * Schedule a function to run recurrently + * Schedule a function to run recurrently. * @param delayMs the number of millis to delay before running the function * @param recurMs the time between each invocation * @param func the function to run @@ -230,7 +230,7 @@ public void run() { /** - * schedule a function to run recurrently with jitter + * Schedule a function to run recurrently with jitter. * @param delaySecs the number of seconds to delay before running the function * @param recurSecs the time between each invocation * @param jitterMs jitter added to the run @@ -248,7 +248,7 @@ public void run() { } /** - * check if timer is active + * check if timer is active. */ private void checkActive() { if (!this.task.isActive()) { @@ -257,7 +257,7 @@ private void checkActive() { } /** - * cancel timer + * cancel timer. */ @Override @@ -270,7 +270,7 @@ public void close() throws InterruptedException { } /** - * is timer waiting. Used in timer simulation + * is timer waiting. Used in timer simulation. */ public boolean isTimerWaiting() { return Time.isThreadWaiting(task); diff --git a/storm-client/src/jvm/org/apache/storm/Thrift.java b/storm-client/src/jvm/org/apache/storm/Thrift.java index e00b0ee6be9..85f8a85a083 100644 --- a/storm-client/src/jvm/org/apache/storm/Thrift.java +++ b/storm-client/src/jvm/org/apache/storm/Thrift.java @@ -15,46 +15,44 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm; -import org.apache.storm.generated.Bolt; -import org.apache.storm.generated.JavaObjectArg; -import org.apache.storm.generated.SpoutSpec; -import org.apache.storm.generated.StateSpoutSpec; -import org.apache.storm.generated.StreamInfo; +package org.apache.storm; +import java.io.Serializable; import java.lang.reflect.Constructor; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.List; import java.util.HashMap; -import java.io.Serializable; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import org.apache.storm.generated.JavaObject; +import org.apache.storm.generated.Bolt; +import org.apache.storm.generated.ComponentCommon; +import org.apache.storm.generated.ComponentObject; +import org.apache.storm.generated.GlobalStreamId; import org.apache.storm.generated.Grouping; +import org.apache.storm.generated.JavaObject; +import org.apache.storm.generated.JavaObjectArg; +import org.apache.storm.generated.NullStruct; +import org.apache.storm.generated.SpoutSpec; +import org.apache.storm.generated.StateSpoutSpec; import org.apache.storm.generated.StormTopology; import org.apache.storm.generated.StormTopology._Fields; -import org.apache.storm.generated.ComponentCommon; -import org.apache.storm.generated.NullStruct; -import org.apache.storm.generated.GlobalStreamId; -import org.apache.storm.generated.ComponentObject; - +import org.apache.storm.generated.StreamInfo; import org.apache.storm.task.IBolt; import org.apache.storm.topology.BoltDeclarer; -import org.apache.storm.topology.IRichBolt; import org.apache.storm.topology.IBasicBolt; +import org.apache.storm.topology.IRichBolt; import org.apache.storm.topology.IRichSpout; import org.apache.storm.topology.SpoutDeclarer; +import org.apache.storm.topology.TopologyBuilder; import org.apache.storm.utils.Utils; import org.json.simple.JSONValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.storm.topology.TopologyBuilder; - public class Thrift { private static Logger LOG = LoggerFactory.getLogger(Thrift.class); @@ -328,19 +326,19 @@ private static void addInputs(BoltDeclarer declarer, Map spoutMap, Map boltMap) { TopologyBuilder builder = new TopologyBuilder(); for (Entry entry : spoutMap.entrySet()) { - String spoutID = entry.getKey(); + String spoutId = entry.getKey(); SpoutDetails spec = entry.getValue(); - SpoutDeclarer spoutDeclarer = builder.setSpout(spoutID, spec.getSpout(), spec.getParallelism()); + SpoutDeclarer spoutDeclarer = builder.setSpout(spoutId, spec.getSpout(), spec.getParallelism()); spoutDeclarer.addConfigurations(spec.getConf()); } for (Entry entry : boltMap.entrySet()) { - String spoutID = entry.getKey(); + String spoutId = entry.getKey(); BoltDetails spec = entry.getValue(); BoltDeclarer boltDeclarer = null; if (spec.bolt instanceof IRichBolt) { - boltDeclarer = builder.setBolt(spoutID, (IRichBolt)spec.getBolt(), spec.getParallelism()); + boltDeclarer = builder.setBolt(spoutId, (IRichBolt)spec.getBolt(), spec.getParallelism()); } else { - boltDeclarer = builder.setBolt(spoutID, (IBasicBolt)spec.getBolt(), spec.getParallelism()); + boltDeclarer = builder.setBolt(spoutId, (IBasicBolt)spec.getBolt(), spec.getParallelism()); } boltDeclarer.addConfigurations(spec.getConf()); addInputs(boltDeclarer, spec.getInputs()); diff --git a/storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java b/storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java new file mode 100644 index 00000000000..10b0b0f7c8d --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java @@ -0,0 +1,116 @@ +/** + * 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.assignments; + +import java.util.List; +import java.util.Map; + +import org.apache.storm.generated.Assignment; + +/** + * Interface for storing local assignments. + */ +public interface ILocalAssignmentsBackend extends AutoCloseable{ + /** + * Decide if the assignments is synchronized from remote state-store. + */ + boolean isSynchronized(); + + /** + * Mark this backend as synchronized when sync work is done. + */ + void setSynchronized(); + + /** + * Initial function for creating backend. + * @param conf config + */ + void prepare(Map conf); + + /** + * Keep a storm assignment to local state or update old assignment. + * @param stormId storm runtime id + * @param assignment assignment as thrift + */ + void keepOrUpdateAssignment(String stormId, Assignment assignment); + + /** + * Get assignment as {@link Assignment} for a storm. + * @param stormId storm runtime id + * @return assignment + */ + Assignment getAssignment(String stormId); + + void removeAssignment(String stormId); + + /** + * List all the storm runtime ids of local assignments. + * @return a list of storm ids + */ + List assignments(); + + /** + * Get all the local assignments of local state. + * @return mapping of storm-id -> assignment + */ + Map assignmentsInfo(); + + /** + * Sync remote assignments to local, if remote is null, we will sync it from zk. + * @param remote specific remote assignments, if it is null, it will sync from zookeeper[only used for nimbus] + */ + void syncRemoteAssignments(Map remote); + + /** + * Keep a mapping storm-name -> storm-id to local state. + * @param stormName storm name + * @param stormId storm runtime id + */ + void keepStormId(String stormName, String stormId); + + /** + * Get storm runtime id from local. + * @param stormName name of a storm + * @return runtime storm id + */ + String getStormId(String stormName); + + /** + * Sync remote storm ids to local, will just used for nimbus. + * @param remote remote ids from state store + */ + void syncRemoteIds(Map remote); + + /** + * Delete a local cache of stormId which is mapped to a specific storm name. + * @param stormName storm name + */ + void deleteStormId(String stormName); + + /** + * Clear all the state for a storm. + * @param stormId storm id + */ + void clearStateForStorm(String stormId); + + /** + * Function to release resource. + */ + void close(); +} diff --git a/storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java b/storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java new file mode 100644 index 00000000000..44cf6d213fc --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java @@ -0,0 +1,158 @@ +/** + * 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.assignments; + +import java.util.ArrayList; +import java.util.concurrent.ConcurrentHashMap; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.storm.cluster.ClusterUtils; +import org.apache.storm.generated.Assignment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An assignment backend which will keep all assignments and id-info in memory. Only used if no backend is specified + * internal. + *

About thread safe: idToAssignment,idToName,nameToId are all memory cache in nimbus local, for + *

    + *
  • idToAssignment: nimbus will modify it and supervisors will sync it at fixed interval, + * so the assignments would come to eventual consistency.
  • + *
  • idToName: storm submitting/killing is guarded by the same lock, a {@link ConcurrentHashMap} is ok.
  • + *
  • nameToId: same as idToName. + *
+ */ +public class InMemoryAssignmentBackend implements ILocalAssignmentsBackend { + private static final Logger LOG = LoggerFactory.getLogger(InMemoryAssignmentBackend.class); + + private Map idToAssignment; + private Map idToName; + private Map nameToId; + private volatile boolean isSynchronized = false; + + @Override + public boolean isSynchronized() { + return this.isSynchronized; + } + + @Override + public void setSynchronized() { + this.isSynchronized = true; + } + + @Override + public void prepare(Map conf) { + // do nothing for conf now + this.idToAssignment = new ConcurrentHashMap<>(); + this.idToName = new ConcurrentHashMap<>(); + this.nameToId = new ConcurrentHashMap<>(); + } + + @Override + public void keepOrUpdateAssignment(String stormId, Assignment assignment) { + this.idToAssignment.put(stormId, assignment); + } + + @Override + public Assignment getAssignment(String stormId) { + return this.idToAssignment.get(stormId); + } + + @Override + public void removeAssignment(String stormId) { + this.idToAssignment.remove(stormId); + } + + @Override + public List assignments() { + if(idToAssignment == null) { + return new ArrayList<>(); + } + List ret = new ArrayList<>(); + ret.addAll(this.idToAssignment.keySet()); + return ret; + } + + @Override + public Map assignmentsInfo() { + Map ret = new HashMap<>(); + ret.putAll(this.idToAssignment); + + return ret; + } + + @Override + public void syncRemoteAssignments(Map remote) { + Map tmp = new ConcurrentHashMap<>(); + for(Map.Entry entry: remote.entrySet()) { + tmp.put(entry.getKey(), ClusterUtils.maybeDeserialize(entry.getValue(), Assignment.class)); + } + this.idToAssignment = tmp; + } + + @Override + public void keepStormId(String stormName, String stormId) { + this.nameToId.put(stormName, stormId); + this.idToName.put(stormId, stormName); + } + + @Override + public String getStormId(String stormName) { + return this.nameToId.get(stormName); + } + + @Override + public void syncRemoteIds(Map remote) { + Map tmpNameToId = new ConcurrentHashMap<>(); + Map tmpIdToName = new ConcurrentHashMap<>(); + for(Map.Entry entry: remote.entrySet()) { + tmpIdToName.put(entry.getKey(), entry.getValue()); + tmpNameToId.put(entry.getValue(), entry.getKey()); + } + this.idToName = tmpIdToName; + this.nameToId = tmpNameToId; + } + + @Override + public void deleteStormId(String stormName) { + String id = this.nameToId.remove(stormName); + if (null != id) { + this.idToName.remove(id); + } + } + + @Override + public void clearStateForStorm(String stormId) { + this.idToAssignment.remove(stormId); + + String name = this.idToName.remove(stormId); + if (null != name) { + this.nameToId.remove(name); + } + } + + @Override + public void close() { + this.idToAssignment = null; + this.nameToId = null; + this.idToName = null; + } +} diff --git a/storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java b/storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java new file mode 100644 index 00000000000..f3110f1122c --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java @@ -0,0 +1,48 @@ +/** + * 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.assignments; + +import com.google.common.base.Preconditions; +import org.apache.storm.Config; +import org.apache.storm.utils.ConfigUtils; +import org.apache.storm.utils.ReflectionUtils; + +import java.util.Map; + +/** + * Factory class for creating local assignments. + */ +public class LocalAssignmentsBackendFactory { + + public static ILocalAssignmentsBackend getBackend(Map conf) { + if (conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS) != null) { + Object targetObj = ReflectionUtils.newInstance((String) conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS)); + Preconditions.checkState(targetObj instanceof ILocalAssignmentsBackend, "{} must implements ILocalAssignmentsBackend", Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS); + ((ILocalAssignmentsBackend)targetObj).prepare(conf); + return (ILocalAssignmentsBackend) targetObj; + } + + return getDefault(); + } + + public static ILocalAssignmentsBackend getDefault() { + ILocalAssignmentsBackend backend = new InMemoryAssignmentBackend(); + backend.prepare(ConfigUtils.readStormConfig()); + return backend; + } +} diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java index 4bd5ded5a78..b3dfc7d3b5f 100644 --- a/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java +++ b/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java @@ -18,6 +18,8 @@ package org.apache.storm.cluster; import org.apache.storm.Config; +import org.apache.storm.assignments.ILocalAssignmentsBackend; +import org.apache.storm.assignments.LocalAssignmentsBackendFactory; import org.apache.storm.generated.ClusterWorkerHeartbeat; import org.apache.storm.generated.ExecutorInfo; import org.apache.storm.generated.ExecutorStats; @@ -49,6 +51,7 @@ public class ClusterUtils { public static final String SUPERVISORS_ROOT = "supervisors"; public static final String WORKERBEATS_ROOT = "workerbeats"; public static final String BACKPRESSURE_ROOT = "backpressure"; + public static final String LEADERINFO_ROOT = "leader-info"; public static final String ERRORS_ROOT = "errors"; public static final String BLOBSTORE_ROOT = "blobstore"; public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber"; @@ -63,6 +66,7 @@ public class ClusterUtils { public static final String SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT; public static final String WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT; public static final String BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT; + public static final String LEADERINFO_SUBTREE = ZK_SEPERATOR + LEADERINFO_ROOT; public static final String ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT; public static final String BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT; public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT; @@ -252,12 +256,13 @@ public static Map convertExecutorBeats(List) stateStorage, (Map) stateStorage, context); - return new StormClusterStateImpl(Storage, context, true); + IStateStorage Storage = _instance.mkStateStorageImpl((Map) stateStorage, + (Map) stateStorage, context); + return new StormClusterStateImpl(Storage, backend, context, true); } } @@ -279,8 +284,12 @@ public static IStateStorage mkStateStorage(Map config, Map getZkSecretAcls(WorkerTokenServiceType type, Map ret = new ArrayList<>(ZooDefs.Ids.CREATOR_ALL_ACL); diff --git a/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java b/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java index ab893bec935..8a922acf8c2 100644 --- a/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java +++ b/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java @@ -42,8 +42,46 @@ public interface IStormClusterState { List assignments(Runnable callback); + /** + * Get the assignment based on storm id from local backend. + * @param stormId topology id + * @param callback callback function + * @return {@link Assignment} + */ Assignment assignmentInfo(String stormId, Runnable callback); + /** + * Get the assignment based on storm id from remote state store, eg: ZK. + * @param stormId topology id + * @param callback callback function + * @return {@link Assignment} + */ + Assignment remoteAssignmentInfo(String stormId, Runnable callback); + + /** + * Get all the topologies assignments mapping stormId -> Assignment from local backend. + * @return stormId -> Assignment mapping + */ + Map assignmentsInfo(); + + /** + * Sync the remote state store assignments to local backend, used when master gains leadership, see + * {@link LeaderListenerCallback} + * @param remote assigned assignments for a specific {@link IStormClusterState} instance, usually a supervisor/node. + */ + void syncRemoteAssignments(Map remote); + + /** + * Flag to indicate if the assignments synced successfully, see {@link #syncRemoteAssignments(Map)}. + * @return true if is synced successfully + */ + boolean isAssignmentsBackendSynchronized(); + + /** + * Mark the assignments as synced successfully, see {@link #isAssignmentsBackendSynchronized()} + */ + void setAssignmentsBackendSynchronized(); + VersionedData assignmentInfoWithVersion(String stormId, Runnable callback); Integer assignmentVersion(String stormId, Runnable callback) throws Exception; @@ -64,6 +102,19 @@ public interface IStormClusterState { */ StormBase stormBase(String stormId, Runnable callback); + /** + * Get storm id from passed name, null if the name doesn't exist on cluster. + * @param stormName storm name + * @return storm id + */ + String stormId(String stormName); + + /** + * Sync all the active storm ids of the cluster, used now when master gains leadership. + * @param ids stormName -> stormId mapping + */ + void syncRemoteIds(Map ids); + ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port); List getWorkerProfileRequests(String stormId, NodeInfo nodeInfo); @@ -94,6 +145,15 @@ public interface IStormClusterState { @Deprecated List backpressureTopologies(); + /** + * Get leader info from state store, which was written when a master gains leadership. + *

Caution: it can not be used for fencing and is only for informational purposes because we use ZK as our + * backend now, which could have a overdue info of nodes. + * @param callback callback func + * @return {@link NimbusInfo} + */ + NimbusInfo getLeader(Runnable callback); + void setTopologyLogConfig(String stormId, LogConfig logConfig); LogConfig topologyLogConfig(String stormId, Runnable cb); @@ -233,23 +293,7 @@ default Map allSupervisorInfo(Runnable callback) { * @return the id of the topology or null if it is not alive. */ default Optional getTopoId(final String topologyName) { - String ret = null; - for (String topoId: activeStorms()) { - StormBase base = stormBase(topoId, null); - if (base != null && topologyName.equals(base.get_name())) { - ret = topoId; - break; - } - } - return Optional.ofNullable(ret); - } - - default Map topologyAssignments() { - Map ret = new HashMap<>(); - for (String topoId: assignments(null)) { - ret.put(topoId, assignmentInfo(topoId, null)); - } - return ret; + return Optional.ofNullable(stormId(topologyName)); } default Map topologyBases() { diff --git a/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java index 719ebbffc7a..fc02b8e5c19 100644 --- a/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java +++ b/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java @@ -20,6 +20,8 @@ import java.nio.ByteBuffer; import java.security.NoSuchAlgorithmException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -28,12 +30,12 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicReference; + import org.apache.commons.lang.StringUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.state.ConnectionState; import org.apache.curator.framework.state.ConnectionStateListener; +import org.apache.storm.assignments.ILocalAssignmentsBackend; import org.apache.storm.callback.ZKStateChangedCallback; import org.apache.storm.generated.Assignment; import org.apache.storm.generated.ClusterWorkerHeartbeat; @@ -64,6 +66,7 @@ public class StormClusterStateImpl implements IStormClusterState { private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class); private IStateStorage stateStorage; + private ILocalAssignmentsBackend assignmentsBackend; private ConcurrentHashMap assignmentInfoCallback; private ConcurrentHashMap assignmentInfoWithVersionCallback; @@ -71,7 +74,7 @@ public class StormClusterStateImpl implements IStormClusterState { private AtomicReference supervisorsCallback; // we want to register a topo directory getChildren callback for all workers of this dir private ConcurrentHashMap backPressureCallback; - + private AtomicReference leaderInfoCallback; private AtomicReference assignmentsCallback; private ConcurrentHashMap stormBaseCallback; private AtomicReference blobstoreCallback; @@ -83,18 +86,20 @@ public class StormClusterStateImpl implements IStormClusterState { private final boolean solo; private final ClusterStateContext context; - public StormClusterStateImpl(IStateStorage StateStorage, ClusterStateContext context, boolean solo) throws Exception { + public StormClusterStateImpl(IStateStorage StateStorage, ILocalAssignmentsBackend assignmentsassignmentsBackend, + ClusterStateContext context, boolean solo) throws Exception { this.stateStorage = StateStorage; this.solo = solo; this.defaultAcls = context.getDefaultZkAcls(); this.context = context; - + this.assignmentsBackend = assignmentsassignmentsBackend; assignmentInfoCallback = new ConcurrentHashMap<>(); assignmentInfoWithVersionCallback = new ConcurrentHashMap<>(); assignmentVersionCallback = new ConcurrentHashMap<>(); supervisorsCallback = new AtomicReference<>(); backPressureCallback = new ConcurrentHashMap<>(); + leaderInfoCallback = new AtomicReference<>(); assignmentsCallback = new AtomicReference<>(); stormBaseCallback = new ConcurrentHashMap<>(); credentialsCallback = new ConcurrentHashMap<>(); @@ -130,6 +135,8 @@ public void changed(Watcher.Event.EventType type, String path) { issueMapCallback(logConfigCallback, toks.get(1)); } else if (root.equals(ClusterUtils.BACKPRESSURE_ROOT) && size > 1) { issueMapCallback(backPressureCallback, toks.get(1)); + } else if (root.equals(ClusterUtils.LEADERINFO_ROOT)) { + issueCallback(leaderInfoCallback); } else { LOG.error("{} Unknown callback for subtree {}", new RuntimeException("Unknown callback for this path"), path); Runtime.getRuntime().exit(30); @@ -159,14 +166,16 @@ public void changed(Watcher.Event.EventType type, String path) { protected void issueCallback(AtomicReference cb) { Runnable callback = cb.getAndSet(null); - if (callback != null) + if (callback != null) { callback.run(); + } } protected void issueMapCallback(ConcurrentHashMap callbackConcurrentHashMap, String key) { Runnable callback = callbackConcurrentHashMap.remove(key); - if (callback != null) + if (callback != null) { callback.run(); + } } @Override @@ -174,11 +183,19 @@ public List assignments(Runnable callback) { if (callback != null) { assignmentsCallback.set(callback); } - return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null); + return this.assignmentsBackend.assignments(); } @Override public Assignment assignmentInfo(String stormId, Runnable callback) { + if (callback != null) { + assignmentInfoCallback.put(stormId, callback); + } + return this.assignmentsBackend.getAssignment(stormId); + } + + @Override + public Assignment remoteAssignmentInfo(String stormId, Runnable callback) { if (callback != null) { assignmentInfoCallback.put(stormId, callback); } @@ -186,6 +203,36 @@ public Assignment assignmentInfo(String stormId, Runnable callback) { return ClusterUtils.maybeDeserialize(serialized, Assignment.class); } + @Override + public Map assignmentsInfo() { + return this.assignmentsBackend.assignmentsInfo(); + } + + @Override + public void syncRemoteAssignments(Map remote) { + if (null != remote) { + this.assignmentsBackend.syncRemoteAssignments(remote); + } else { + Map tmp = new HashMap<>(); + List stormIds = this.stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, false); + for (String stormId : stormIds) { + byte[] assignment = this.stateStorage.get_data(ClusterUtils.assignmentPath(stormId), false); + tmp.put(stormId, assignment); + } + this.assignmentsBackend.syncRemoteAssignments(tmp); + } + } + + @Override + public boolean isAssignmentsBackendSynchronized() { + return this.assignmentsBackend.isSynchronized(); + } + + @Override + public void setAssignmentsBackendSynchronized() { + this.assignmentsBackend.setSynchronized(); + } + @Override public VersionedData assignmentInfoWithVersion(String stormId, Runnable callback) { if (callback != null) { @@ -267,6 +314,25 @@ public StormBase stormBase(String stormId, Runnable callback) { return ClusterUtils.maybeDeserialize(stateStorage.get_data(ClusterUtils.stormPath(stormId), callback != null), StormBase.class); } + @Override + public String stormId(String stormName) { + return this.assignmentsBackend.getStormId(stormName); + } + + @Override + public void syncRemoteIds(Map remote) { + if (null != remote) { + this.assignmentsBackend.syncRemoteIds(remote); + }else { + Map tmp = new HashMap<>(); + List activeStorms = activeStorms(); + for (String stormId: activeStorms) { + tmp.put(stormId, stormBase(stormId, null).get_name()); + } + this.assignmentsBackend.syncRemoteIds(tmp); + } + } + @Override public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port) { byte[] bytes = stateStorage.get_worker_hb(ClusterUtils.workerbeatPath(stormId, node, port), false); @@ -280,8 +346,9 @@ public List getWorkerProfileRequests(String stormId, NodeInfo no List profileRequests = getTopologyProfileRequests(stormId); for (ProfileRequest profileRequest : profileRequests) { NodeInfo nodeInfo1 = profileRequest.get_nodeInfo(); - if (nodeInfo1.equals(nodeInfo)) + if (nodeInfo1.equals(nodeInfo)) { requests.add(profileRequest); + } } return requests; } @@ -296,8 +363,9 @@ public List getTopologyProfileRequests(String stormId) { String childPath = path + ClusterUtils.ZK_SEPERATOR + str; byte[] raw = stateStorage.get_data(childPath, false); ProfileRequest request = ClusterUtils.maybeDeserialize(raw, ProfileRequest.class); - if (request != null) + if (request != null) { profileRequests.add(request); + } } } return profileRequests; @@ -322,13 +390,13 @@ public void deleteTopologyProfileRequests(String stormId, ProfileRequest profile } /** - * need to take executor->node+port in explicitly so that we don't run into a situation where a long dead worker with a skewed clock overrides all the - * timestamps. By only checking heartbeats with an assigned node+port, and only reading executors from that heartbeat that are actually assigned, we avoid - * situations like that + * need to take executor->node+port in explicitly so that we don't run into a situation where a long dead worker + * with a skewed clock overrides all the timestamps. By only checking heartbeats with an assigned node+port, + * and only reading executors from that heartbeat that are actually assigned, we avoid situations like that. * - * @param stormId - * @param executorNodePort - * @return + * @param stormId topology id + * @param executorNodePort executor id -> node + port + * @return mapping of executorInfo -> executor beat */ @Override public Map executorBeats(String stormId, Map, NodeInfo> executorNodePort) { @@ -345,8 +413,9 @@ public Map executorBeats(String stormId, Map list : entry.getValue()) { executorInfoList.add(new ExecutorInfo(list.get(0).intValue(), list.get(list.size() - 1).intValue())); } - if (whb != null) + if (whb != null) { executorWhbs.putAll(ClusterUtils.convertExecutorBeats(executorInfoList, whb)); + } } return executorWhbs; } @@ -398,6 +467,14 @@ public void teardownTopologyErrors(String stormId) { } } + @Override + public NimbusInfo getLeader(Runnable callback) { + if (null != callback) { + this.leaderInfoCallback.set(callback); + } + return Utils.javaDeserialize(this.stateStorage.get_data(ClusterUtils.LEADERINFO_SUBTREE, callback != null), NimbusInfo.class); + } + @Override public List backpressureTopologies() { return stateStorage.get_children(ClusterUtils.BACKPRESSURE_SUBTREE, false); @@ -508,6 +585,7 @@ public void removeWorkerBackpressure(String stormId, String node, Long port) { public void activateStorm(String stormId, StormBase stormBase) { String path = ClusterUtils.stormPath(stormId); stateStorage.set_data(path, Utils.serialize(stormBase), defaultAcls); + this.assignmentsBackend.keepStormId(stormBase.get_name(), stormId); } /** @@ -533,8 +611,9 @@ public void updateStorm(String stormId, StormBase newElems) { newComponentExecutors.put(entry.getKey(), entry.getValue()); } } - if (newComponentExecutors.size() > 0) + if (newComponentExecutors.size() > 0) { newElems.set_component_executors(newComponentExecutors); + } } Map ComponentDebug = new HashMap<>(); @@ -605,7 +684,9 @@ public void removeStormBase(String stormId) { @Override public void setAssignment(String stormId, Assignment info) { - stateStorage.set_data(ClusterUtils.assignmentPath(stormId), Utils.serialize(info), defaultAcls); + byte[] serAssignment = Utils.serialize(info); + stateStorage.set_data(ClusterUtils.assignmentPath(stormId), serAssignment, defaultAcls); + this.assignmentsBackend.keepOrUpdateAssignment(stormId, info); } @Override @@ -636,6 +717,7 @@ public List blobstore(Runnable callback) { @Override public void removeStorm(String stormId) { stateStorage.delete_node(ClusterUtils.assignmentPath(stormId)); + this.assignmentsBackend.clearStateForStorm(stormId); stateStorage.delete_node(ClusterUtils.credentialsPath(stormId)); stateStorage.delete_node(ClusterUtils.logConfigPath(stormId)); stateStorage.delete_node(ClusterUtils.profilerConfigPath(stormId)); @@ -697,8 +779,9 @@ public List errors(String stormId, String componentId) { for (String child : childrens) { String childPath = path + ClusterUtils.ZK_SEPERATOR + child; ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class); - if (errorInfo != null) + if (errorInfo != null) { errorInfos.add(errorInfo); + } } } Collections.sort(errorInfos, new Comparator() { @@ -745,6 +828,7 @@ public void disconnect() { stateStorage.unregister(stateId); if (solo) { stateStorage.close(); + this.assignmentsBackend.close(); } } diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java index d5c29f9cc1d..8ae6c5f998e 100644 --- a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java +++ b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java @@ -15,8 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.cluster; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.ConcurrentHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.state.*; import org.apache.curator.framework.state.ConnectionState; @@ -33,13 +41,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; - public class ZKStateStorage implements IStateStorage { private static Logger LOG = LoggerFactory.getLogger(ZKStateStorage.class); diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java index 0ae745f02d5..50760455ea6 100644 --- a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java +++ b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm.cluster; -import org.apache.storm.utils.Utils; -import org.apache.zookeeper.data.ACL; +package org.apache.storm.cluster; import java.util.List; import java.util.Map; +import org.apache.storm.utils.Utils; +import org.apache.zookeeper.data.ACL; + public class ZKStateStorageFactory implements StateStorageFactory { @Override diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java index ca0d4d04f3a..94ea9afa3b2 100644 --- a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java +++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java @@ -57,6 +57,7 @@ import org.apache.storm.generated.ExecutorStats; import org.apache.storm.generated.LSWorkerHeartbeat; import org.apache.storm.generated.LogConfig; +import org.apache.storm.generated.SupervisorWorkerHeartbeat; import org.apache.storm.messaging.IConnection; import org.apache.storm.messaging.IContext; import org.apache.storm.security.auth.AuthUtils; @@ -64,7 +65,9 @@ import org.apache.storm.stats.StatsUtil; import org.apache.storm.utils.ConfigUtils; import org.apache.storm.utils.LocalState; +import org.apache.storm.utils.NimbusClient; import org.apache.storm.utils.ObjectReader; +import org.apache.storm.utils.SupervisorClient; import org.apache.storm.utils.Time; import org.apache.storm.utils.Utils; import org.apache.zookeeper.data.ACL; @@ -82,6 +85,7 @@ public class Worker implements Shutdownable, DaemonCommon { private final IContext context; private final String topologyId; private final String assignmentId; + private final int supervisorPort; private final int port; private final String workerId; private final LogConfigManager logConfigManager; @@ -105,15 +109,18 @@ public class Worker implements Shutdownable, DaemonCommon { * @param context - * @param topologyId - topology id * @param assignmentId - assignment id + * @param supervisorPort - parent supervisor thrift server port * @param port - port on which the worker runs * @param workerId - worker id */ - public Worker(Map conf, IContext context, String topologyId, String assignmentId, int port, String workerId) { + public Worker(Map conf, IContext context, String topologyId, String assignmentId, + int supervisorPort, int port, String workerId) { this.conf = conf; this.context = context; this.topologyId = topologyId; this.assignmentId = assignmentId; + this.supervisorPort = supervisorPort; this.port = port; this.workerId = workerId; this.logConfigManager = new LogConfigManager(); @@ -137,7 +144,7 @@ public void start() throws Exception { ConfigUtils.overrideLoginConfigWithSystemProperty(ConfigUtils.readSupervisorStormConf(conf, topologyId)); ClusterStateContext csContext = new ClusterStateContext(DaemonType.WORKER, topologyConf); IStateStorage stateStorage = ClusterUtils.mkStateStorage(conf, topologyConf, csContext); - IStormClusterState stormClusterState = ClusterUtils.mkStormClusterState(stateStorage, csContext); + IStormClusterState stormClusterState = ClusterUtils.mkStormClusterState(stateStorage, null, csContext); Credentials initialCredentials = stormClusterState.credentials(topologyId, null); Map initCreds = new HashMap<>(); @@ -156,8 +163,8 @@ public void start() throws Exception { private Object loadWorker(Map topologyConf, IStateStorage stateStorage, IStormClusterState stormClusterState, Map initCreds, Credentials initialCredentials) throws Exception { - workerState = new WorkerState(conf, context, topologyId, assignmentId, port, workerId, topologyConf, stateStorage, - stormClusterState, autoCreds); + workerState = new WorkerState(conf, context, topologyId, assignmentId, supervisorPort, port, workerId, + topologyConf, stateStorage, stormClusterState, autoCreds); // Heartbeat here so that worker process dies if this fails // it's important that worker heartbeat to supervisor ASAP so that supervisor knows @@ -178,7 +185,7 @@ private Object loadWorker(Map topologyConf, IStateStorage stateS }); workerState.executorHeartbeatTimer - .scheduleRecurring(0, (Integer) conf.get(Config.WORKER_HEARTBEAT_FREQUENCY_SECS), + .scheduleRecurring(0, (Integer) conf.get(Config.EXECUTOR_METRICS_FREQUENCY_SECS), Worker.this::doExecutorHeartbeats); workerState.registerCallbacks(); @@ -313,12 +320,14 @@ public void doRefreshLoad() { public void doHeartBeat() throws IOException { LocalState state = ConfigUtils.workerState(workerState.conf, workerState.workerId); - state.setWorkerHeartBeat(new LSWorkerHeartbeat(Time.currentTimeSecs(), workerState.topologyId, + LSWorkerHeartbeat lsWorkerHeartbeat = new LSWorkerHeartbeat(Time.currentTimeSecs(), workerState.topologyId, workerState.localExecutors.stream() .map(executor -> new ExecutorInfo(executor.get(0).intValue(), executor.get(1).intValue())) - .collect(Collectors.toList()), workerState.port)); + .collect(Collectors.toList()), workerState.port); + state.setWorkerHeartBeat(lsWorkerHeartbeat); state.cleanup(60); // this is just in case supervisor is down so that disk doesn't fill up. // it shouldn't take supervisor 120 seconds between listing dir and reading it + heartbeatToMasterIfLocalbeatFail(lsWorkerHeartbeat); } public void doExecutorHeartbeats() { @@ -392,6 +401,30 @@ public void establishLogSettingCallback() { workerState.stormClusterState.topologyLogConfig(topologyId, this::checkLogConfigChanged); } + /** + * Send a heartbeat to local supervisor first to check if supervisor is ok for heartbeating. + */ + private void heartbeatToMasterIfLocalbeatFail(LSWorkerHeartbeat lsWorkerHeartbeat) { + if (ConfigUtils.isLocalMode(this.conf)) { + return; + } + //In distributed mode, send heartbeat directly to master if local supervisor goes down. + SupervisorWorkerHeartbeat workerHeartbeat = new SupervisorWorkerHeartbeat(lsWorkerHeartbeat.get_topology_id(), + lsWorkerHeartbeat.get_executors(), lsWorkerHeartbeat.get_time_secs()); + try (SupervisorClient client = SupervisorClient.getConfiguredClient(conf, Utils.hostname(), supervisorPort)){ + client.getClient().sendSupervisorWorkerHeartbeat(workerHeartbeat); + } catch (Exception tr1) { + //If any error/exception thrown, report directly to nimbus. + LOG.warn("Exception when send heartbeat to local supervisor", tr1.getMessage()); + try (NimbusClient nimbusClient = NimbusClient.getConfiguredClient(conf)){ + nimbusClient.getClient().sendSupervisorWorkerHeartbeat(workerHeartbeat); + } catch (Exception tr2) { + //if any error/exception thrown, just ignore. + LOG.error("Exception when send heartbeat to master", tr2.getMessage()); + } + } + } + @Override public void shutdown() { try { @@ -463,15 +496,17 @@ public boolean isWaiting() { } public static void main(String[] args) throws Exception { - Preconditions.checkArgument(args.length == 4, "Illegal number of arguments. Expected: 4, Actual: " + args.length); + Preconditions.checkArgument(args.length == 5, "Illegal number of arguments. Expected: 5, Actual: " + args.length); String stormId = args[0]; String assignmentId = args[1]; - String portStr = args[2]; - String workerId = args[3]; + String supervisorPort = args[2]; + String portStr = args[3]; + String workerId = args[4]; Map conf = Utils.readStormConfig(); Utils.setupDefaultUncaughtExceptionHandler(); StormCommon.validateDistributedMode(conf); - Worker worker = new Worker(conf, null, stormId, assignmentId, Integer.parseInt(portStr), workerId); + Worker worker = new Worker(conf, null, stormId, assignmentId, Integer.parseInt(supervisorPort), + Integer.parseInt(portStr), workerId); worker.start(); Utils.addShutdownHookWithForceKillIn1Sec(worker::shutdown); } diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java index d08047100f4..992dcbc4d44 100644 --- a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java +++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java @@ -81,6 +81,7 @@ import org.apache.storm.utils.JCQueue; import org.apache.storm.utils.Utils; import org.apache.storm.utils.ObjectReader; +import org.apache.storm.utils.SupervisorClient; import org.apache.storm.utils.ThriftTopologyUtils; import org.apache.storm.utils.Utils.SmartThread; import org.slf4j.Logger; @@ -199,6 +200,7 @@ public Map getUserSharedResources() { final IConnection receiver; final String topologyId; final String assignmentId; + final int supervisorPort; final int port; final String workerId; final IStateStorage stateStorage; @@ -272,18 +274,17 @@ public StormTimer getUserTimer() { private final Collection autoCredentials; private static final long LOAD_REFRESH_INTERVAL_MS = 5000L; - public WorkerState(Map conf, IContext mqContext, String topologyId, String assignmentId, int port, String workerId, - Map topologyConf, IStateStorage stateStorage, IStormClusterState stormClusterState, - Collection autoCredentials) - throws IOException, InvalidTopologyException { + public WorkerState(Map conf, IContext mqContext, String topologyId, String assignmentId, + int supervisorPort, int port, String workerId, Map topologyConf, IStateStorage stateStorage, + IStormClusterState stormClusterState, Collection autoCredentials) throws IOException, InvalidTopologyException { this.autoCredentials = autoCredentials; - this.localExecutors = new HashSet<>(readWorkerExecutors(stormClusterState, topologyId, assignmentId, port)); - this.conf = conf; + this.localExecutors = new HashSet<>(readWorkerExecutors(stormClusterState, topologyId, assignmentId, port)); this.mqContext = (null != mqContext) ? mqContext : TransportFactory.makeContext(topologyConf); this.receiver = this.mqContext.bind(topologyId, port); this.topologyId = topologyId; this.assignmentId = assignmentId; + this.supervisorPort = supervisorPort; this.port = port; this.workerId = workerId; this.stateStorage = stateStorage; @@ -354,23 +355,7 @@ public SmartThread makeTransferThread() { } public void refreshConnections(Runnable callback) throws Exception { - Integer version = stormClusterState.assignmentVersion(topologyId, callback); - version = (null == version) ? 0 : version; - VersionedData assignmentVersion = assignmentVersions.get().get(topologyId); - Assignment assignment; - if (null != assignmentVersion && (assignmentVersion.getVersion() == version)) { - assignment = assignmentVersion.getData(); - } else { - VersionedData - newAssignmentVersion = new VersionedData<>(version, - stormClusterState.assignmentInfoWithVersion(topologyId, callback).getData()); - assignmentVersions.getAndUpdate(prev -> { - Map> next = new HashMap<>(prev); - next.put(topologyId, newAssignmentVersion); - return next; - }); - assignment = newAssignmentVersion.getData(); - } + Assignment assignment = getLocalAssignment(conf, stormClusterState, topologyId); Set neededConnections = new HashSet<>(); Map newTaskToNodePort = new HashMap<>(); @@ -645,8 +630,7 @@ private List> readWorkerExecutors(IStormClusterState stormClusterStat LOG.info("Reading assignments"); List> executorsAssignedToThisWorker = new ArrayList<>(); executorsAssignedToThisWorker.add(Constants.SYSTEM_EXECUTOR_ID); - Map, NodeInfo> executorToNodePort = - stormClusterState.assignmentInfo(topologyId, null).get_executor_node_port(); + Map, NodeInfo> executorToNodePort = getLocalAssignment(conf, stormClusterState, topologyId).get_executor_node_port(); for (Map.Entry, NodeInfo> entry : executorToNodePort.entrySet()) { NodeInfo nodeInfo = entry.getValue(); if (nodeInfo.get_node().equals(assignmentId) && nodeInfo.get_port().iterator().next() == port) { @@ -656,6 +640,21 @@ private List> readWorkerExecutors(IStormClusterState stormClusterStat return executorsAssignedToThisWorker; } + private Assignment getLocalAssignment(Map conf, IStormClusterState stormClusterState, String topologyId) { + if (!ConfigUtils.isLocalMode(conf)) { + try (SupervisorClient supervisorClient = SupervisorClient.getConfiguredClient(conf, Utils.hostname(), + supervisorPort)){ + Assignment assignment = supervisorClient.getClient().getLocalAssignmentForStorm(topologyId); + return assignment; + } catch (Throwable tr1) { + //if any error/exception thrown, fetch it from zookeeper + return stormClusterState.remoteAssignmentInfo(topologyId, null); + } + } else { + return stormClusterState.remoteAssignmentInfo(topologyId, null); + } + } + private Map, JCQueue> mkReceiveQueueMap(Map topologyConf, Set> executors) { Integer recvQueueSize = ObjectReader.getInt(topologyConf.get(Config.TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE)); Integer recvBatchSize = ObjectReader.getInt(topologyConf.get(Config.TOPOLOGY_PRODUCER_BATCH_SIZE)); 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 887dbc0f919..4bbd4f4fbfe 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 _list886 = iprot.readListBegin(); - struct.pulseIds = new ArrayList(_list886.size); - String _elem887; - for (int _i888 = 0; _i888 < _list886.size; ++_i888) + org.apache.thrift.protocol.TList _list912 = iprot.readListBegin(); + struct.pulseIds = new ArrayList(_list912.size); + String _elem913; + for (int _i914 = 0; _i914 < _list912.size; ++_i914) { - _elem887 = iprot.readString(); - struct.pulseIds.add(_elem887); + _elem913 = iprot.readString(); + struct.pulseIds.add(_elem913); } 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 _iter889 : struct.pulseIds) + for (String _iter915 : struct.pulseIds) { - oprot.writeString(_iter889); + oprot.writeString(_iter915); } 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 _iter890 : struct.pulseIds) + for (String _iter916 : struct.pulseIds) { - oprot.writeString(_iter890); + oprot.writeString(_iter916); } } } @@ -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 _list891 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.pulseIds = new ArrayList(_list891.size); - String _elem892; - for (int _i893 = 0; _i893 < _list891.size; ++_i893) + org.apache.thrift.protocol.TList _list917 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.pulseIds = new ArrayList(_list917.size); + String _elem918; + for (int _i919 = 0; _i919 < _list917.size; ++_i919) { - _elem892 = iprot.readString(); - struct.pulseIds.add(_elem892); + _elem918 = iprot.readString(); + struct.pulseIds.add(_elem918); } } 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 cfed785185c..fae00cfbc16 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 _list878 = iprot.readListBegin(); - struct.pulses = new ArrayList(_list878.size); - HBPulse _elem879; - for (int _i880 = 0; _i880 < _list878.size; ++_i880) + org.apache.thrift.protocol.TList _list904 = iprot.readListBegin(); + struct.pulses = new ArrayList(_list904.size); + HBPulse _elem905; + for (int _i906 = 0; _i906 < _list904.size; ++_i906) { - _elem879 = new HBPulse(); - _elem879.read(iprot); - struct.pulses.add(_elem879); + _elem905 = new HBPulse(); + _elem905.read(iprot); + struct.pulses.add(_elem905); } 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 _iter881 : struct.pulses) + for (HBPulse _iter907 : struct.pulses) { - _iter881.write(oprot); + _iter907.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 _iter882 : struct.pulses) + for (HBPulse _iter908 : struct.pulses) { - _iter882.write(oprot); + _iter908.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 _list883 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.pulses = new ArrayList(_list883.size); - HBPulse _elem884; - for (int _i885 = 0; _i885 < _list883.size; ++_i885) + org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.pulses = new ArrayList(_list909.size); + HBPulse _elem910; + for (int _i911 = 0; _i911 < _list909.size; ++_i911) { - _elem884 = new HBPulse(); - _elem884.read(iprot); - struct.pulses.add(_elem884); + _elem910 = new HBPulse(); + _elem910.read(iprot); + struct.pulses.add(_elem910); } } struct.set_pulses_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 8b0254859a7..ddefe4cad2d 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java +++ b/storm-client/src/jvm/org/apache/storm/generated/Nimbus.java @@ -170,6 +170,27 @@ public interface Iface { public List getOwnerResourceSummaries(String owner) throws AuthorizationException, org.apache.thrift.TException; + /** + * Get assigned assignments for a specific supervisor + * + * @param node + */ + public SupervisorAssignments getSupervisorAssignments(String node) throws AuthorizationException, org.apache.thrift.TException; + + /** + * Send supervisor worker heartbeats for a specific supervisor + * + * @param heartbeats + */ + public void sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats) throws AuthorizationException, org.apache.thrift.TException; + + /** + * Send supervisor local worker heartbeat when a supervisor is unreachable + * + * @param heatbeat + */ + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heatbeat) throws AuthorizationException, org.apache.thrift.TException; + public void processWorkerMetrics(WorkerMetrics metrics) throws org.apache.thrift.TException; } @@ -268,6 +289,12 @@ public interface AsyncIface { public void getOwnerResourceSummaries(String owner, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getSupervisorAssignments(String node, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heatbeat, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void processWorkerMetrics(WorkerMetrics metrics, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; } @@ -1526,6 +1553,78 @@ public List recv_getOwnerResourceSummaries() throws Author throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getOwnerResourceSummaries failed: unknown result"); } + public SupervisorAssignments getSupervisorAssignments(String node) throws AuthorizationException, org.apache.thrift.TException + { + send_getSupervisorAssignments(node); + return recv_getSupervisorAssignments(); + } + + public void send_getSupervisorAssignments(String node) throws org.apache.thrift.TException + { + getSupervisorAssignments_args args = new getSupervisorAssignments_args(); + args.set_node(node); + sendBase("getSupervisorAssignments", args); + } + + public SupervisorAssignments recv_getSupervisorAssignments() throws AuthorizationException, org.apache.thrift.TException + { + getSupervisorAssignments_result result = new getSupervisorAssignments_result(); + receiveBase(result, "getSupervisorAssignments"); + if (result.is_set_success()) { + return result.success; + } + if (result.aze != null) { + throw result.aze; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getSupervisorAssignments failed: unknown result"); + } + + public void sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats) throws AuthorizationException, org.apache.thrift.TException + { + send_sendSupervisorWorkerHeartbeats(heartbeats); + recv_sendSupervisorWorkerHeartbeats(); + } + + public void send_sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats) throws org.apache.thrift.TException + { + sendSupervisorWorkerHeartbeats_args args = new sendSupervisorWorkerHeartbeats_args(); + args.set_heartbeats(heartbeats); + sendBase("sendSupervisorWorkerHeartbeats", args); + } + + public void recv_sendSupervisorWorkerHeartbeats() throws AuthorizationException, org.apache.thrift.TException + { + sendSupervisorWorkerHeartbeats_result result = new sendSupervisorWorkerHeartbeats_result(); + receiveBase(result, "sendSupervisorWorkerHeartbeats"); + if (result.aze != null) { + throw result.aze; + } + return; + } + + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heatbeat) throws AuthorizationException, org.apache.thrift.TException + { + send_sendSupervisorWorkerHeartbeat(heatbeat); + recv_sendSupervisorWorkerHeartbeat(); + } + + public void send_sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heatbeat) throws org.apache.thrift.TException + { + sendSupervisorWorkerHeartbeat_args args = new sendSupervisorWorkerHeartbeat_args(); + args.set_heatbeat(heatbeat); + sendBase("sendSupervisorWorkerHeartbeat", args); + } + + public void recv_sendSupervisorWorkerHeartbeat() throws AuthorizationException, org.apache.thrift.TException + { + sendSupervisorWorkerHeartbeat_result result = new sendSupervisorWorkerHeartbeat_result(); + receiveBase(result, "sendSupervisorWorkerHeartbeat"); + if (result.aze != null) { + throw result.aze; + } + return; + } + public void processWorkerMetrics(WorkerMetrics metrics) throws org.apache.thrift.TException { send_processWorkerMetrics(metrics); @@ -3114,6 +3213,102 @@ public List getResult() throws AuthorizationException, org } } + public void getSupervisorAssignments(String node, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getSupervisorAssignments_call method_call = new getSupervisorAssignments_call(node, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getSupervisorAssignments_call extends org.apache.thrift.async.TAsyncMethodCall { + private String node; + public getSupervisorAssignments_call(String node, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.node = node; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getSupervisorAssignments", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getSupervisorAssignments_args args = new getSupervisorAssignments_args(); + args.set_node(node); + args.write(prot); + prot.writeMessageEnd(); + } + + public SupervisorAssignments getResult() throws AuthorizationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getSupervisorAssignments(); + } + } + + public void sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + sendSupervisorWorkerHeartbeats_call method_call = new sendSupervisorWorkerHeartbeats_call(heartbeats, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class sendSupervisorWorkerHeartbeats_call extends org.apache.thrift.async.TAsyncMethodCall { + private SupervisorWorkerHeartbeats heartbeats; + public sendSupervisorWorkerHeartbeats_call(SupervisorWorkerHeartbeats heartbeats, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.heartbeats = heartbeats; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("sendSupervisorWorkerHeartbeats", org.apache.thrift.protocol.TMessageType.CALL, 0)); + sendSupervisorWorkerHeartbeats_args args = new sendSupervisorWorkerHeartbeats_args(); + args.set_heartbeats(heartbeats); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws AuthorizationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_sendSupervisorWorkerHeartbeats(); + } + } + + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heatbeat, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + sendSupervisorWorkerHeartbeat_call method_call = new sendSupervisorWorkerHeartbeat_call(heatbeat, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class sendSupervisorWorkerHeartbeat_call extends org.apache.thrift.async.TAsyncMethodCall { + private SupervisorWorkerHeartbeat heatbeat; + public sendSupervisorWorkerHeartbeat_call(SupervisorWorkerHeartbeat heatbeat, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.heatbeat = heatbeat; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("sendSupervisorWorkerHeartbeat", org.apache.thrift.protocol.TMessageType.CALL, 0)); + sendSupervisorWorkerHeartbeat_args args = new sendSupervisorWorkerHeartbeat_args(); + args.set_heatbeat(heatbeat); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws AuthorizationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_sendSupervisorWorkerHeartbeat(); + } + } + public void processWorkerMetrics(WorkerMetrics metrics, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); processWorkerMetrics_call method_call = new processWorkerMetrics_call(metrics, resultHandler, this, ___protocolFactory, ___transport); @@ -3205,6 +3400,9 @@ protected Processor(I iface, Map extends org.apache.thrift.ProcessFunction { + public getSupervisorAssignments() { + super("getSupervisorAssignments"); + } + + public getSupervisorAssignments_args getEmptyArgsInstance() { + return new getSupervisorAssignments_args(); + } + + protected boolean isOneway() { + return false; + } + + public getSupervisorAssignments_result getResult(I iface, getSupervisorAssignments_args args) throws org.apache.thrift.TException { + getSupervisorAssignments_result result = new getSupervisorAssignments_result(); + try { + result.success = iface.getSupervisorAssignments(args.node); + } catch (AuthorizationException aze) { + result.aze = aze; + } + return result; + } + } + + public static class sendSupervisorWorkerHeartbeats extends org.apache.thrift.ProcessFunction { + public sendSupervisorWorkerHeartbeats() { + super("sendSupervisorWorkerHeartbeats"); + } + + public sendSupervisorWorkerHeartbeats_args getEmptyArgsInstance() { + return new sendSupervisorWorkerHeartbeats_args(); + } + + protected boolean isOneway() { + return false; + } + + public sendSupervisorWorkerHeartbeats_result getResult(I iface, sendSupervisorWorkerHeartbeats_args args) throws org.apache.thrift.TException { + sendSupervisorWorkerHeartbeats_result result = new sendSupervisorWorkerHeartbeats_result(); + try { + iface.sendSupervisorWorkerHeartbeats(args.heartbeats); + } catch (AuthorizationException aze) { + result.aze = aze; + } + return result; + } + } + + public static class sendSupervisorWorkerHeartbeat extends org.apache.thrift.ProcessFunction { + public sendSupervisorWorkerHeartbeat() { + super("sendSupervisorWorkerHeartbeat"); + } + + public sendSupervisorWorkerHeartbeat_args getEmptyArgsInstance() { + return new sendSupervisorWorkerHeartbeat_args(); + } + + protected boolean isOneway() { + return false; + } + + public sendSupervisorWorkerHeartbeat_result getResult(I iface, sendSupervisorWorkerHeartbeat_args args) throws org.apache.thrift.TException { + sendSupervisorWorkerHeartbeat_result result = new sendSupervisorWorkerHeartbeat_result(); + try { + iface.sendSupervisorWorkerHeartbeat(args.heatbeat); + } catch (AuthorizationException aze) { + result.aze = aze; + } + return result; + } + } + public static class processWorkerMetrics extends org.apache.thrift.ProcessFunction { public processWorkerMetrics() { super("processWorkerMetrics"); @@ -4429,6 +4699,9 @@ protected AsyncProcessor(I iface, Map extends org.apache.thrift.AsyncProcessFunction { + public getSupervisorAssignments() { + super("getSupervisorAssignments"); + } + + public getSupervisorAssignments_args getEmptyArgsInstance() { + return new getSupervisorAssignments_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(SupervisorAssignments o) { + getSupervisorAssignments_result result = new getSupervisorAssignments_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getSupervisorAssignments_result result = new getSupervisorAssignments_result(); + if (e instanceof AuthorizationException) { + result.aze = (AuthorizationException) e; + result.set_aze_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getSupervisorAssignments_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getSupervisorAssignments(args.node,resultHandler); + } + } + + public static class sendSupervisorWorkerHeartbeats extends org.apache.thrift.AsyncProcessFunction { + public sendSupervisorWorkerHeartbeats() { + super("sendSupervisorWorkerHeartbeats"); + } + + public sendSupervisorWorkerHeartbeats_args getEmptyArgsInstance() { + return new sendSupervisorWorkerHeartbeats_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + sendSupervisorWorkerHeartbeats_result result = new sendSupervisorWorkerHeartbeats_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + sendSupervisorWorkerHeartbeats_result result = new sendSupervisorWorkerHeartbeats_result(); + if (e instanceof AuthorizationException) { + result.aze = (AuthorizationException) e; + result.set_aze_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, sendSupervisorWorkerHeartbeats_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.sendSupervisorWorkerHeartbeats(args.heartbeats,resultHandler); + } + } + + public static class sendSupervisorWorkerHeartbeat extends org.apache.thrift.AsyncProcessFunction { + public sendSupervisorWorkerHeartbeat() { + super("sendSupervisorWorkerHeartbeat"); + } + + public sendSupervisorWorkerHeartbeat_args getEmptyArgsInstance() { + return new sendSupervisorWorkerHeartbeat_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + sendSupervisorWorkerHeartbeat_result result = new sendSupervisorWorkerHeartbeat_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + sendSupervisorWorkerHeartbeat_result result = new sendSupervisorWorkerHeartbeat_result(); + if (e instanceof AuthorizationException) { + result.aze = (AuthorizationException) e; + result.set_aze_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, sendSupervisorWorkerHeartbeat_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.sendSupervisorWorkerHeartbeat(args.heatbeat,resultHandler); + } + } + public static class processWorkerMetrics extends org.apache.thrift.AsyncProcessFunction { public processWorkerMetrics() { super("processWorkerMetrics"); @@ -18415,14 +18857,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 _list894 = iprot.readListBegin(); - struct.success = new ArrayList(_list894.size); - ProfileRequest _elem895; - for (int _i896 = 0; _i896 < _list894.size; ++_i896) + org.apache.thrift.protocol.TList _list920 = iprot.readListBegin(); + struct.success = new ArrayList(_list920.size); + ProfileRequest _elem921; + for (int _i922 = 0; _i922 < _list920.size; ++_i922) { - _elem895 = new ProfileRequest(); - _elem895.read(iprot); - struct.success.add(_elem895); + _elem921 = new ProfileRequest(); + _elem921.read(iprot); + struct.success.add(_elem921); } iprot.readListEnd(); } @@ -18448,9 +18890,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 _iter897 : struct.success) + for (ProfileRequest _iter923 : struct.success) { - _iter897.write(oprot); + _iter923.write(oprot); } oprot.writeListEnd(); } @@ -18481,9 +18923,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getComponentPending if (struct.is_set_success()) { { oprot.writeI32(struct.success.size()); - for (ProfileRequest _iter898 : struct.success) + for (ProfileRequest _iter924 : struct.success) { - _iter898.write(oprot); + _iter924.write(oprot); } } } @@ -18495,14 +18937,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 _list899 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list899.size); - ProfileRequest _elem900; - for (int _i901 = 0; _i901 < _list899.size; ++_i901) + org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list925.size); + ProfileRequest _elem926; + for (int _i927 = 0; _i927 < _list925.size; ++_i927) { - _elem900 = new ProfileRequest(); - _elem900.read(iprot); - struct.success.add(_elem900); + _elem926 = new ProfileRequest(); + _elem926.read(iprot); + struct.success.add(_elem926); } } struct.set_success_isSet(true); @@ -43965,70 +44407,2864 @@ public String getFieldName() { public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_args.class, metaDataMap); + } + + public getTopologyConf_args() { + } + + public getTopologyConf_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyConf_args(getTopologyConf_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public getTopologyConf_args deepCopy() { + return new getTopologyConf_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyConf_args) + return this.equals((getTopologyConf_args)that); + return false; + } + + public boolean equals(getTopologyConf_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyConf_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyConf_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyConf_argsStandardSchemeFactory implements SchemeFactory { + public getTopologyConf_argsStandardScheme getScheme() { + return new getTopologyConf_argsStandardScheme(); + } + } + + private static class getTopologyConf_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyConf_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyConf_argsTupleSchemeFactory implements SchemeFactory { + public getTopologyConf_argsTupleScheme getScheme() { + return new getTopologyConf_argsTupleScheme(); + } + } + + private static class getTopologyConf_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class getTopologyConf_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyConf_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyConf_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyConf_resultTupleSchemeFactory()); + } + + private String success; // required + private NotAliveException e; // required + private AuthorizationException aze; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"), + AZE((short)2, "aze"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + case 2: // AZE + return AZE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_result.class, metaDataMap); + } + + public getTopologyConf_result() { + } + + public getTopologyConf_result( + String success, + NotAliveException e, + AuthorizationException aze) + { + this(); + this.success = success; + this.e = e; + this.aze = aze; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyConf_result(getTopologyConf_result other) { + if (other.is_set_success()) { + this.success = other.success; + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } + } + + public getTopologyConf_result deepCopy() { + return new getTopologyConf_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + this.aze = null; + } + + public String get_success() { + return this.success; + } + + public void set_success(String success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((String)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + case AZE: + return get_aze(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + case AZE: + return is_set_aze(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyConf_result) + return this.equals((getTopologyConf_result)that); + return false; + } + + public boolean equals(getTopologyConf_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_aze = true && (is_set_aze()); + list.add(present_aze); + if (present_aze) + list.add(aze); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyConf_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyConf_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyConf_resultStandardSchemeFactory implements SchemeFactory { + public getTopologyConf_resultStandardScheme getScheme() { + return new getTopologyConf_resultStandardScheme(); + } + } + + private static class getTopologyConf_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // AZE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyConf_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeString(struct.success); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.aze != null) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + struct.aze.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyConf_resultTupleSchemeFactory implements SchemeFactory { + public getTopologyConf_resultTupleScheme getScheme() { + return new getTopologyConf_resultTupleScheme(); + } + } + + private static class getTopologyConf_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + if (struct.is_set_aze()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_success()) { + oprot.writeString(struct.success); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_aze()) { + struct.aze.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(2)) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } + } + } + + } + + public static class getTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopology_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopology_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopology_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_args.class, metaDataMap); + } + + public getTopology_args() { + } + + public getTopology_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public getTopology_args(getTopology_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public getTopology_args deepCopy() { + return new getTopology_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopology_args) + return this.equals((getTopology_args)that); + return false; + } + + public boolean equals(getTopology_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopology_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopology_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopology_argsStandardSchemeFactory implements SchemeFactory { + public getTopology_argsStandardScheme getScheme() { + return new getTopology_argsStandardScheme(); + } + } + + private static class getTopology_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopology_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopology_argsTupleSchemeFactory implements SchemeFactory { + public getTopology_argsTupleScheme getScheme() { + return new getTopology_argsTupleScheme(); + } + } + + private static class getTopology_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class getTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopology_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopology_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopology_resultTupleSchemeFactory()); + } + + private StormTopology success; // required + private NotAliveException e; // required + private AuthorizationException aze; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"), + AZE((short)2, "aze"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + case 2: // AZE + return AZE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_result.class, metaDataMap); + } + + public getTopology_result() { + } + + public getTopology_result( + StormTopology success, + NotAliveException e, + AuthorizationException aze) + { + this(); + this.success = success; + this.e = e; + this.aze = aze; + } + + /** + * Performs a deep copy on other. + */ + public getTopology_result(getTopology_result other) { + if (other.is_set_success()) { + this.success = new StormTopology(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } + } + + public getTopology_result deepCopy() { + return new getTopology_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + this.aze = null; + } + + public StormTopology get_success() { + return this.success; + } + + public void set_success(StormTopology success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((StormTopology)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + case AZE: + return get_aze(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + case AZE: + return is_set_aze(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopology_result) + return this.equals((getTopology_result)that); + return false; + } + + public boolean equals(getTopology_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_aze = true && (is_set_aze()); + list.add(present_aze); + if (present_aze) + list.add(aze); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopology_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopology_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopology_resultStandardSchemeFactory implements SchemeFactory { + public getTopology_resultStandardScheme getScheme() { + return new getTopology_resultStandardScheme(); + } + } + + private static class getTopology_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new StormTopology(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // AZE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopology_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.aze != null) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + struct.aze.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopology_resultTupleSchemeFactory implements SchemeFactory { + public getTopology_resultTupleScheme getScheme() { + return new getTopology_resultTupleScheme(); + } + } + + private static class getTopology_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + if (struct.is_set_aze()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_aze()) { + struct.aze.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.success = new StormTopology(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(2)) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } + } + } + + } + + public static class getUserTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserTopology_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getUserTopology_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getUserTopology_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_args.class, metaDataMap); + } + + public getUserTopology_args() { + } + + public getUserTopology_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public getUserTopology_args(getUserTopology_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public getUserTopology_args deepCopy() { + return new getUserTopology_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getUserTopology_args) + return this.equals((getUserTopology_args)that); + return false; + } + + public boolean equals(getUserTopology_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(getUserTopology_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getUserTopology_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getUserTopology_argsStandardSchemeFactory implements SchemeFactory { + public getUserTopology_argsStandardScheme getScheme() { + return new getUserTopology_argsStandardScheme(); + } + } + + private static class getUserTopology_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTopology_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getUserTopology_argsTupleSchemeFactory implements SchemeFactory { + public getUserTopology_argsTupleScheme getScheme() { + return new getUserTopology_argsTupleScheme(); + } + } + + private static class getUserTopology_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getUserTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getUserTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class getUserTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserTopology_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getUserTopology_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getUserTopology_resultTupleSchemeFactory()); + } + + private StormTopology success; // required + private NotAliveException e; // required + private AuthorizationException aze; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"), + AZE((short)2, "aze"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + case 2: // AZE + return AZE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_result.class, metaDataMap); + } + + public getUserTopology_result() { + } + + public getUserTopology_result( + StormTopology success, + NotAliveException e, + AuthorizationException aze) + { + this(); + this.success = success; + this.e = e; + this.aze = aze; + } + + /** + * Performs a deep copy on other. + */ + public getUserTopology_result(getUserTopology_result other) { + if (other.is_set_success()) { + this.success = new StormTopology(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } + } + + public getUserTopology_result deepCopy() { + return new getUserTopology_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + this.aze = null; + } + + public StormTopology get_success() { + return this.success; + } + + public void set_success(StormTopology success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((StormTopology)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + case AZE: + return get_aze(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + case AZE: + return is_set_aze(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getUserTopology_result) + return this.equals((getUserTopology_result)that); + return false; + } + + public boolean equals(getUserTopology_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_aze = true && (is_set_aze()); + list.add(present_aze); + if (present_aze) + list.add(aze); + + return list.hashCode(); + } + + @Override + public int compareTo(getUserTopology_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getUserTopology_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getUserTopology_resultStandardSchemeFactory implements SchemeFactory { + public getUserTopology_resultStandardScheme getScheme() { + return new getUserTopology_resultStandardScheme(); + } + } + + private static class getUserTopology_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new StormTopology(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // AZE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTopology_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.aze != null) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + struct.aze.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getUserTopology_resultTupleSchemeFactory implements SchemeFactory { + public getUserTopology_resultTupleScheme getScheme() { + return new getUserTopology_resultTupleScheme(); + } + } + + private static class getUserTopology_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getUserTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + if (struct.is_set_aze()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_aze()) { + struct.aze.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getUserTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.success = new StormTopology(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(2)) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } + } + } + + } + + public static class getTopologyHistory_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyHistory_args"); + + private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyHistory_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyHistory_argsTupleSchemeFactory()); + } + + private String user; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + USER((short)1, "user"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // USER + return USER; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyHistory_args.class, metaDataMap); } - public getTopologyConf_args() { + public getTopologyHistory_args() { } - public getTopologyConf_args( - String id) + public getTopologyHistory_args( + String user) { this(); - this.id = id; + this.user = user; } /** * Performs a deep copy on other. */ - public getTopologyConf_args(getTopologyConf_args other) { - if (other.is_set_id()) { - this.id = other.id; + public getTopologyHistory_args(getTopologyHistory_args other) { + if (other.is_set_user()) { + this.user = other.user; } } - public getTopologyConf_args deepCopy() { - return new getTopologyConf_args(this); + public getTopologyHistory_args deepCopy() { + return new getTopologyHistory_args(this); } @Override public void clear() { - this.id = null; + this.user = null; } - public String get_id() { - return this.id; + public String get_user() { + return this.user; } - public void set_id(String id) { - this.id = id; + public void set_user(String user) { + this.user = user; } - public void unset_id() { - this.id = null; + public void unset_user() { + this.user = null; } - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; + /** Returns true if field user is set (has been assigned a value) and false otherwise */ + public boolean is_set_user() { + return this.user != null; } - public void set_id_isSet(boolean value) { + public void set_user_isSet(boolean value) { if (!value) { - this.id = null; + this.user = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { - case ID: + case USER: if (value == null) { - unset_id(); + unset_user(); } else { - set_id((String)value); + set_user((String)value); } break; @@ -44037,8 +47273,8 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { - case ID: - return get_id(); + case USER: + return get_user(); } throw new IllegalStateException(); @@ -44051,8 +47287,8 @@ public boolean isSet(_Fields field) { } switch (field) { - case ID: - return is_set_id(); + case USER: + return is_set_user(); } throw new IllegalStateException(); } @@ -44061,21 +47297,21 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getTopologyConf_args) - return this.equals((getTopologyConf_args)that); + if (that instanceof getTopologyHistory_args) + return this.equals((getTopologyHistory_args)that); return false; } - public boolean equals(getTopologyConf_args that) { + public boolean equals(getTopologyHistory_args that) { if (that == null) return false; - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) + boolean this_present_user = true && this.is_set_user(); + boolean that_present_user = true && that.is_set_user(); + if (this_present_user || that_present_user) { + if (!(this_present_user && that_present_user)) return false; - if (!this.id.equals(that.id)) + if (!this.user.equals(that.user)) return false; } @@ -44086,28 +47322,28 @@ public boolean equals(getTopologyConf_args that) { public int hashCode() { List list = new ArrayList(); - boolean present_id = true && (is_set_id()); - list.add(present_id); - if (present_id) - list.add(id); + boolean present_user = true && (is_set_user()); + list.add(present_user); + if (present_user) + list.add(user); return list.hashCode(); } @Override - public int compareTo(getTopologyConf_args other) { + public int compareTo(getTopologyHistory_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + lastComparison = Boolean.valueOf(is_set_user()).compareTo(other.is_set_user()); if (lastComparison != 0) { return lastComparison; } - if (is_set_id()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (is_set_user()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, other.user); if (lastComparison != 0) { return lastComparison; } @@ -44129,14 +47365,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getTopologyConf_args("); + StringBuilder sb = new StringBuilder("getTopologyHistory_args("); boolean first = true; - sb.append("id:"); - if (this.id == null) { + sb.append("user:"); + if (this.user == null) { sb.append("null"); } else { - sb.append(this.id); + sb.append(this.user); } first = false; sb.append(")"); @@ -44164,15 +47400,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getTopologyConf_argsStandardSchemeFactory implements SchemeFactory { - public getTopologyConf_argsStandardScheme getScheme() { - return new getTopologyConf_argsStandardScheme(); + private static class getTopologyHistory_argsStandardSchemeFactory implements SchemeFactory { + public getTopologyHistory_argsStandardScheme getScheme() { + return new getTopologyHistory_argsStandardScheme(); } } - private static class getTopologyConf_argsStandardScheme extends StandardScheme { + private static class getTopologyHistory_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -44182,10 +47418,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_arg break; } switch (schemeField.id) { - case 1: // ID + case 1: // USER if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.id = iprot.readString(); - struct.set_id_isSet(true); + struct.user = iprot.readString(); + struct.set_user_isSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -44199,13 +47435,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_arg struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyConf_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyHistory_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(struct.id); + if (struct.user != null) { + oprot.writeFieldBegin(USER_FIELD_DESC); + oprot.writeString(struct.user); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -44214,62 +47450,59 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyConf_ar } - private static class getTopologyConf_argsTupleSchemeFactory implements SchemeFactory { - public getTopologyConf_argsTupleScheme getScheme() { - return new getTopologyConf_argsTupleScheme(); + private static class getTopologyHistory_argsTupleSchemeFactory implements SchemeFactory { + public getTopologyHistory_argsTupleScheme getScheme() { + return new getTopologyHistory_argsTupleScheme(); } } - private static class getTopologyConf_argsTupleScheme extends TupleScheme { + private static class getTopologyHistory_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); - if (struct.is_set_id()) { + if (struct.is_set_user()) { optionals.set(0); } oprot.writeBitSet(optionals, 1); - if (struct.is_set_id()) { - oprot.writeString(struct.id); + if (struct.is_set_user()) { + oprot.writeString(struct.user); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - struct.id = iprot.readString(); - struct.set_id_isSet(true); + struct.user = iprot.readString(); + struct.set_user_isSet(true); } } } } - public static class getTopologyConf_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyConf_result"); + public static class getTopologyHistory_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyHistory_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); - private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getTopologyConf_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getTopologyConf_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getTopologyHistory_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyHistory_resultTupleSchemeFactory()); } - private String success; // required - private NotAliveException e; // required + private TopologyHistoryInfo success; // required private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { SUCCESS((short)0, "success"), - E((short)1, "e"), - AZE((short)2, "aze"); + AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -44286,9 +47519,7 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; - case 1: // E - return E; - case 2: // AZE + case 1: // AZE return AZE; default: return null; @@ -44334,60 +47565,52 @@ public String getFieldName() { static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyHistoryInfo.class))); tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyHistory_result.class, metaDataMap); } - public getTopologyConf_result() { + public getTopologyHistory_result() { } - public getTopologyConf_result( - String success, - NotAliveException e, + public getTopologyHistory_result( + TopologyHistoryInfo success, AuthorizationException aze) { this(); this.success = success; - this.e = e; this.aze = aze; } /** * Performs a deep copy on other. */ - public getTopologyConf_result(getTopologyConf_result other) { + public getTopologyHistory_result(getTopologyHistory_result other) { if (other.is_set_success()) { - this.success = other.success; - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); + this.success = new TopologyHistoryInfo(other.success); } if (other.is_set_aze()) { this.aze = new AuthorizationException(other.aze); } } - public getTopologyConf_result deepCopy() { - return new getTopologyConf_result(this); + public getTopologyHistory_result deepCopy() { + return new getTopologyHistory_result(this); } @Override public void clear() { this.success = null; - this.e = null; this.aze = null; } - public String get_success() { + public TopologyHistoryInfo get_success() { return this.success; } - public void set_success(String success) { + public void set_success(TopologyHistoryInfo success) { this.success = success; } @@ -44406,29 +47629,6 @@ public void set_success_isSet(boolean value) { } } - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - public AuthorizationException get_aze() { return this.aze; } @@ -44458,15 +47658,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unset_success(); } else { - set_success((String)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); + set_success((TopologyHistoryInfo)value); } break; @@ -44486,9 +47678,6 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return get_success(); - case E: - return get_e(); - case AZE: return get_aze(); @@ -44505,8 +47694,6 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return is_set_success(); - case E: - return is_set_e(); case AZE: return is_set_aze(); } @@ -44517,12 +47704,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getTopologyConf_result) - return this.equals((getTopologyConf_result)that); + if (that instanceof getTopologyHistory_result) + return this.equals((getTopologyHistory_result)that); return false; } - public boolean equals(getTopologyConf_result that) { + public boolean equals(getTopologyHistory_result that) { if (that == null) return false; @@ -44535,15 +47722,6 @@ public boolean equals(getTopologyConf_result that) { return false; } - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - boolean this_present_aze = true && this.is_set_aze(); boolean that_present_aze = true && that.is_set_aze(); if (this_present_aze || that_present_aze) { @@ -44565,11 +47743,6 @@ public int hashCode() { if (present_success) list.add(success); - boolean present_e = true && (is_set_e()); - list.add(present_e); - if (present_e) - list.add(e); - boolean present_aze = true && (is_set_aze()); list.add(present_aze); if (present_aze) @@ -44579,7 +47752,7 @@ public int hashCode() { } @Override - public int compareTo(getTopologyConf_result other) { + public int compareTo(getTopologyHistory_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -44596,16 +47769,6 @@ public int compareTo(getTopologyConf_result other) { return lastComparison; } } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); if (lastComparison != 0) { return lastComparison; @@ -44633,7 +47796,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getTopologyConf_result("); + StringBuilder sb = new StringBuilder("getTopologyHistory_result("); boolean first = true; sb.append("success:"); @@ -44644,14 +47807,6 @@ public String toString() { } first = false; if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - if (!first) sb.append(", "); sb.append("aze:"); if (this.aze == null) { sb.append("null"); @@ -44666,6 +47821,9 @@ public String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity + if (success != null) { + success.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -44684,15 +47842,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getTopologyConf_resultStandardSchemeFactory implements SchemeFactory { - public getTopologyConf_resultStandardScheme getScheme() { - return new getTopologyConf_resultStandardScheme(); + private static class getTopologyHistory_resultStandardSchemeFactory implements SchemeFactory { + public getTopologyHistory_resultStandardScheme getScheme() { + return new getTopologyHistory_resultStandardScheme(); } } - private static class getTopologyConf_resultStandardScheme extends StandardScheme { + private static class getTopologyHistory_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -44703,23 +47861,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_res } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.success = iprot.readString(); - struct.set_success_isSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 1: // E if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.e = new NotAliveException(); - struct.e.read(iprot); - struct.set_e_isSet(true); + struct.success = new TopologyHistoryInfo(); + struct.success.read(iprot); + struct.set_success_isSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // AZE + case 1: // AZE if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { struct.aze = new AuthorizationException(); struct.aze.read(iprot); @@ -44737,18 +47887,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_res struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyConf_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyHistory_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); if (struct.success != null) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeString(struct.success); - oprot.writeFieldEnd(); - } - if (struct.e != null) { - oprot.writeFieldBegin(E_FIELD_DESC); - struct.e.write(oprot); + struct.success.write(oprot); oprot.writeFieldEnd(); } if (struct.aze != null) { @@ -44762,33 +47907,27 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyConf_re } - private static class getTopologyConf_resultTupleSchemeFactory implements SchemeFactory { - public getTopologyConf_resultTupleScheme getScheme() { - return new getTopologyConf_resultTupleScheme(); + private static class getTopologyHistory_resultTupleSchemeFactory implements SchemeFactory { + public getTopologyHistory_resultTupleScheme getScheme() { + return new getTopologyHistory_resultTupleScheme(); } } - private static class getTopologyConf_resultTupleScheme extends TupleScheme { + private static class getTopologyHistory_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.is_set_success()) { optionals.set(0); } - if (struct.is_set_e()) { - optionals.set(1); - } if (struct.is_set_aze()) { - optionals.set(2); + optionals.set(1); } - oprot.writeBitSet(optionals, 3); + oprot.writeBitSet(optionals, 2); if (struct.is_set_success()) { - oprot.writeString(struct.success); - } - if (struct.is_set_e()) { - struct.e.write(oprot); + struct.success.write(oprot); } if (struct.is_set_aze()) { struct.aze.write(oprot); @@ -44796,19 +47935,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_res } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(3); + BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { - struct.success = iprot.readString(); + struct.success = new TopologyHistoryInfo(); + struct.success.read(iprot); struct.set_success_isSet(true); } if (incoming.get(1)) { - struct.e = new NotAliveException(); - struct.e.read(iprot); - struct.set_e_isSet(true); - } - if (incoming.get(2)) { struct.aze = new AuthorizationException(); struct.aze.read(iprot); struct.set_aze_isSet(true); @@ -44818,22 +47953,22 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_resu } - public static class getTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopology_args"); + public static class getOwnerResourceSummaries_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getOwnerResourceSummaries_args"); - private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getTopology_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getTopology_argsTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getOwnerResourceSummaries_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getOwnerResourceSummaries_argsTupleSchemeFactory()); } - private String id; // required + private String owner; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - ID((short)1, "id"); + OWNER((short)1, "owner"); private static final Map byName = new HashMap(); @@ -44848,8 +47983,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // ID - return ID; + case 1: // OWNER + return OWNER; default: return null; } @@ -44893,70 +48028,70 @@ public String getFieldName() { public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getOwnerResourceSummaries_args.class, metaDataMap); } - public getTopology_args() { + public getOwnerResourceSummaries_args() { } - public getTopology_args( - String id) + public getOwnerResourceSummaries_args( + String owner) { this(); - this.id = id; + this.owner = owner; } /** * Performs a deep copy on other. */ - public getTopology_args(getTopology_args other) { - if (other.is_set_id()) { - this.id = other.id; + public getOwnerResourceSummaries_args(getOwnerResourceSummaries_args other) { + if (other.is_set_owner()) { + this.owner = other.owner; } } - public getTopology_args deepCopy() { - return new getTopology_args(this); + public getOwnerResourceSummaries_args deepCopy() { + return new getOwnerResourceSummaries_args(this); } @Override public void clear() { - this.id = null; + this.owner = null; } - public String get_id() { - return this.id; + public String get_owner() { + return this.owner; } - public void set_id(String id) { - this.id = id; + public void set_owner(String owner) { + this.owner = owner; } - public void unset_id() { - this.id = null; + public void unset_owner() { + this.owner = null; } - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; + /** Returns true if field owner is set (has been assigned a value) and false otherwise */ + public boolean is_set_owner() { + return this.owner != null; } - public void set_id_isSet(boolean value) { + public void set_owner_isSet(boolean value) { if (!value) { - this.id = null; + this.owner = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { - case ID: + case OWNER: if (value == null) { - unset_id(); + unset_owner(); } else { - set_id((String)value); + set_owner((String)value); } break; @@ -44965,8 +48100,8 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { - case ID: - return get_id(); + case OWNER: + return get_owner(); } throw new IllegalStateException(); @@ -44979,8 +48114,8 @@ public boolean isSet(_Fields field) { } switch (field) { - case ID: - return is_set_id(); + case OWNER: + return is_set_owner(); } throw new IllegalStateException(); } @@ -44989,21 +48124,21 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getTopology_args) - return this.equals((getTopology_args)that); + if (that instanceof getOwnerResourceSummaries_args) + return this.equals((getOwnerResourceSummaries_args)that); return false; } - public boolean equals(getTopology_args that) { + public boolean equals(getOwnerResourceSummaries_args that) { if (that == null) return false; - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) + boolean this_present_owner = true && this.is_set_owner(); + boolean that_present_owner = true && that.is_set_owner(); + if (this_present_owner || that_present_owner) { + if (!(this_present_owner && that_present_owner)) return false; - if (!this.id.equals(that.id)) + if (!this.owner.equals(that.owner)) return false; } @@ -45014,28 +48149,28 @@ public boolean equals(getTopology_args that) { public int hashCode() { List list = new ArrayList(); - boolean present_id = true && (is_set_id()); - list.add(present_id); - if (present_id) - list.add(id); + boolean present_owner = true && (is_set_owner()); + list.add(present_owner); + if (present_owner) + list.add(owner); return list.hashCode(); } @Override - public int compareTo(getTopology_args other) { + public int compareTo(getOwnerResourceSummaries_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + lastComparison = Boolean.valueOf(is_set_owner()).compareTo(other.is_set_owner()); if (lastComparison != 0) { return lastComparison; } - if (is_set_id()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (is_set_owner()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner); if (lastComparison != 0) { return lastComparison; } @@ -45057,14 +48192,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getTopology_args("); + StringBuilder sb = new StringBuilder("getOwnerResourceSummaries_args("); boolean first = true; - sb.append("id:"); - if (this.id == null) { + sb.append("owner:"); + if (this.owner == null) { sb.append("null"); } else { - sb.append(this.id); + sb.append(this.owner); } first = false; sb.append(")"); @@ -45092,15 +48227,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getTopology_argsStandardSchemeFactory implements SchemeFactory { - public getTopology_argsStandardScheme getScheme() { - return new getTopology_argsStandardScheme(); + private static class getOwnerResourceSummaries_argsStandardSchemeFactory implements SchemeFactory { + public getOwnerResourceSummaries_argsStandardScheme getScheme() { + return new getOwnerResourceSummaries_argsStandardScheme(); } } - private static class getTopology_argsStandardScheme extends StandardScheme { + private static class getOwnerResourceSummaries_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getOwnerResourceSummaries_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -45110,10 +48245,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_args st break; } switch (schemeField.id) { - case 1: // ID + case 1: // OWNER if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.id = iprot.readString(); - struct.set_id_isSet(true); + struct.owner = iprot.readString(); + struct.set_owner_isSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -45127,13 +48262,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_args st struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getTopology_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getOwnerResourceSummaries_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(struct.id); + if (struct.owner != null) { + oprot.writeFieldBegin(OWNER_FIELD_DESC); + oprot.writeString(struct.owner); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -45142,62 +48277,59 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getTopology_args s } - private static class getTopology_argsTupleSchemeFactory implements SchemeFactory { - public getTopology_argsTupleScheme getScheme() { - return new getTopology_argsTupleScheme(); + private static class getOwnerResourceSummaries_argsTupleSchemeFactory implements SchemeFactory { + public getOwnerResourceSummaries_argsTupleScheme getScheme() { + return new getOwnerResourceSummaries_argsTupleScheme(); } } - private static class getTopology_argsTupleScheme extends TupleScheme { + private static class getOwnerResourceSummaries_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getTopology_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSummaries_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); - if (struct.is_set_id()) { + if (struct.is_set_owner()) { optionals.set(0); } oprot.writeBitSet(optionals, 1); - if (struct.is_set_id()) { - oprot.writeString(struct.id); + if (struct.is_set_owner()) { + oprot.writeString(struct.owner); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getTopology_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSummaries_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - struct.id = iprot.readString(); - struct.set_id_isSet(true); + struct.owner = iprot.readString(); + struct.set_owner_isSet(true); } } } } - public static class getTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopology_result"); + public static class getOwnerResourceSummaries_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getOwnerResourceSummaries_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getTopology_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getTopology_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getOwnerResourceSummaries_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getOwnerResourceSummaries_resultTupleSchemeFactory()); } - private StormTopology success; // required - private NotAliveException e; // required + private List success; // required private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { SUCCESS((short)0, "success"), - E((short)1, "e"), - AZE((short)2, "aze"); + AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -45214,9 +48346,7 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; - case 1: // E - return E; - case 2: // AZE + case 1: // AZE return AZE; default: return null; @@ -45262,60 +48392,72 @@ public String getFieldName() { static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); - tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, OwnerResourceSummary.class)))); tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getOwnerResourceSummaries_result.class, metaDataMap); } - public getTopology_result() { + public getOwnerResourceSummaries_result() { } - public getTopology_result( - StormTopology success, - NotAliveException e, + public getOwnerResourceSummaries_result( + List success, AuthorizationException aze) { this(); this.success = success; - this.e = e; this.aze = aze; } /** * Performs a deep copy on other. */ - public getTopology_result(getTopology_result other) { + public getOwnerResourceSummaries_result(getOwnerResourceSummaries_result other) { if (other.is_set_success()) { - this.success = new StormTopology(other.success); - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); + List __this__success = new ArrayList(other.success.size()); + for (OwnerResourceSummary other_element : other.success) { + __this__success.add(new OwnerResourceSummary(other_element)); + } + this.success = __this__success; } if (other.is_set_aze()) { this.aze = new AuthorizationException(other.aze); } } - public getTopology_result deepCopy() { - return new getTopology_result(this); + public getOwnerResourceSummaries_result deepCopy() { + return new getOwnerResourceSummaries_result(this); } @Override public void clear() { this.success = null; - this.e = null; this.aze = null; } - public StormTopology get_success() { + public int get_success_size() { + return (this.success == null) ? 0 : this.success.size(); + } + + public java.util.Iterator get_success_iterator() { + return (this.success == null) ? null : this.success.iterator(); + } + + public void add_to_success(OwnerResourceSummary elem) { + if (this.success == null) { + this.success = new ArrayList(); + } + this.success.add(elem); + } + + public List get_success() { return this.success; } - public void set_success(StormTopology success) { + public void set_success(List success) { this.success = success; } @@ -45334,29 +48476,6 @@ public void set_success_isSet(boolean value) { } } - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - public AuthorizationException get_aze() { return this.aze; } @@ -45386,15 +48505,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unset_success(); } else { - set_success((StormTopology)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); + set_success((List)value); } break; @@ -45414,9 +48525,6 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return get_success(); - case E: - return get_e(); - case AZE: return get_aze(); @@ -45433,8 +48541,6 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return is_set_success(); - case E: - return is_set_e(); case AZE: return is_set_aze(); } @@ -45445,12 +48551,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getTopology_result) - return this.equals((getTopology_result)that); + if (that instanceof getOwnerResourceSummaries_result) + return this.equals((getOwnerResourceSummaries_result)that); return false; } - public boolean equals(getTopology_result that) { + public boolean equals(getOwnerResourceSummaries_result that) { if (that == null) return false; @@ -45463,15 +48569,6 @@ public boolean equals(getTopology_result that) { return false; } - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - boolean this_present_aze = true && this.is_set_aze(); boolean that_present_aze = true && that.is_set_aze(); if (this_present_aze || that_present_aze) { @@ -45493,11 +48590,6 @@ public int hashCode() { if (present_success) list.add(success); - boolean present_e = true && (is_set_e()); - list.add(present_e); - if (present_e) - list.add(e); - boolean present_aze = true && (is_set_aze()); list.add(present_aze); if (present_aze) @@ -45507,7 +48599,7 @@ public int hashCode() { } @Override - public int compareTo(getTopology_result other) { + public int compareTo(getOwnerResourceSummaries_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -45524,16 +48616,6 @@ public int compareTo(getTopology_result other) { return lastComparison; } } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); if (lastComparison != 0) { return lastComparison; @@ -45561,7 +48643,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getTopology_result("); + StringBuilder sb = new StringBuilder("getOwnerResourceSummaries_result("); boolean first = true; sb.append("success:"); @@ -45572,14 +48654,6 @@ public String toString() { } first = false; if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - if (!first) sb.append(", "); sb.append("aze:"); if (this.aze == null) { sb.append("null"); @@ -45594,9 +48668,6 @@ public String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity - if (success != null) { - success.validate(); - } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -45615,15 +48686,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getTopology_resultStandardSchemeFactory implements SchemeFactory { - public getTopology_resultStandardScheme getScheme() { - return new getTopology_resultStandardScheme(); + private static class getOwnerResourceSummaries_resultStandardSchemeFactory implements SchemeFactory { + public getOwnerResourceSummaries_resultStandardScheme getScheme() { + return new getOwnerResourceSummaries_resultStandardScheme(); } } - private static class getTopology_resultStandardScheme extends StandardScheme { + private static class getOwnerResourceSummaries_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getOwnerResourceSummaries_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -45634,24 +48705,25 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_result } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new StormTopology(); - struct.success.read(iprot); + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list928 = iprot.readListBegin(); + struct.success = new ArrayList(_list928.size); + OwnerResourceSummary _elem929; + for (int _i930 = 0; _i930 < _list928.size; ++_i930) + { + _elem929 = new OwnerResourceSummary(); + _elem929.read(iprot); + struct.success.add(_elem929); + } + iprot.readListEnd(); + } struct.set_success_isSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 1: // E - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.e = new NotAliveException(); - struct.e.read(iprot); - struct.set_e_isSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // AZE + case 1: // AZE if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { struct.aze = new AuthorizationException(); struct.aze.read(iprot); @@ -45669,18 +48741,20 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_result struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getTopology_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getOwnerResourceSummaries_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); if (struct.success != null) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.e != null) { - oprot.writeFieldBegin(E_FIELD_DESC); - struct.e.write(oprot); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); + for (OwnerResourceSummary _iter931 : struct.success) + { + _iter931.write(oprot); + } + oprot.writeListEnd(); + } oprot.writeFieldEnd(); } if (struct.aze != null) { @@ -45694,33 +48768,33 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getTopology_result } - private static class getTopology_resultTupleSchemeFactory implements SchemeFactory { - public getTopology_resultTupleScheme getScheme() { - return new getTopology_resultTupleScheme(); + private static class getOwnerResourceSummaries_resultTupleSchemeFactory implements SchemeFactory { + public getOwnerResourceSummaries_resultTupleScheme getScheme() { + return new getOwnerResourceSummaries_resultTupleScheme(); } } - private static class getTopology_resultTupleScheme extends TupleScheme { + private static class getOwnerResourceSummaries_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getTopology_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSummaries_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.is_set_success()) { optionals.set(0); } - if (struct.is_set_e()) { - optionals.set(1); - } if (struct.is_set_aze()) { - optionals.set(2); + optionals.set(1); } - oprot.writeBitSet(optionals, 3); + oprot.writeBitSet(optionals, 2); if (struct.is_set_success()) { - struct.success.write(oprot); - } - if (struct.is_set_e()) { - struct.e.write(oprot); + { + oprot.writeI32(struct.success.size()); + for (OwnerResourceSummary _iter932 : struct.success) + { + _iter932.write(oprot); + } + } } if (struct.is_set_aze()) { struct.aze.write(oprot); @@ -45728,20 +48802,24 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getTopology_result } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getTopology_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSummaries_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(3); + BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { - struct.success = new StormTopology(); - struct.success.read(iprot); + { + org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list933.size); + OwnerResourceSummary _elem934; + for (int _i935 = 0; _i935 < _list933.size; ++_i935) + { + _elem934 = new OwnerResourceSummary(); + _elem934.read(iprot); + struct.success.add(_elem934); + } + } struct.set_success_isSet(true); } if (incoming.get(1)) { - struct.e = new NotAliveException(); - struct.e.read(iprot); - struct.set_e_isSet(true); - } - if (incoming.get(2)) { struct.aze = new AuthorizationException(); struct.aze.read(iprot); struct.set_aze_isSet(true); @@ -45751,22 +48829,22 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getTopology_result s } - public static class getUserTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserTopology_args"); + public static class getSupervisorAssignments_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getSupervisorAssignments_args"); - private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField NODE_FIELD_DESC = new org.apache.thrift.protocol.TField("node", org.apache.thrift.protocol.TType.STRING, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getUserTopology_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getUserTopology_argsTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getSupervisorAssignments_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getSupervisorAssignments_argsTupleSchemeFactory()); } - private String id; // required + private String node; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - ID((short)1, "id"); + NODE((short)1, "node"); private static final Map byName = new HashMap(); @@ -45781,8 +48859,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // ID - return ID; + case 1: // NODE + return NODE; default: return null; } @@ -45826,70 +48904,70 @@ public String getFieldName() { public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.NODE, new org.apache.thrift.meta_data.FieldMetaData("node", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getSupervisorAssignments_args.class, metaDataMap); } - public getUserTopology_args() { + public getSupervisorAssignments_args() { } - public getUserTopology_args( - String id) + public getSupervisorAssignments_args( + String node) { this(); - this.id = id; + this.node = node; } /** * Performs a deep copy on other. */ - public getUserTopology_args(getUserTopology_args other) { - if (other.is_set_id()) { - this.id = other.id; + public getSupervisorAssignments_args(getSupervisorAssignments_args other) { + if (other.is_set_node()) { + this.node = other.node; } } - public getUserTopology_args deepCopy() { - return new getUserTopology_args(this); + public getSupervisorAssignments_args deepCopy() { + return new getSupervisorAssignments_args(this); } @Override public void clear() { - this.id = null; + this.node = null; } - public String get_id() { - return this.id; + public String get_node() { + return this.node; } - public void set_id(String id) { - this.id = id; + public void set_node(String node) { + this.node = node; } - public void unset_id() { - this.id = null; + public void unset_node() { + this.node = null; } - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; + /** Returns true if field node is set (has been assigned a value) and false otherwise */ + public boolean is_set_node() { + return this.node != null; } - public void set_id_isSet(boolean value) { + public void set_node_isSet(boolean value) { if (!value) { - this.id = null; + this.node = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { - case ID: + case NODE: if (value == null) { - unset_id(); + unset_node(); } else { - set_id((String)value); + set_node((String)value); } break; @@ -45898,8 +48976,8 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { - case ID: - return get_id(); + case NODE: + return get_node(); } throw new IllegalStateException(); @@ -45912,8 +48990,8 @@ public boolean isSet(_Fields field) { } switch (field) { - case ID: - return is_set_id(); + case NODE: + return is_set_node(); } throw new IllegalStateException(); } @@ -45922,21 +49000,21 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getUserTopology_args) - return this.equals((getUserTopology_args)that); + if (that instanceof getSupervisorAssignments_args) + return this.equals((getSupervisorAssignments_args)that); return false; } - public boolean equals(getUserTopology_args that) { + public boolean equals(getSupervisorAssignments_args that) { if (that == null) return false; - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) + boolean this_present_node = true && this.is_set_node(); + boolean that_present_node = true && that.is_set_node(); + if (this_present_node || that_present_node) { + if (!(this_present_node && that_present_node)) return false; - if (!this.id.equals(that.id)) + if (!this.node.equals(that.node)) return false; } @@ -45947,28 +49025,28 @@ public boolean equals(getUserTopology_args that) { public int hashCode() { List list = new ArrayList(); - boolean present_id = true && (is_set_id()); - list.add(present_id); - if (present_id) - list.add(id); + boolean present_node = true && (is_set_node()); + list.add(present_node); + if (present_node) + list.add(node); return list.hashCode(); } @Override - public int compareTo(getUserTopology_args other) { + public int compareTo(getSupervisorAssignments_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + lastComparison = Boolean.valueOf(is_set_node()).compareTo(other.is_set_node()); if (lastComparison != 0) { return lastComparison; } - if (is_set_id()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (is_set_node()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.node, other.node); if (lastComparison != 0) { return lastComparison; } @@ -45990,14 +49068,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getUserTopology_args("); + StringBuilder sb = new StringBuilder("getSupervisorAssignments_args("); boolean first = true; - sb.append("id:"); - if (this.id == null) { + sb.append("node:"); + if (this.node == null) { sb.append("null"); } else { - sb.append(this.id); + sb.append(this.node); } first = false; sb.append(")"); @@ -46025,15 +49103,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getUserTopology_argsStandardSchemeFactory implements SchemeFactory { - public getUserTopology_argsStandardScheme getScheme() { - return new getUserTopology_argsStandardScheme(); + private static class getSupervisorAssignments_argsStandardSchemeFactory implements SchemeFactory { + public getSupervisorAssignments_argsStandardScheme getScheme() { + return new getSupervisorAssignments_argsStandardScheme(); } } - private static class getUserTopology_argsStandardScheme extends StandardScheme { + private static class getSupervisorAssignments_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getSupervisorAssignments_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -46043,10 +49121,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_arg break; } switch (schemeField.id) { - case 1: // ID + case 1: // NODE if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.id = iprot.readString(); - struct.set_id_isSet(true); + struct.node = iprot.readString(); + struct.set_node_isSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -46060,13 +49138,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_arg struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTopology_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getSupervisorAssignments_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(struct.id); + if (struct.node != null) { + oprot.writeFieldBegin(NODE_FIELD_DESC); + oprot.writeString(struct.node); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -46075,62 +49153,59 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTopology_ar } - private static class getUserTopology_argsTupleSchemeFactory implements SchemeFactory { - public getUserTopology_argsTupleScheme getScheme() { - return new getUserTopology_argsTupleScheme(); + private static class getSupervisorAssignments_argsTupleSchemeFactory implements SchemeFactory { + public getSupervisorAssignments_argsTupleScheme getScheme() { + return new getSupervisorAssignments_argsTupleScheme(); } } - private static class getUserTopology_argsTupleScheme extends TupleScheme { + private static class getSupervisorAssignments_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getUserTopology_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getSupervisorAssignments_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); - if (struct.is_set_id()) { + if (struct.is_set_node()) { optionals.set(0); } oprot.writeBitSet(optionals, 1); - if (struct.is_set_id()) { - oprot.writeString(struct.id); + if (struct.is_set_node()) { + oprot.writeString(struct.node); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getUserTopology_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getSupervisorAssignments_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - struct.id = iprot.readString(); - struct.set_id_isSet(true); + struct.node = iprot.readString(); + struct.set_node_isSet(true); } } } } - public static class getUserTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserTopology_result"); + public static class getSupervisorAssignments_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getSupervisorAssignments_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getUserTopology_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getUserTopology_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getSupervisorAssignments_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getSupervisorAssignments_resultTupleSchemeFactory()); } - private StormTopology success; // required - private NotAliveException e; // required + private SupervisorAssignments success; // required private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { SUCCESS((short)0, "success"), - E((short)1, "e"), - AZE((short)2, "aze"); + AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -46147,9 +49222,7 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; - case 1: // E - return E; - case 2: // AZE + case 1: // AZE return AZE; default: return null; @@ -46195,60 +49268,52 @@ public String getFieldName() { static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); - tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorAssignments.class))); tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getSupervisorAssignments_result.class, metaDataMap); } - public getUserTopology_result() { + public getSupervisorAssignments_result() { } - public getUserTopology_result( - StormTopology success, - NotAliveException e, + public getSupervisorAssignments_result( + SupervisorAssignments success, AuthorizationException aze) { this(); this.success = success; - this.e = e; this.aze = aze; } /** * Performs a deep copy on other. */ - public getUserTopology_result(getUserTopology_result other) { + public getSupervisorAssignments_result(getSupervisorAssignments_result other) { if (other.is_set_success()) { - this.success = new StormTopology(other.success); - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); + this.success = new SupervisorAssignments(other.success); } if (other.is_set_aze()) { this.aze = new AuthorizationException(other.aze); } } - public getUserTopology_result deepCopy() { - return new getUserTopology_result(this); + public getSupervisorAssignments_result deepCopy() { + return new getSupervisorAssignments_result(this); } @Override public void clear() { this.success = null; - this.e = null; this.aze = null; } - public StormTopology get_success() { + public SupervisorAssignments get_success() { return this.success; } - public void set_success(StormTopology success) { + public void set_success(SupervisorAssignments success) { this.success = success; } @@ -46267,29 +49332,6 @@ public void set_success_isSet(boolean value) { } } - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - public AuthorizationException get_aze() { return this.aze; } @@ -46319,15 +49361,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unset_success(); } else { - set_success((StormTopology)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); + set_success((SupervisorAssignments)value); } break; @@ -46347,9 +49381,6 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return get_success(); - case E: - return get_e(); - case AZE: return get_aze(); @@ -46366,8 +49397,6 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return is_set_success(); - case E: - return is_set_e(); case AZE: return is_set_aze(); } @@ -46378,12 +49407,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getUserTopology_result) - return this.equals((getUserTopology_result)that); + if (that instanceof getSupervisorAssignments_result) + return this.equals((getSupervisorAssignments_result)that); return false; } - public boolean equals(getUserTopology_result that) { + public boolean equals(getSupervisorAssignments_result that) { if (that == null) return false; @@ -46396,15 +49425,6 @@ public boolean equals(getUserTopology_result that) { return false; } - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - boolean this_present_aze = true && this.is_set_aze(); boolean that_present_aze = true && that.is_set_aze(); if (this_present_aze || that_present_aze) { @@ -46426,11 +49446,6 @@ public int hashCode() { if (present_success) list.add(success); - boolean present_e = true && (is_set_e()); - list.add(present_e); - if (present_e) - list.add(e); - boolean present_aze = true && (is_set_aze()); list.add(present_aze); if (present_aze) @@ -46440,7 +49455,7 @@ public int hashCode() { } @Override - public int compareTo(getUserTopology_result other) { + public int compareTo(getSupervisorAssignments_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -46457,16 +49472,6 @@ public int compareTo(getUserTopology_result other) { return lastComparison; } } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); if (lastComparison != 0) { return lastComparison; @@ -46494,7 +49499,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getUserTopology_result("); + StringBuilder sb = new StringBuilder("getSupervisorAssignments_result("); boolean first = true; sb.append("success:"); @@ -46505,14 +49510,6 @@ public String toString() { } first = false; if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - if (!first) sb.append(", "); sb.append("aze:"); if (this.aze == null) { sb.append("null"); @@ -46548,15 +49545,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getUserTopology_resultStandardSchemeFactory implements SchemeFactory { - public getUserTopology_resultStandardScheme getScheme() { - return new getUserTopology_resultStandardScheme(); + private static class getSupervisorAssignments_resultStandardSchemeFactory implements SchemeFactory { + public getSupervisorAssignments_resultStandardScheme getScheme() { + return new getSupervisorAssignments_resultStandardScheme(); } } - private static class getUserTopology_resultStandardScheme extends StandardScheme { + private static class getSupervisorAssignments_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getSupervisorAssignments_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -46568,23 +49565,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_res switch (schemeField.id) { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new StormTopology(); + struct.success = new SupervisorAssignments(); struct.success.read(iprot); struct.set_success_isSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 1: // E - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.e = new NotAliveException(); - struct.e.read(iprot); - struct.set_e_isSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // AZE + case 1: // AZE if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { struct.aze = new AuthorizationException(); struct.aze.read(iprot); @@ -46602,7 +49590,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_res struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTopology_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getSupervisorAssignments_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -46611,11 +49599,6 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTopology_re struct.success.write(oprot); oprot.writeFieldEnd(); } - if (struct.e != null) { - oprot.writeFieldBegin(E_FIELD_DESC); - struct.e.write(oprot); - oprot.writeFieldEnd(); - } if (struct.aze != null) { oprot.writeFieldBegin(AZE_FIELD_DESC); struct.aze.write(oprot); @@ -46627,54 +49610,43 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTopology_re } - private static class getUserTopology_resultTupleSchemeFactory implements SchemeFactory { - public getUserTopology_resultTupleScheme getScheme() { - return new getUserTopology_resultTupleScheme(); + private static class getSupervisorAssignments_resultTupleSchemeFactory implements SchemeFactory { + public getSupervisorAssignments_resultTupleScheme getScheme() { + return new getSupervisorAssignments_resultTupleScheme(); } } - private static class getUserTopology_resultTupleScheme extends TupleScheme { + private static class getSupervisorAssignments_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getUserTopology_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getSupervisorAssignments_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.is_set_success()) { optionals.set(0); } - if (struct.is_set_e()) { - optionals.set(1); - } if (struct.is_set_aze()) { - optionals.set(2); + optionals.set(1); } - oprot.writeBitSet(optionals, 3); + oprot.writeBitSet(optionals, 2); if (struct.is_set_success()) { struct.success.write(oprot); } - if (struct.is_set_e()) { - struct.e.write(oprot); - } if (struct.is_set_aze()) { struct.aze.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getUserTopology_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getSupervisorAssignments_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(3); + BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { - struct.success = new StormTopology(); + struct.success = new SupervisorAssignments(); struct.success.read(iprot); struct.set_success_isSet(true); } if (incoming.get(1)) { - struct.e = new NotAliveException(); - struct.e.read(iprot); - struct.set_e_isSet(true); - } - if (incoming.get(2)) { struct.aze = new AuthorizationException(); struct.aze.read(iprot); struct.set_aze_isSet(true); @@ -46684,22 +49656,22 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getUserTopology_resu } - public static class getTopologyHistory_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyHistory_args"); + public static class sendSupervisorWorkerHeartbeats_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("sendSupervisorWorkerHeartbeats_args"); - private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField HEARTBEATS_FIELD_DESC = new org.apache.thrift.protocol.TField("heartbeats", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getTopologyHistory_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getTopologyHistory_argsTupleSchemeFactory()); + schemes.put(StandardScheme.class, new sendSupervisorWorkerHeartbeats_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new sendSupervisorWorkerHeartbeats_argsTupleSchemeFactory()); } - private String user; // required + private SupervisorWorkerHeartbeats heartbeats; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - USER((short)1, "user"); + HEARTBEATS((short)1, "heartbeats"); private static final Map byName = new HashMap(); @@ -46714,8 +49686,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // USER - return USER; + case 1: // HEARTBEATS + return HEARTBEATS; default: return null; } @@ -46759,70 +49731,70 @@ public String getFieldName() { public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.HEARTBEATS, new org.apache.thrift.meta_data.FieldMetaData("heartbeats", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorWorkerHeartbeats.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyHistory_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(sendSupervisorWorkerHeartbeats_args.class, metaDataMap); } - public getTopologyHistory_args() { + public sendSupervisorWorkerHeartbeats_args() { } - public getTopologyHistory_args( - String user) + public sendSupervisorWorkerHeartbeats_args( + SupervisorWorkerHeartbeats heartbeats) { this(); - this.user = user; + this.heartbeats = heartbeats; } /** * Performs a deep copy on other. */ - public getTopologyHistory_args(getTopologyHistory_args other) { - if (other.is_set_user()) { - this.user = other.user; + public sendSupervisorWorkerHeartbeats_args(sendSupervisorWorkerHeartbeats_args other) { + if (other.is_set_heartbeats()) { + this.heartbeats = new SupervisorWorkerHeartbeats(other.heartbeats); } } - public getTopologyHistory_args deepCopy() { - return new getTopologyHistory_args(this); + public sendSupervisorWorkerHeartbeats_args deepCopy() { + return new sendSupervisorWorkerHeartbeats_args(this); } @Override public void clear() { - this.user = null; + this.heartbeats = null; } - public String get_user() { - return this.user; + public SupervisorWorkerHeartbeats get_heartbeats() { + return this.heartbeats; } - public void set_user(String user) { - this.user = user; + public void set_heartbeats(SupervisorWorkerHeartbeats heartbeats) { + this.heartbeats = heartbeats; } - public void unset_user() { - this.user = null; + public void unset_heartbeats() { + this.heartbeats = null; } - /** Returns true if field user is set (has been assigned a value) and false otherwise */ - public boolean is_set_user() { - return this.user != null; + /** Returns true if field heartbeats is set (has been assigned a value) and false otherwise */ + public boolean is_set_heartbeats() { + return this.heartbeats != null; } - public void set_user_isSet(boolean value) { + public void set_heartbeats_isSet(boolean value) { if (!value) { - this.user = null; + this.heartbeats = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { - case USER: + case HEARTBEATS: if (value == null) { - unset_user(); + unset_heartbeats(); } else { - set_user((String)value); + set_heartbeats((SupervisorWorkerHeartbeats)value); } break; @@ -46831,8 +49803,8 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { - case USER: - return get_user(); + case HEARTBEATS: + return get_heartbeats(); } throw new IllegalStateException(); @@ -46845,8 +49817,8 @@ public boolean isSet(_Fields field) { } switch (field) { - case USER: - return is_set_user(); + case HEARTBEATS: + return is_set_heartbeats(); } throw new IllegalStateException(); } @@ -46855,21 +49827,21 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getTopologyHistory_args) - return this.equals((getTopologyHistory_args)that); + if (that instanceof sendSupervisorWorkerHeartbeats_args) + return this.equals((sendSupervisorWorkerHeartbeats_args)that); return false; } - public boolean equals(getTopologyHistory_args that) { + public boolean equals(sendSupervisorWorkerHeartbeats_args that) { if (that == null) return false; - boolean this_present_user = true && this.is_set_user(); - boolean that_present_user = true && that.is_set_user(); - if (this_present_user || that_present_user) { - if (!(this_present_user && that_present_user)) + boolean this_present_heartbeats = true && this.is_set_heartbeats(); + boolean that_present_heartbeats = true && that.is_set_heartbeats(); + if (this_present_heartbeats || that_present_heartbeats) { + if (!(this_present_heartbeats && that_present_heartbeats)) return false; - if (!this.user.equals(that.user)) + if (!this.heartbeats.equals(that.heartbeats)) return false; } @@ -46880,28 +49852,28 @@ public boolean equals(getTopologyHistory_args that) { public int hashCode() { List list = new ArrayList(); - boolean present_user = true && (is_set_user()); - list.add(present_user); - if (present_user) - list.add(user); + boolean present_heartbeats = true && (is_set_heartbeats()); + list.add(present_heartbeats); + if (present_heartbeats) + list.add(heartbeats); return list.hashCode(); } @Override - public int compareTo(getTopologyHistory_args other) { + public int compareTo(sendSupervisorWorkerHeartbeats_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = Boolean.valueOf(is_set_user()).compareTo(other.is_set_user()); + lastComparison = Boolean.valueOf(is_set_heartbeats()).compareTo(other.is_set_heartbeats()); if (lastComparison != 0) { return lastComparison; } - if (is_set_user()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, other.user); + if (is_set_heartbeats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.heartbeats, other.heartbeats); if (lastComparison != 0) { return lastComparison; } @@ -46923,14 +49895,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getTopologyHistory_args("); + StringBuilder sb = new StringBuilder("sendSupervisorWorkerHeartbeats_args("); boolean first = true; - sb.append("user:"); - if (this.user == null) { + sb.append("heartbeats:"); + if (this.heartbeats == null) { sb.append("null"); } else { - sb.append(this.user); + sb.append(this.heartbeats); } first = false; sb.append(")"); @@ -46940,6 +49912,9 @@ public String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity + if (heartbeats != null) { + heartbeats.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -46958,15 +49933,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getTopologyHistory_argsStandardSchemeFactory implements SchemeFactory { - public getTopologyHistory_argsStandardScheme getScheme() { - return new getTopologyHistory_argsStandardScheme(); + private static class sendSupervisorWorkerHeartbeats_argsStandardSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeats_argsStandardScheme getScheme() { + return new sendSupervisorWorkerHeartbeats_argsStandardScheme(); } } - private static class getTopologyHistory_argsStandardScheme extends StandardScheme { + private static class sendSupervisorWorkerHeartbeats_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, sendSupervisorWorkerHeartbeats_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -46976,10 +49951,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_ break; } switch (schemeField.id) { - case 1: // USER - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.user = iprot.readString(); - struct.set_user_isSet(true); + case 1: // HEARTBEATS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.heartbeats = new SupervisorWorkerHeartbeats(); + struct.heartbeats.read(iprot); + struct.set_heartbeats_isSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -46993,13 +49969,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_ struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyHistory_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, sendSupervisorWorkerHeartbeats_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.user != null) { - oprot.writeFieldBegin(USER_FIELD_DESC); - oprot.writeString(struct.user); + if (struct.heartbeats != null) { + oprot.writeFieldBegin(HEARTBEATS_FIELD_DESC); + struct.heartbeats.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -47008,58 +49984,56 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyHistory } - private static class getTopologyHistory_argsTupleSchemeFactory implements SchemeFactory { - public getTopologyHistory_argsTupleScheme getScheme() { - return new getTopologyHistory_argsTupleScheme(); + private static class sendSupervisorWorkerHeartbeats_argsTupleSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeats_argsTupleScheme getScheme() { + return new sendSupervisorWorkerHeartbeats_argsTupleScheme(); } } - private static class getTopologyHistory_argsTupleScheme extends TupleScheme { + private static class sendSupervisorWorkerHeartbeats_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeats_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); - if (struct.is_set_user()) { + if (struct.is_set_heartbeats()) { optionals.set(0); } oprot.writeBitSet(optionals, 1); - if (struct.is_set_user()) { - oprot.writeString(struct.user); + if (struct.is_set_heartbeats()) { + struct.heartbeats.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeats_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - struct.user = iprot.readString(); - struct.set_user_isSet(true); + struct.heartbeats = new SupervisorWorkerHeartbeats(); + struct.heartbeats.read(iprot); + struct.set_heartbeats_isSet(true); } } } } - public static class getTopologyHistory_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyHistory_result"); + public static class sendSupervisorWorkerHeartbeats_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("sendSupervisorWorkerHeartbeats_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getTopologyHistory_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getTopologyHistory_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new sendSupervisorWorkerHeartbeats_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new sendSupervisorWorkerHeartbeats_resultTupleSchemeFactory()); } - private TopologyHistoryInfo success; // required private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"), AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -47075,8 +50049,6 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; case 1: // AZE return AZE; default: @@ -47122,71 +50094,40 @@ public String getFieldName() { public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyHistoryInfo.class))); tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyHistory_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(sendSupervisorWorkerHeartbeats_result.class, metaDataMap); } - public getTopologyHistory_result() { + public sendSupervisorWorkerHeartbeats_result() { } - public getTopologyHistory_result( - TopologyHistoryInfo success, + public sendSupervisorWorkerHeartbeats_result( AuthorizationException aze) { this(); - this.success = success; this.aze = aze; } /** * Performs a deep copy on other. */ - public getTopologyHistory_result(getTopologyHistory_result other) { - if (other.is_set_success()) { - this.success = new TopologyHistoryInfo(other.success); - } + public sendSupervisorWorkerHeartbeats_result(sendSupervisorWorkerHeartbeats_result other) { if (other.is_set_aze()) { this.aze = new AuthorizationException(other.aze); } } - public getTopologyHistory_result deepCopy() { - return new getTopologyHistory_result(this); + public sendSupervisorWorkerHeartbeats_result deepCopy() { + return new sendSupervisorWorkerHeartbeats_result(this); } @Override public void clear() { - this.success = null; this.aze = null; } - public TopologyHistoryInfo get_success() { - return this.success; - } - - public void set_success(TopologyHistoryInfo success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - public AuthorizationException get_aze() { return this.aze; } @@ -47212,14 +50153,6 @@ public void set_aze_isSet(boolean value) { public void setFieldValue(_Fields field, Object value) { switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((TopologyHistoryInfo)value); - } - break; - case AZE: if (value == null) { unset_aze(); @@ -47233,9 +50166,6 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { - case SUCCESS: - return get_success(); - case AZE: return get_aze(); @@ -47250,8 +50180,6 @@ public boolean isSet(_Fields field) { } switch (field) { - case SUCCESS: - return is_set_success(); case AZE: return is_set_aze(); } @@ -47262,24 +50190,15 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getTopologyHistory_result) - return this.equals((getTopologyHistory_result)that); + if (that instanceof sendSupervisorWorkerHeartbeats_result) + return this.equals((sendSupervisorWorkerHeartbeats_result)that); return false; } - public boolean equals(getTopologyHistory_result that) { + public boolean equals(sendSupervisorWorkerHeartbeats_result that) { if (that == null) return false; - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - boolean this_present_aze = true && this.is_set_aze(); boolean that_present_aze = true && that.is_set_aze(); if (this_present_aze || that_present_aze) { @@ -47296,11 +50215,6 @@ public boolean equals(getTopologyHistory_result that) { public int hashCode() { List list = new ArrayList(); - boolean present_success = true && (is_set_success()); - list.add(present_success); - if (present_success) - list.add(success); - boolean present_aze = true && (is_set_aze()); list.add(present_aze); if (present_aze) @@ -47310,23 +50224,13 @@ public int hashCode() { } @Override - public int compareTo(getTopologyHistory_result other) { + public int compareTo(sendSupervisorWorkerHeartbeats_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); if (lastComparison != 0) { return lastComparison; @@ -47354,17 +50258,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getTopologyHistory_result("); + StringBuilder sb = new StringBuilder("sendSupervisorWorkerHeartbeats_result("); boolean first = true; - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); sb.append("aze:"); if (this.aze == null) { sb.append("null"); @@ -47379,9 +50275,6 @@ public String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity - if (success != null) { - success.validate(); - } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -47400,15 +50293,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getTopologyHistory_resultStandardSchemeFactory implements SchemeFactory { - public getTopologyHistory_resultStandardScheme getScheme() { - return new getTopologyHistory_resultStandardScheme(); + private static class sendSupervisorWorkerHeartbeats_resultStandardSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeats_resultStandardScheme getScheme() { + return new sendSupervisorWorkerHeartbeats_resultStandardScheme(); } } - private static class getTopologyHistory_resultStandardScheme extends StandardScheme { + private static class sendSupervisorWorkerHeartbeats_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, sendSupervisorWorkerHeartbeats_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -47418,15 +50311,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_ break; } switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TopologyHistoryInfo(); - struct.success.read(iprot); - struct.set_success_isSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; case 1: // AZE if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { struct.aze = new AuthorizationException(); @@ -47445,15 +50329,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_ struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyHistory_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, sendSupervisorWorkerHeartbeats_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } if (struct.aze != null) { oprot.writeFieldBegin(AZE_FIELD_DESC); struct.aze.write(oprot); @@ -47465,43 +50344,32 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyHistory } - private static class getTopologyHistory_resultTupleSchemeFactory implements SchemeFactory { - public getTopologyHistory_resultTupleScheme getScheme() { - return new getTopologyHistory_resultTupleScheme(); + private static class sendSupervisorWorkerHeartbeats_resultTupleSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeats_resultTupleScheme getScheme() { + return new sendSupervisorWorkerHeartbeats_resultTupleScheme(); } } - private static class getTopologyHistory_resultTupleScheme extends TupleScheme { + private static class sendSupervisorWorkerHeartbeats_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeats_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); - if (struct.is_set_success()) { - optionals.set(0); - } if (struct.is_set_aze()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.is_set_success()) { - struct.success.write(oprot); + optionals.set(0); } + oprot.writeBitSet(optionals, 1); if (struct.is_set_aze()) { struct.aze.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeats_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(2); + BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - struct.success = new TopologyHistoryInfo(); - struct.success.read(iprot); - struct.set_success_isSet(true); - } - if (incoming.get(1)) { struct.aze = new AuthorizationException(); struct.aze.read(iprot); struct.set_aze_isSet(true); @@ -47511,22 +50379,22 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_r } - public static class getOwnerResourceSummaries_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getOwnerResourceSummaries_args"); + public static class sendSupervisorWorkerHeartbeat_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("sendSupervisorWorkerHeartbeat_args"); - private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField HEATBEAT_FIELD_DESC = new org.apache.thrift.protocol.TField("heatbeat", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getOwnerResourceSummaries_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getOwnerResourceSummaries_argsTupleSchemeFactory()); + schemes.put(StandardScheme.class, new sendSupervisorWorkerHeartbeat_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new sendSupervisorWorkerHeartbeat_argsTupleSchemeFactory()); } - private String owner; // required + private SupervisorWorkerHeartbeat heatbeat; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OWNER((short)1, "owner"); + HEATBEAT((short)1, "heatbeat"); private static final Map byName = new HashMap(); @@ -47541,8 +50409,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // OWNER - return OWNER; + case 1: // HEATBEAT + return HEATBEAT; default: return null; } @@ -47586,70 +50454,70 @@ public String getFieldName() { public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.HEATBEAT, new org.apache.thrift.meta_data.FieldMetaData("heatbeat", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorWorkerHeartbeat.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getOwnerResourceSummaries_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(sendSupervisorWorkerHeartbeat_args.class, metaDataMap); } - public getOwnerResourceSummaries_args() { + public sendSupervisorWorkerHeartbeat_args() { } - public getOwnerResourceSummaries_args( - String owner) + public sendSupervisorWorkerHeartbeat_args( + SupervisorWorkerHeartbeat heatbeat) { this(); - this.owner = owner; + this.heatbeat = heatbeat; } /** * Performs a deep copy on other. */ - public getOwnerResourceSummaries_args(getOwnerResourceSummaries_args other) { - if (other.is_set_owner()) { - this.owner = other.owner; + public sendSupervisorWorkerHeartbeat_args(sendSupervisorWorkerHeartbeat_args other) { + if (other.is_set_heatbeat()) { + this.heatbeat = new SupervisorWorkerHeartbeat(other.heatbeat); } } - public getOwnerResourceSummaries_args deepCopy() { - return new getOwnerResourceSummaries_args(this); + public sendSupervisorWorkerHeartbeat_args deepCopy() { + return new sendSupervisorWorkerHeartbeat_args(this); } @Override public void clear() { - this.owner = null; + this.heatbeat = null; } - public String get_owner() { - return this.owner; + public SupervisorWorkerHeartbeat get_heatbeat() { + return this.heatbeat; } - public void set_owner(String owner) { - this.owner = owner; + public void set_heatbeat(SupervisorWorkerHeartbeat heatbeat) { + this.heatbeat = heatbeat; } - public void unset_owner() { - this.owner = null; + public void unset_heatbeat() { + this.heatbeat = null; } - /** Returns true if field owner is set (has been assigned a value) and false otherwise */ - public boolean is_set_owner() { - return this.owner != null; + /** Returns true if field heatbeat is set (has been assigned a value) and false otherwise */ + public boolean is_set_heatbeat() { + return this.heatbeat != null; } - public void set_owner_isSet(boolean value) { + public void set_heatbeat_isSet(boolean value) { if (!value) { - this.owner = null; + this.heatbeat = null; } } public void setFieldValue(_Fields field, Object value) { switch (field) { - case OWNER: + case HEATBEAT: if (value == null) { - unset_owner(); + unset_heatbeat(); } else { - set_owner((String)value); + set_heatbeat((SupervisorWorkerHeartbeat)value); } break; @@ -47658,8 +50526,8 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { - case OWNER: - return get_owner(); + case HEATBEAT: + return get_heatbeat(); } throw new IllegalStateException(); @@ -47672,8 +50540,8 @@ public boolean isSet(_Fields field) { } switch (field) { - case OWNER: - return is_set_owner(); + case HEATBEAT: + return is_set_heatbeat(); } throw new IllegalStateException(); } @@ -47682,21 +50550,21 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getOwnerResourceSummaries_args) - return this.equals((getOwnerResourceSummaries_args)that); + if (that instanceof sendSupervisorWorkerHeartbeat_args) + return this.equals((sendSupervisorWorkerHeartbeat_args)that); return false; } - public boolean equals(getOwnerResourceSummaries_args that) { + public boolean equals(sendSupervisorWorkerHeartbeat_args that) { if (that == null) return false; - boolean this_present_owner = true && this.is_set_owner(); - boolean that_present_owner = true && that.is_set_owner(); - if (this_present_owner || that_present_owner) { - if (!(this_present_owner && that_present_owner)) + boolean this_present_heatbeat = true && this.is_set_heatbeat(); + boolean that_present_heatbeat = true && that.is_set_heatbeat(); + if (this_present_heatbeat || that_present_heatbeat) { + if (!(this_present_heatbeat && that_present_heatbeat)) return false; - if (!this.owner.equals(that.owner)) + if (!this.heatbeat.equals(that.heatbeat)) return false; } @@ -47707,28 +50575,28 @@ public boolean equals(getOwnerResourceSummaries_args that) { public int hashCode() { List list = new ArrayList(); - boolean present_owner = true && (is_set_owner()); - list.add(present_owner); - if (present_owner) - list.add(owner); + boolean present_heatbeat = true && (is_set_heatbeat()); + list.add(present_heatbeat); + if (present_heatbeat) + list.add(heatbeat); return list.hashCode(); } @Override - public int compareTo(getOwnerResourceSummaries_args other) { + public int compareTo(sendSupervisorWorkerHeartbeat_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = Boolean.valueOf(is_set_owner()).compareTo(other.is_set_owner()); + lastComparison = Boolean.valueOf(is_set_heatbeat()).compareTo(other.is_set_heatbeat()); if (lastComparison != 0) { return lastComparison; } - if (is_set_owner()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner); + if (is_set_heatbeat()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.heatbeat, other.heatbeat); if (lastComparison != 0) { return lastComparison; } @@ -47750,14 +50618,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getOwnerResourceSummaries_args("); + StringBuilder sb = new StringBuilder("sendSupervisorWorkerHeartbeat_args("); boolean first = true; - sb.append("owner:"); - if (this.owner == null) { + sb.append("heatbeat:"); + if (this.heatbeat == null) { sb.append("null"); } else { - sb.append(this.owner); + sb.append(this.heatbeat); } first = false; sb.append(")"); @@ -47767,6 +50635,9 @@ public String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity + if (heatbeat != null) { + heatbeat.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -47785,15 +50656,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getOwnerResourceSummaries_argsStandardSchemeFactory implements SchemeFactory { - public getOwnerResourceSummaries_argsStandardScheme getScheme() { - return new getOwnerResourceSummaries_argsStandardScheme(); + private static class sendSupervisorWorkerHeartbeat_argsStandardSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeat_argsStandardScheme getScheme() { + return new sendSupervisorWorkerHeartbeat_argsStandardScheme(); } } - private static class getOwnerResourceSummaries_argsStandardScheme extends StandardScheme { + private static class sendSupervisorWorkerHeartbeat_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getOwnerResourceSummaries_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, sendSupervisorWorkerHeartbeat_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -47803,10 +50674,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getOwnerResourceSum break; } switch (schemeField.id) { - case 1: // OWNER - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.owner = iprot.readString(); - struct.set_owner_isSet(true); + case 1: // HEATBEAT + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.heatbeat = new SupervisorWorkerHeartbeat(); + struct.heatbeat.read(iprot); + struct.set_heatbeat_isSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -47820,13 +50692,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getOwnerResourceSum struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getOwnerResourceSummaries_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, sendSupervisorWorkerHeartbeat_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.owner != null) { - oprot.writeFieldBegin(OWNER_FIELD_DESC); - oprot.writeString(struct.owner); + if (struct.heatbeat != null) { + oprot.writeFieldBegin(HEATBEAT_FIELD_DESC); + struct.heatbeat.write(oprot); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -47835,58 +50707,56 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getOwnerResourceSu } - private static class getOwnerResourceSummaries_argsTupleSchemeFactory implements SchemeFactory { - public getOwnerResourceSummaries_argsTupleScheme getScheme() { - return new getOwnerResourceSummaries_argsTupleScheme(); + private static class sendSupervisorWorkerHeartbeat_argsTupleSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeat_argsTupleScheme getScheme() { + return new sendSupervisorWorkerHeartbeat_argsTupleScheme(); } } - private static class getOwnerResourceSummaries_argsTupleScheme extends TupleScheme { + private static class sendSupervisorWorkerHeartbeat_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSummaries_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeat_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); - if (struct.is_set_owner()) { + if (struct.is_set_heatbeat()) { optionals.set(0); } oprot.writeBitSet(optionals, 1); - if (struct.is_set_owner()) { - oprot.writeString(struct.owner); + if (struct.is_set_heatbeat()) { + struct.heatbeat.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSummaries_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeat_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - struct.owner = iprot.readString(); - struct.set_owner_isSet(true); + struct.heatbeat = new SupervisorWorkerHeartbeat(); + struct.heatbeat.read(iprot); + struct.set_heatbeat_isSet(true); } } } } - public static class getOwnerResourceSummaries_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getOwnerResourceSummaries_result"); + public static class sendSupervisorWorkerHeartbeat_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("sendSupervisorWorkerHeartbeat_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getOwnerResourceSummaries_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getOwnerResourceSummaries_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new sendSupervisorWorkerHeartbeat_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new sendSupervisorWorkerHeartbeat_resultTupleSchemeFactory()); } - private List success; // required private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"), AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -47902,8 +50772,6 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; case 1: // AZE return AZE; default: @@ -47949,91 +50817,40 @@ public String getFieldName() { public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, OwnerResourceSummary.class)))); tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getOwnerResourceSummaries_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(sendSupervisorWorkerHeartbeat_result.class, metaDataMap); } - public getOwnerResourceSummaries_result() { + public sendSupervisorWorkerHeartbeat_result() { } - public getOwnerResourceSummaries_result( - List success, + public sendSupervisorWorkerHeartbeat_result( AuthorizationException aze) { this(); - this.success = success; this.aze = aze; } /** * Performs a deep copy on other. */ - public getOwnerResourceSummaries_result(getOwnerResourceSummaries_result other) { - if (other.is_set_success()) { - List __this__success = new ArrayList(other.success.size()); - for (OwnerResourceSummary other_element : other.success) { - __this__success.add(new OwnerResourceSummary(other_element)); - } - this.success = __this__success; - } + public sendSupervisorWorkerHeartbeat_result(sendSupervisorWorkerHeartbeat_result other) { if (other.is_set_aze()) { this.aze = new AuthorizationException(other.aze); } } - public getOwnerResourceSummaries_result deepCopy() { - return new getOwnerResourceSummaries_result(this); + public sendSupervisorWorkerHeartbeat_result deepCopy() { + return new sendSupervisorWorkerHeartbeat_result(this); } @Override public void clear() { - this.success = null; this.aze = null; } - public int get_success_size() { - return (this.success == null) ? 0 : this.success.size(); - } - - public java.util.Iterator get_success_iterator() { - return (this.success == null) ? null : this.success.iterator(); - } - - public void add_to_success(OwnerResourceSummary elem) { - if (this.success == null) { - this.success = new ArrayList(); - } - this.success.add(elem); - } - - public List get_success() { - return this.success; - } - - public void set_success(List success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - public AuthorizationException get_aze() { return this.aze; } @@ -48059,14 +50876,6 @@ public void set_aze_isSet(boolean value) { public void setFieldValue(_Fields field, Object value) { switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((List)value); - } - break; - case AZE: if (value == null) { unset_aze(); @@ -48080,9 +50889,6 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { - case SUCCESS: - return get_success(); - case AZE: return get_aze(); @@ -48097,8 +50903,6 @@ public boolean isSet(_Fields field) { } switch (field) { - case SUCCESS: - return is_set_success(); case AZE: return is_set_aze(); } @@ -48109,24 +50913,15 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getOwnerResourceSummaries_result) - return this.equals((getOwnerResourceSummaries_result)that); + if (that instanceof sendSupervisorWorkerHeartbeat_result) + return this.equals((sendSupervisorWorkerHeartbeat_result)that); return false; } - public boolean equals(getOwnerResourceSummaries_result that) { + public boolean equals(sendSupervisorWorkerHeartbeat_result that) { if (that == null) return false; - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - boolean this_present_aze = true && this.is_set_aze(); boolean that_present_aze = true && that.is_set_aze(); if (this_present_aze || that_present_aze) { @@ -48143,11 +50938,6 @@ public boolean equals(getOwnerResourceSummaries_result that) { public int hashCode() { List list = new ArrayList(); - boolean present_success = true && (is_set_success()); - list.add(present_success); - if (present_success) - list.add(success); - boolean present_aze = true && (is_set_aze()); list.add(present_aze); if (present_aze) @@ -48157,23 +50947,13 @@ public int hashCode() { } @Override - public int compareTo(getOwnerResourceSummaries_result other) { + public int compareTo(sendSupervisorWorkerHeartbeat_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); if (lastComparison != 0) { return lastComparison; @@ -48201,17 +50981,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getOwnerResourceSummaries_result("); + StringBuilder sb = new StringBuilder("sendSupervisorWorkerHeartbeat_result("); boolean first = true; - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); sb.append("aze:"); if (this.aze == null) { sb.append("null"); @@ -48244,15 +51016,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getOwnerResourceSummaries_resultStandardSchemeFactory implements SchemeFactory { - public getOwnerResourceSummaries_resultStandardScheme getScheme() { - return new getOwnerResourceSummaries_resultStandardScheme(); + private static class sendSupervisorWorkerHeartbeat_resultStandardSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeat_resultStandardScheme getScheme() { + return new sendSupervisorWorkerHeartbeat_resultStandardScheme(); } } - private static class getOwnerResourceSummaries_resultStandardScheme extends StandardScheme { + private static class sendSupervisorWorkerHeartbeat_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getOwnerResourceSummaries_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, sendSupervisorWorkerHeartbeat_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -48262,25 +51034,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getOwnerResourceSum break; } switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list902 = iprot.readListBegin(); - struct.success = new ArrayList(_list902.size); - OwnerResourceSummary _elem903; - for (int _i904 = 0; _i904 < _list902.size; ++_i904) - { - _elem903 = new OwnerResourceSummary(); - _elem903.read(iprot); - struct.success.add(_elem903); - } - iprot.readListEnd(); - } - struct.set_success_isSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; case 1: // AZE if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { struct.aze = new AuthorizationException(); @@ -48299,22 +51052,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getOwnerResourceSum struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getOwnerResourceSummaries_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, sendSupervisorWorkerHeartbeat_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (OwnerResourceSummary _iter905 : struct.success) - { - _iter905.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } if (struct.aze != null) { oprot.writeFieldBegin(AZE_FIELD_DESC); struct.aze.write(oprot); @@ -48326,58 +51067,32 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getOwnerResourceSu } - private static class getOwnerResourceSummaries_resultTupleSchemeFactory implements SchemeFactory { - public getOwnerResourceSummaries_resultTupleScheme getScheme() { - return new getOwnerResourceSummaries_resultTupleScheme(); + private static class sendSupervisorWorkerHeartbeat_resultTupleSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeat_resultTupleScheme getScheme() { + return new sendSupervisorWorkerHeartbeat_resultTupleScheme(); } } - private static class getOwnerResourceSummaries_resultTupleScheme extends TupleScheme { + private static class sendSupervisorWorkerHeartbeat_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSummaries_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeat_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); - if (struct.is_set_success()) { - optionals.set(0); - } if (struct.is_set_aze()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.is_set_success()) { - { - oprot.writeI32(struct.success.size()); - for (OwnerResourceSummary _iter906 : struct.success) - { - _iter906.write(oprot); - } - } + optionals.set(0); } + oprot.writeBitSet(optionals, 1); if (struct.is_set_aze()) { struct.aze.write(oprot); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getOwnerResourceSummaries_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeat_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(2); + BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - { - org.apache.thrift.protocol.TList _list907 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list907.size); - OwnerResourceSummary _elem908; - for (int _i909 = 0; _i909 < _list907.size; ++_i909) - { - _elem908 = new OwnerResourceSummary(); - _elem908.read(iprot); - struct.success.add(_elem908); - } - } - struct.set_success_isSet(true); - } - if (incoming.get(1)) { struct.aze = new AuthorizationException(); struct.aze.read(iprot); struct.set_aze_isSet(true); diff --git a/storm-client/src/jvm/org/apache/storm/generated/Supervisor.java b/storm-client/src/jvm/org/apache/storm/generated/Supervisor.java new file mode 100644 index 00000000000..6923e60df5b --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/generated/Supervisor.java @@ -0,0 +1,2968 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class Supervisor { + + public interface Iface { + + /** + * Send node specific assignments to supervisor + * + * @param assignments + */ + public void sendSupervisorAssignments(SupervisorAssignments assignments) throws AuthorizationException, org.apache.thrift.TException; + + /** + * Get local assignment for a storm + * + * @param id + */ + public Assignment getLocalAssignmentForStorm(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; + + /** + * Send worker heartbeat to local supervisor + * + * @param heartbeat + */ + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heartbeat) throws AuthorizationException, org.apache.thrift.TException; + + } + + public interface AsyncIface { + + public void sendSupervisorAssignments(SupervisorAssignments assignments, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getLocalAssignmentForStorm(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heartbeat, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + } + + public static class Client extends org.apache.thrift.TServiceClient implements Iface { + public static class Factory implements org.apache.thrift.TServiceClientFactory { + public Factory() {} + public Client getClient(org.apache.thrift.protocol.TProtocol prot) { + return new Client(prot); + } + public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + return new Client(iprot, oprot); + } + } + + public Client(org.apache.thrift.protocol.TProtocol prot) + { + super(prot, prot); + } + + public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + super(iprot, oprot); + } + + public void sendSupervisorAssignments(SupervisorAssignments assignments) throws AuthorizationException, org.apache.thrift.TException + { + send_sendSupervisorAssignments(assignments); + recv_sendSupervisorAssignments(); + } + + public void send_sendSupervisorAssignments(SupervisorAssignments assignments) throws org.apache.thrift.TException + { + sendSupervisorAssignments_args args = new sendSupervisorAssignments_args(); + args.set_assignments(assignments); + sendBase("sendSupervisorAssignments", args); + } + + public void recv_sendSupervisorAssignments() throws AuthorizationException, org.apache.thrift.TException + { + sendSupervisorAssignments_result result = new sendSupervisorAssignments_result(); + receiveBase(result, "sendSupervisorAssignments"); + if (result.aze != null) { + throw result.aze; + } + return; + } + + public Assignment getLocalAssignmentForStorm(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException + { + send_getLocalAssignmentForStorm(id); + return recv_getLocalAssignmentForStorm(); + } + + public void send_getLocalAssignmentForStorm(String id) throws org.apache.thrift.TException + { + getLocalAssignmentForStorm_args args = new getLocalAssignmentForStorm_args(); + args.set_id(id); + sendBase("getLocalAssignmentForStorm", args); + } + + public Assignment recv_getLocalAssignmentForStorm() throws NotAliveException, AuthorizationException, org.apache.thrift.TException + { + getLocalAssignmentForStorm_result result = new getLocalAssignmentForStorm_result(); + receiveBase(result, "getLocalAssignmentForStorm"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + if (result.aze != null) { + throw result.aze; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getLocalAssignmentForStorm failed: unknown result"); + } + + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heartbeat) throws AuthorizationException, org.apache.thrift.TException + { + send_sendSupervisorWorkerHeartbeat(heartbeat); + recv_sendSupervisorWorkerHeartbeat(); + } + + public void send_sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heartbeat) throws org.apache.thrift.TException + { + sendSupervisorWorkerHeartbeat_args args = new sendSupervisorWorkerHeartbeat_args(); + args.set_heartbeat(heartbeat); + sendBase("sendSupervisorWorkerHeartbeat", args); + } + + public void recv_sendSupervisorWorkerHeartbeat() throws AuthorizationException, org.apache.thrift.TException + { + sendSupervisorWorkerHeartbeat_result result = new sendSupervisorWorkerHeartbeat_result(); + receiveBase(result, "sendSupervisorWorkerHeartbeat"); + if (result.aze != null) { + throw result.aze; + } + return; + } + + } + public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { + public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { + private org.apache.thrift.async.TAsyncClientManager clientManager; + private org.apache.thrift.protocol.TProtocolFactory protocolFactory; + public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { + this.clientManager = clientManager; + this.protocolFactory = protocolFactory; + } + public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { + return new AsyncClient(protocolFactory, clientManager, transport); + } + } + + public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { + super(protocolFactory, clientManager, transport); + } + + public void sendSupervisorAssignments(SupervisorAssignments assignments, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + sendSupervisorAssignments_call method_call = new sendSupervisorAssignments_call(assignments, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class sendSupervisorAssignments_call extends org.apache.thrift.async.TAsyncMethodCall { + private SupervisorAssignments assignments; + public sendSupervisorAssignments_call(SupervisorAssignments assignments, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.assignments = assignments; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("sendSupervisorAssignments", org.apache.thrift.protocol.TMessageType.CALL, 0)); + sendSupervisorAssignments_args args = new sendSupervisorAssignments_args(); + args.set_assignments(assignments); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws AuthorizationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_sendSupervisorAssignments(); + } + } + + public void getLocalAssignmentForStorm(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getLocalAssignmentForStorm_call method_call = new getLocalAssignmentForStorm_call(id, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getLocalAssignmentForStorm_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + public getLocalAssignmentForStorm_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getLocalAssignmentForStorm", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getLocalAssignmentForStorm_args args = new getLocalAssignmentForStorm_args(); + args.set_id(id); + args.write(prot); + prot.writeMessageEnd(); + } + + public Assignment getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getLocalAssignmentForStorm(); + } + } + + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heartbeat, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + sendSupervisorWorkerHeartbeat_call method_call = new sendSupervisorWorkerHeartbeat_call(heartbeat, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class sendSupervisorWorkerHeartbeat_call extends org.apache.thrift.async.TAsyncMethodCall { + private SupervisorWorkerHeartbeat heartbeat; + public sendSupervisorWorkerHeartbeat_call(SupervisorWorkerHeartbeat heartbeat, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.heartbeat = heartbeat; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("sendSupervisorWorkerHeartbeat", org.apache.thrift.protocol.TMessageType.CALL, 0)); + sendSupervisorWorkerHeartbeat_args args = new sendSupervisorWorkerHeartbeat_args(); + args.set_heartbeat(heartbeat); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws AuthorizationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_sendSupervisorWorkerHeartbeat(); + } + } + + } + + public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); + public Processor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected Processor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("sendSupervisorAssignments", new sendSupervisorAssignments()); + processMap.put("getLocalAssignmentForStorm", new getLocalAssignmentForStorm()); + processMap.put("sendSupervisorWorkerHeartbeat", new sendSupervisorWorkerHeartbeat()); + return processMap; + } + + public static class sendSupervisorAssignments extends org.apache.thrift.ProcessFunction { + public sendSupervisorAssignments() { + super("sendSupervisorAssignments"); + } + + public sendSupervisorAssignments_args getEmptyArgsInstance() { + return new sendSupervisorAssignments_args(); + } + + protected boolean isOneway() { + return false; + } + + public sendSupervisorAssignments_result getResult(I iface, sendSupervisorAssignments_args args) throws org.apache.thrift.TException { + sendSupervisorAssignments_result result = new sendSupervisorAssignments_result(); + try { + iface.sendSupervisorAssignments(args.assignments); + } catch (AuthorizationException aze) { + result.aze = aze; + } + return result; + } + } + + public static class getLocalAssignmentForStorm extends org.apache.thrift.ProcessFunction { + public getLocalAssignmentForStorm() { + super("getLocalAssignmentForStorm"); + } + + public getLocalAssignmentForStorm_args getEmptyArgsInstance() { + return new getLocalAssignmentForStorm_args(); + } + + protected boolean isOneway() { + return false; + } + + public getLocalAssignmentForStorm_result getResult(I iface, getLocalAssignmentForStorm_args args) throws org.apache.thrift.TException { + getLocalAssignmentForStorm_result result = new getLocalAssignmentForStorm_result(); + try { + result.success = iface.getLocalAssignmentForStorm(args.id); + } catch (NotAliveException e) { + result.e = e; + } catch (AuthorizationException aze) { + result.aze = aze; + } + return result; + } + } + + public static class sendSupervisorWorkerHeartbeat extends org.apache.thrift.ProcessFunction { + public sendSupervisorWorkerHeartbeat() { + super("sendSupervisorWorkerHeartbeat"); + } + + public sendSupervisorWorkerHeartbeat_args getEmptyArgsInstance() { + return new sendSupervisorWorkerHeartbeat_args(); + } + + protected boolean isOneway() { + return false; + } + + public sendSupervisorWorkerHeartbeat_result getResult(I iface, sendSupervisorWorkerHeartbeat_args args) throws org.apache.thrift.TException { + sendSupervisorWorkerHeartbeat_result result = new sendSupervisorWorkerHeartbeat_result(); + try { + iface.sendSupervisorWorkerHeartbeat(args.heartbeat); + } catch (AuthorizationException aze) { + result.aze = aze; + } + return result; + } + } + + } + + public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName()); + public AsyncProcessor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected AsyncProcessor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("sendSupervisorAssignments", new sendSupervisorAssignments()); + processMap.put("getLocalAssignmentForStorm", new getLocalAssignmentForStorm()); + processMap.put("sendSupervisorWorkerHeartbeat", new sendSupervisorWorkerHeartbeat()); + return processMap; + } + + public static class sendSupervisorAssignments extends org.apache.thrift.AsyncProcessFunction { + public sendSupervisorAssignments() { + super("sendSupervisorAssignments"); + } + + public sendSupervisorAssignments_args getEmptyArgsInstance() { + return new sendSupervisorAssignments_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + sendSupervisorAssignments_result result = new sendSupervisorAssignments_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + sendSupervisorAssignments_result result = new sendSupervisorAssignments_result(); + if (e instanceof AuthorizationException) { + result.aze = (AuthorizationException) e; + result.set_aze_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, sendSupervisorAssignments_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.sendSupervisorAssignments(args.assignments,resultHandler); + } + } + + public static class getLocalAssignmentForStorm extends org.apache.thrift.AsyncProcessFunction { + public getLocalAssignmentForStorm() { + super("getLocalAssignmentForStorm"); + } + + public getLocalAssignmentForStorm_args getEmptyArgsInstance() { + return new getLocalAssignmentForStorm_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Assignment o) { + getLocalAssignmentForStorm_result result = new getLocalAssignmentForStorm_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getLocalAssignmentForStorm_result result = new getLocalAssignmentForStorm_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else if (e instanceof AuthorizationException) { + result.aze = (AuthorizationException) e; + result.set_aze_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getLocalAssignmentForStorm_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getLocalAssignmentForStorm(args.id,resultHandler); + } + } + + public static class sendSupervisorWorkerHeartbeat extends org.apache.thrift.AsyncProcessFunction { + public sendSupervisorWorkerHeartbeat() { + super("sendSupervisorWorkerHeartbeat"); + } + + public sendSupervisorWorkerHeartbeat_args getEmptyArgsInstance() { + return new sendSupervisorWorkerHeartbeat_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + sendSupervisorWorkerHeartbeat_result result = new sendSupervisorWorkerHeartbeat_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + sendSupervisorWorkerHeartbeat_result result = new sendSupervisorWorkerHeartbeat_result(); + if (e instanceof AuthorizationException) { + result.aze = (AuthorizationException) e; + result.set_aze_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, sendSupervisorWorkerHeartbeat_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.sendSupervisorWorkerHeartbeat(args.heartbeat,resultHandler); + } + } + + } + + public static class sendSupervisorAssignments_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("sendSupervisorAssignments_args"); + + private static final org.apache.thrift.protocol.TField ASSIGNMENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("assignments", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new sendSupervisorAssignments_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new sendSupervisorAssignments_argsTupleSchemeFactory()); + } + + private SupervisorAssignments assignments; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ASSIGNMENTS((short)1, "assignments"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ASSIGNMENTS + return ASSIGNMENTS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ASSIGNMENTS, new org.apache.thrift.meta_data.FieldMetaData("assignments", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorAssignments.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(sendSupervisorAssignments_args.class, metaDataMap); + } + + public sendSupervisorAssignments_args() { + } + + public sendSupervisorAssignments_args( + SupervisorAssignments assignments) + { + this(); + this.assignments = assignments; + } + + /** + * Performs a deep copy on other. + */ + public sendSupervisorAssignments_args(sendSupervisorAssignments_args other) { + if (other.is_set_assignments()) { + this.assignments = new SupervisorAssignments(other.assignments); + } + } + + public sendSupervisorAssignments_args deepCopy() { + return new sendSupervisorAssignments_args(this); + } + + @Override + public void clear() { + this.assignments = null; + } + + public SupervisorAssignments get_assignments() { + return this.assignments; + } + + public void set_assignments(SupervisorAssignments assignments) { + this.assignments = assignments; + } + + public void unset_assignments() { + this.assignments = null; + } + + /** Returns true if field assignments is set (has been assigned a value) and false otherwise */ + public boolean is_set_assignments() { + return this.assignments != null; + } + + public void set_assignments_isSet(boolean value) { + if (!value) { + this.assignments = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ASSIGNMENTS: + if (value == null) { + unset_assignments(); + } else { + set_assignments((SupervisorAssignments)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ASSIGNMENTS: + return get_assignments(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ASSIGNMENTS: + return is_set_assignments(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof sendSupervisorAssignments_args) + return this.equals((sendSupervisorAssignments_args)that); + return false; + } + + public boolean equals(sendSupervisorAssignments_args that) { + if (that == null) + return false; + + boolean this_present_assignments = true && this.is_set_assignments(); + boolean that_present_assignments = true && that.is_set_assignments(); + if (this_present_assignments || that_present_assignments) { + if (!(this_present_assignments && that_present_assignments)) + return false; + if (!this.assignments.equals(that.assignments)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_assignments = true && (is_set_assignments()); + list.add(present_assignments); + if (present_assignments) + list.add(assignments); + + return list.hashCode(); + } + + @Override + public int compareTo(sendSupervisorAssignments_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_assignments()).compareTo(other.is_set_assignments()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_assignments()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assignments, other.assignments); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("sendSupervisorAssignments_args("); + boolean first = true; + + sb.append("assignments:"); + if (this.assignments == null) { + sb.append("null"); + } else { + sb.append(this.assignments); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (assignments != null) { + assignments.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class sendSupervisorAssignments_argsStandardSchemeFactory implements SchemeFactory { + public sendSupervisorAssignments_argsStandardScheme getScheme() { + return new sendSupervisorAssignments_argsStandardScheme(); + } + } + + private static class sendSupervisorAssignments_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, sendSupervisorAssignments_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ASSIGNMENTS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.assignments = new SupervisorAssignments(); + struct.assignments.read(iprot); + struct.set_assignments_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, sendSupervisorAssignments_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.assignments != null) { + oprot.writeFieldBegin(ASSIGNMENTS_FIELD_DESC); + struct.assignments.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class sendSupervisorAssignments_argsTupleSchemeFactory implements SchemeFactory { + public sendSupervisorAssignments_argsTupleScheme getScheme() { + return new sendSupervisorAssignments_argsTupleScheme(); + } + } + + private static class sendSupervisorAssignments_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, sendSupervisorAssignments_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_assignments()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_assignments()) { + struct.assignments.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, sendSupervisorAssignments_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.assignments = new SupervisorAssignments(); + struct.assignments.read(iprot); + struct.set_assignments_isSet(true); + } + } + } + + } + + public static class sendSupervisorAssignments_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("sendSupervisorAssignments_result"); + + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new sendSupervisorAssignments_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new sendSupervisorAssignments_resultTupleSchemeFactory()); + } + + private AuthorizationException aze; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + AZE((short)1, "aze"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // AZE + return AZE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(sendSupervisorAssignments_result.class, metaDataMap); + } + + public sendSupervisorAssignments_result() { + } + + public sendSupervisorAssignments_result( + AuthorizationException aze) + { + this(); + this.aze = aze; + } + + /** + * Performs a deep copy on other. + */ + public sendSupervisorAssignments_result(sendSupervisorAssignments_result other) { + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } + } + + public sendSupervisorAssignments_result deepCopy() { + return new sendSupervisorAssignments_result(this); + } + + @Override + public void clear() { + this.aze = null; + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case AZE: + return get_aze(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case AZE: + return is_set_aze(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof sendSupervisorAssignments_result) + return this.equals((sendSupervisorAssignments_result)that); + return false; + } + + public boolean equals(sendSupervisorAssignments_result that) { + if (that == null) + return false; + + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_aze = true && (is_set_aze()); + list.add(present_aze); + if (present_aze) + list.add(aze); + + return list.hashCode(); + } + + @Override + public int compareTo(sendSupervisorAssignments_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("sendSupervisorAssignments_result("); + boolean first = true; + + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class sendSupervisorAssignments_resultStandardSchemeFactory implements SchemeFactory { + public sendSupervisorAssignments_resultStandardScheme getScheme() { + return new sendSupervisorAssignments_resultStandardScheme(); + } + } + + private static class sendSupervisorAssignments_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, sendSupervisorAssignments_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // AZE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, sendSupervisorAssignments_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.aze != null) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + struct.aze.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class sendSupervisorAssignments_resultTupleSchemeFactory implements SchemeFactory { + public sendSupervisorAssignments_resultTupleScheme getScheme() { + return new sendSupervisorAssignments_resultTupleScheme(); + } + } + + private static class sendSupervisorAssignments_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, sendSupervisorAssignments_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_aze()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_aze()) { + struct.aze.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, sendSupervisorAssignments_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } + } + } + + } + + public static class getLocalAssignmentForStorm_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getLocalAssignmentForStorm_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getLocalAssignmentForStorm_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getLocalAssignmentForStorm_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getLocalAssignmentForStorm_args.class, metaDataMap); + } + + public getLocalAssignmentForStorm_args() { + } + + public getLocalAssignmentForStorm_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public getLocalAssignmentForStorm_args(getLocalAssignmentForStorm_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public getLocalAssignmentForStorm_args deepCopy() { + return new getLocalAssignmentForStorm_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getLocalAssignmentForStorm_args) + return this.equals((getLocalAssignmentForStorm_args)that); + return false; + } + + public boolean equals(getLocalAssignmentForStorm_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(getLocalAssignmentForStorm_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getLocalAssignmentForStorm_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getLocalAssignmentForStorm_argsStandardSchemeFactory implements SchemeFactory { + public getLocalAssignmentForStorm_argsStandardScheme getScheme() { + return new getLocalAssignmentForStorm_argsStandardScheme(); + } + } + + private static class getLocalAssignmentForStorm_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getLocalAssignmentForStorm_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getLocalAssignmentForStorm_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getLocalAssignmentForStorm_argsTupleSchemeFactory implements SchemeFactory { + public getLocalAssignmentForStorm_argsTupleScheme getScheme() { + return new getLocalAssignmentForStorm_argsTupleScheme(); + } + } + + private static class getLocalAssignmentForStorm_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getLocalAssignmentForStorm_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getLocalAssignmentForStorm_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class getLocalAssignmentForStorm_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getLocalAssignmentForStorm_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getLocalAssignmentForStorm_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getLocalAssignmentForStorm_resultTupleSchemeFactory()); + } + + private Assignment success; // required + private NotAliveException e; // required + private AuthorizationException aze; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"), + AZE((short)2, "aze"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + case 2: // AZE + return AZE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Assignment.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getLocalAssignmentForStorm_result.class, metaDataMap); + } + + public getLocalAssignmentForStorm_result() { + } + + public getLocalAssignmentForStorm_result( + Assignment success, + NotAliveException e, + AuthorizationException aze) + { + this(); + this.success = success; + this.e = e; + this.aze = aze; + } + + /** + * Performs a deep copy on other. + */ + public getLocalAssignmentForStorm_result(getLocalAssignmentForStorm_result other) { + if (other.is_set_success()) { + this.success = new Assignment(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } + } + + public getLocalAssignmentForStorm_result deepCopy() { + return new getLocalAssignmentForStorm_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + this.aze = null; + } + + public Assignment get_success() { + return this.success; + } + + public void set_success(Assignment success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((Assignment)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + case AZE: + return get_aze(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + case AZE: + return is_set_aze(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getLocalAssignmentForStorm_result) + return this.equals((getLocalAssignmentForStorm_result)that); + return false; + } + + public boolean equals(getLocalAssignmentForStorm_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_aze = true && (is_set_aze()); + list.add(present_aze); + if (present_aze) + list.add(aze); + + return list.hashCode(); + } + + @Override + public int compareTo(getLocalAssignmentForStorm_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getLocalAssignmentForStorm_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getLocalAssignmentForStorm_resultStandardSchemeFactory implements SchemeFactory { + public getLocalAssignmentForStorm_resultStandardScheme getScheme() { + return new getLocalAssignmentForStorm_resultStandardScheme(); + } + } + + private static class getLocalAssignmentForStorm_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getLocalAssignmentForStorm_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new Assignment(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // AZE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getLocalAssignmentForStorm_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.aze != null) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + struct.aze.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getLocalAssignmentForStorm_resultTupleSchemeFactory implements SchemeFactory { + public getLocalAssignmentForStorm_resultTupleScheme getScheme() { + return new getLocalAssignmentForStorm_resultTupleScheme(); + } + } + + private static class getLocalAssignmentForStorm_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getLocalAssignmentForStorm_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + if (struct.is_set_aze()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_aze()) { + struct.aze.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getLocalAssignmentForStorm_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.success = new Assignment(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(2)) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } + } + } + + } + + public static class sendSupervisorWorkerHeartbeat_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("sendSupervisorWorkerHeartbeat_args"); + + private static final org.apache.thrift.protocol.TField HEARTBEAT_FIELD_DESC = new org.apache.thrift.protocol.TField("heartbeat", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new sendSupervisorWorkerHeartbeat_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new sendSupervisorWorkerHeartbeat_argsTupleSchemeFactory()); + } + + private SupervisorWorkerHeartbeat heartbeat; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + HEARTBEAT((short)1, "heartbeat"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // HEARTBEAT + return HEARTBEAT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.HEARTBEAT, new org.apache.thrift.meta_data.FieldMetaData("heartbeat", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorWorkerHeartbeat.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(sendSupervisorWorkerHeartbeat_args.class, metaDataMap); + } + + public sendSupervisorWorkerHeartbeat_args() { + } + + public sendSupervisorWorkerHeartbeat_args( + SupervisorWorkerHeartbeat heartbeat) + { + this(); + this.heartbeat = heartbeat; + } + + /** + * Performs a deep copy on other. + */ + public sendSupervisorWorkerHeartbeat_args(sendSupervisorWorkerHeartbeat_args other) { + if (other.is_set_heartbeat()) { + this.heartbeat = new SupervisorWorkerHeartbeat(other.heartbeat); + } + } + + public sendSupervisorWorkerHeartbeat_args deepCopy() { + return new sendSupervisorWorkerHeartbeat_args(this); + } + + @Override + public void clear() { + this.heartbeat = null; + } + + public SupervisorWorkerHeartbeat get_heartbeat() { + return this.heartbeat; + } + + public void set_heartbeat(SupervisorWorkerHeartbeat heartbeat) { + this.heartbeat = heartbeat; + } + + public void unset_heartbeat() { + this.heartbeat = null; + } + + /** Returns true if field heartbeat is set (has been assigned a value) and false otherwise */ + public boolean is_set_heartbeat() { + return this.heartbeat != null; + } + + public void set_heartbeat_isSet(boolean value) { + if (!value) { + this.heartbeat = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case HEARTBEAT: + if (value == null) { + unset_heartbeat(); + } else { + set_heartbeat((SupervisorWorkerHeartbeat)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case HEARTBEAT: + return get_heartbeat(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case HEARTBEAT: + return is_set_heartbeat(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof sendSupervisorWorkerHeartbeat_args) + return this.equals((sendSupervisorWorkerHeartbeat_args)that); + return false; + } + + public boolean equals(sendSupervisorWorkerHeartbeat_args that) { + if (that == null) + return false; + + boolean this_present_heartbeat = true && this.is_set_heartbeat(); + boolean that_present_heartbeat = true && that.is_set_heartbeat(); + if (this_present_heartbeat || that_present_heartbeat) { + if (!(this_present_heartbeat && that_present_heartbeat)) + return false; + if (!this.heartbeat.equals(that.heartbeat)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_heartbeat = true && (is_set_heartbeat()); + list.add(present_heartbeat); + if (present_heartbeat) + list.add(heartbeat); + + return list.hashCode(); + } + + @Override + public int compareTo(sendSupervisorWorkerHeartbeat_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_heartbeat()).compareTo(other.is_set_heartbeat()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_heartbeat()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.heartbeat, other.heartbeat); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("sendSupervisorWorkerHeartbeat_args("); + boolean first = true; + + sb.append("heartbeat:"); + if (this.heartbeat == null) { + sb.append("null"); + } else { + sb.append(this.heartbeat); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (heartbeat != null) { + heartbeat.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class sendSupervisorWorkerHeartbeat_argsStandardSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeat_argsStandardScheme getScheme() { + return new sendSupervisorWorkerHeartbeat_argsStandardScheme(); + } + } + + private static class sendSupervisorWorkerHeartbeat_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, sendSupervisorWorkerHeartbeat_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // HEARTBEAT + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.heartbeat = new SupervisorWorkerHeartbeat(); + struct.heartbeat.read(iprot); + struct.set_heartbeat_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, sendSupervisorWorkerHeartbeat_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.heartbeat != null) { + oprot.writeFieldBegin(HEARTBEAT_FIELD_DESC); + struct.heartbeat.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class sendSupervisorWorkerHeartbeat_argsTupleSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeat_argsTupleScheme getScheme() { + return new sendSupervisorWorkerHeartbeat_argsTupleScheme(); + } + } + + private static class sendSupervisorWorkerHeartbeat_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeat_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_heartbeat()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_heartbeat()) { + struct.heartbeat.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeat_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.heartbeat = new SupervisorWorkerHeartbeat(); + struct.heartbeat.read(iprot); + struct.set_heartbeat_isSet(true); + } + } + } + + } + + public static class sendSupervisorWorkerHeartbeat_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("sendSupervisorWorkerHeartbeat_result"); + + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new sendSupervisorWorkerHeartbeat_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new sendSupervisorWorkerHeartbeat_resultTupleSchemeFactory()); + } + + private AuthorizationException aze; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + AZE((short)1, "aze"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // AZE + return AZE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(sendSupervisorWorkerHeartbeat_result.class, metaDataMap); + } + + public sendSupervisorWorkerHeartbeat_result() { + } + + public sendSupervisorWorkerHeartbeat_result( + AuthorizationException aze) + { + this(); + this.aze = aze; + } + + /** + * Performs a deep copy on other. + */ + public sendSupervisorWorkerHeartbeat_result(sendSupervisorWorkerHeartbeat_result other) { + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } + } + + public sendSupervisorWorkerHeartbeat_result deepCopy() { + return new sendSupervisorWorkerHeartbeat_result(this); + } + + @Override + public void clear() { + this.aze = null; + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case AZE: + return get_aze(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case AZE: + return is_set_aze(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof sendSupervisorWorkerHeartbeat_result) + return this.equals((sendSupervisorWorkerHeartbeat_result)that); + return false; + } + + public boolean equals(sendSupervisorWorkerHeartbeat_result that) { + if (that == null) + return false; + + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_aze = true && (is_set_aze()); + list.add(present_aze); + if (present_aze) + list.add(aze); + + return list.hashCode(); + } + + @Override + public int compareTo(sendSupervisorWorkerHeartbeat_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("sendSupervisorWorkerHeartbeat_result("); + boolean first = true; + + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class sendSupervisorWorkerHeartbeat_resultStandardSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeat_resultStandardScheme getScheme() { + return new sendSupervisorWorkerHeartbeat_resultStandardScheme(); + } + } + + private static class sendSupervisorWorkerHeartbeat_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, sendSupervisorWorkerHeartbeat_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // AZE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, sendSupervisorWorkerHeartbeat_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.aze != null) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + struct.aze.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class sendSupervisorWorkerHeartbeat_resultTupleSchemeFactory implements SchemeFactory { + public sendSupervisorWorkerHeartbeat_resultTupleScheme getScheme() { + return new sendSupervisorWorkerHeartbeat_resultTupleScheme(); + } + } + + private static class sendSupervisorWorkerHeartbeat_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeat_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_aze()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_aze()) { + struct.aze.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, sendSupervisorWorkerHeartbeat_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } + } + } + + } + +} diff --git a/storm-client/src/jvm/org/apache/storm/generated/SupervisorAssignments.java b/storm-client/src/jvm/org/apache/storm/generated/SupervisorAssignments.java new file mode 100644 index 00000000000..1cfb9469ab4 --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/generated/SupervisorAssignments.java @@ -0,0 +1,478 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class SupervisorAssignments implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorAssignments"); + + private static final org.apache.thrift.protocol.TField STORM_ASSIGNMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("storm_assignment", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SupervisorAssignmentsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SupervisorAssignmentsTupleSchemeFactory()); + } + + private Map storm_assignment; // 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 { + STORM_ASSIGNMENT((short)1, "storm_assignment"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STORM_ASSIGNMENT + return STORM_ASSIGNMENT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.STORM_ASSIGNMENT}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STORM_ASSIGNMENT, new org.apache.thrift.meta_data.FieldMetaData("storm_assignment", 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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Assignment.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorAssignments.class, metaDataMap); + } + + public SupervisorAssignments() { + this.storm_assignment = new HashMap(); + + } + + /** + * Performs a deep copy on other. + */ + public SupervisorAssignments(SupervisorAssignments other) { + if (other.is_set_storm_assignment()) { + Map __this__storm_assignment = new HashMap(other.storm_assignment.size()); + for (Map.Entry other_element : other.storm_assignment.entrySet()) { + + String other_element_key = other_element.getKey(); + Assignment other_element_value = other_element.getValue(); + + String __this__storm_assignment_copy_key = other_element_key; + + Assignment __this__storm_assignment_copy_value = new Assignment(other_element_value); + + __this__storm_assignment.put(__this__storm_assignment_copy_key, __this__storm_assignment_copy_value); + } + this.storm_assignment = __this__storm_assignment; + } + } + + public SupervisorAssignments deepCopy() { + return new SupervisorAssignments(this); + } + + @Override + public void clear() { + this.storm_assignment = new HashMap(); + + } + + public int get_storm_assignment_size() { + return (this.storm_assignment == null) ? 0 : this.storm_assignment.size(); + } + + public void put_to_storm_assignment(String key, Assignment val) { + if (this.storm_assignment == null) { + this.storm_assignment = new HashMap(); + } + this.storm_assignment.put(key, val); + } + + public Map get_storm_assignment() { + return this.storm_assignment; + } + + public void set_storm_assignment(Map storm_assignment) { + this.storm_assignment = storm_assignment; + } + + public void unset_storm_assignment() { + this.storm_assignment = null; + } + + /** Returns true if field storm_assignment is set (has been assigned a value) and false otherwise */ + public boolean is_set_storm_assignment() { + return this.storm_assignment != null; + } + + public void set_storm_assignment_isSet(boolean value) { + if (!value) { + this.storm_assignment = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STORM_ASSIGNMENT: + if (value == null) { + unset_storm_assignment(); + } else { + set_storm_assignment((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STORM_ASSIGNMENT: + return get_storm_assignment(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STORM_ASSIGNMENT: + return is_set_storm_assignment(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof SupervisorAssignments) + return this.equals((SupervisorAssignments)that); + return false; + } + + public boolean equals(SupervisorAssignments that) { + if (that == null) + return false; + + boolean this_present_storm_assignment = true && this.is_set_storm_assignment(); + boolean that_present_storm_assignment = true && that.is_set_storm_assignment(); + if (this_present_storm_assignment || that_present_storm_assignment) { + if (!(this_present_storm_assignment && that_present_storm_assignment)) + return false; + if (!this.storm_assignment.equals(that.storm_assignment)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_storm_assignment = true && (is_set_storm_assignment()); + list.add(present_storm_assignment); + if (present_storm_assignment) + list.add(storm_assignment); + + return list.hashCode(); + } + + @Override + public int compareTo(SupervisorAssignments other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_storm_assignment()).compareTo(other.is_set_storm_assignment()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_storm_assignment()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storm_assignment, other.storm_assignment); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("SupervisorAssignments("); + boolean first = true; + + if (is_set_storm_assignment()) { + sb.append("storm_assignment:"); + if (this.storm_assignment == null) { + sb.append("null"); + } else { + sb.append(this.storm_assignment); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class SupervisorAssignmentsStandardSchemeFactory implements SchemeFactory { + public SupervisorAssignmentsStandardScheme getScheme() { + return new SupervisorAssignmentsStandardScheme(); + } + } + + private static class SupervisorAssignmentsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorAssignments struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STORM_ASSIGNMENT + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map886 = iprot.readMapBegin(); + struct.storm_assignment = new HashMap(2*_map886.size); + String _key887; + Assignment _val888; + for (int _i889 = 0; _i889 < _map886.size; ++_i889) + { + _key887 = iprot.readString(); + _val888 = new Assignment(); + _val888.read(iprot); + struct.storm_assignment.put(_key887, _val888); + } + iprot.readMapEnd(); + } + struct.set_storm_assignment_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorAssignments struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.storm_assignment != null) { + if (struct.is_set_storm_assignment()) { + oprot.writeFieldBegin(STORM_ASSIGNMENT_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.storm_assignment.size())); + for (Map.Entry _iter890 : struct.storm_assignment.entrySet()) + { + oprot.writeString(_iter890.getKey()); + _iter890.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class SupervisorAssignmentsTupleSchemeFactory implements SchemeFactory { + public SupervisorAssignmentsTupleScheme getScheme() { + return new SupervisorAssignmentsTupleScheme(); + } + } + + private static class SupervisorAssignmentsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorAssignments struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_storm_assignment()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_storm_assignment()) { + { + oprot.writeI32(struct.storm_assignment.size()); + for (Map.Entry _iter891 : struct.storm_assignment.entrySet()) + { + oprot.writeString(_iter891.getKey()); + _iter891.getValue().write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorAssignments struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TMap _map892 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.storm_assignment = new HashMap(2*_map892.size); + String _key893; + Assignment _val894; + for (int _i895 = 0; _i895 < _map892.size; ++_i895) + { + _key893 = iprot.readString(); + _val894 = new Assignment(); + _val894.read(iprot); + struct.storm_assignment.put(_key893, _val894); + } + } + struct.set_storm_assignment_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 45293ed59ab..6d2a0515664 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java +++ b/storm-client/src/jvm/org/apache/storm/generated/SupervisorInfo.java @@ -58,12 +58,13 @@ public class SupervisorInfo implements org.apache.thrift.TBase, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -74,6 +75,7 @@ public class SupervisorInfo implements org.apache.thrift.TBase used_ports; // optional private List meta; // optional private Map scheduler_meta; // optional @@ -86,12 +88,13 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { TIME_SECS((short)1, "time_secs"), HOSTNAME((short)2, "hostname"), ASSIGNMENT_ID((short)3, "assignment_id"), - USED_PORTS((short)4, "used_ports"), - META((short)5, "meta"), - SCHEDULER_META((short)6, "scheduler_meta"), - UPTIME_SECS((short)7, "uptime_secs"), - VERSION((short)8, "version"), - RESOURCES_MAP((short)9, "resources_map"); + SERVER_PORT((short)4, "server_port"), + USED_PORTS((short)5, "used_ports"), + META((short)6, "meta"), + SCHEDULER_META((short)7, "scheduler_meta"), + UPTIME_SECS((short)8, "uptime_secs"), + VERSION((short)9, "version"), + RESOURCES_MAP((short)10, "resources_map"); private static final Map byName = new HashMap(); @@ -112,17 +115,19 @@ public static _Fields findByThriftId(int fieldId) { return HOSTNAME; case 3: // ASSIGNMENT_ID return ASSIGNMENT_ID; - case 4: // USED_PORTS + case 4: // SERVER_PORT + return SERVER_PORT; + case 5: // USED_PORTS return USED_PORTS; - case 5: // META + case 6: // META return META; - case 6: // SCHEDULER_META + case 7: // SCHEDULER_META return SCHEDULER_META; - case 7: // UPTIME_SECS + case 8: // UPTIME_SECS return UPTIME_SECS; - case 8: // VERSION + case 9: // VERSION return VERSION; - case 9: // RESOURCES_MAP + case 10: // RESOURCES_MAP return RESOURCES_MAP; default: return null; @@ -165,9 +170,10 @@ public String getFieldName() { // isset id assignments private static final int __TIME_SECS_ISSET_ID = 0; - private static final int __UPTIME_SECS_ISSET_ID = 1; + private static final int __SERVER_PORT_ISSET_ID = 1; + private static final int __UPTIME_SECS_ISSET_ID = 2; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.ASSIGNMENT_ID,_Fields.USED_PORTS,_Fields.META,_Fields.SCHEDULER_META,_Fields.UPTIME_SECS,_Fields.VERSION,_Fields.RESOURCES_MAP}; + private static final _Fields optionals[] = {_Fields.ASSIGNMENT_ID,_Fields.SERVER_PORT,_Fields.USED_PORTS,_Fields.META,_Fields.SCHEDULER_META,_Fields.UPTIME_SECS,_Fields.VERSION,_Fields.RESOURCES_MAP}; 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); @@ -177,6 +183,8 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.ASSIGNMENT_ID, new org.apache.thrift.meta_data.FieldMetaData("assignment_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.SERVER_PORT, new org.apache.thrift.meta_data.FieldMetaData("server_port", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); tmpMap.put(_Fields.USED_PORTS, new org.apache.thrift.meta_data.FieldMetaData("used_ports", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); @@ -224,6 +232,7 @@ public SupervisorInfo(SupervisorInfo other) { if (other.is_set_assignment_id()) { this.assignment_id = other.assignment_id; } + this.server_port = other.server_port; if (other.is_set_used_ports()) { List __this__used_ports = new ArrayList(other.used_ports); this.used_ports = __this__used_ports; @@ -256,6 +265,8 @@ public void clear() { this.time_secs = 0; this.hostname = null; this.assignment_id = null; + set_server_port_isSet(false); + this.server_port = 0; this.used_ports = null; this.meta = null; this.scheduler_meta = null; @@ -333,6 +344,28 @@ public void set_assignment_id_isSet(boolean value) { } } + public int get_server_port() { + return this.server_port; + } + + public void set_server_port(int server_port) { + this.server_port = server_port; + set_server_port_isSet(true); + } + + public void unset_server_port() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SERVER_PORT_ISSET_ID); + } + + /** Returns true if field server_port is set (has been assigned a value) and false otherwise */ + public boolean is_set_server_port() { + return EncodingUtils.testBit(__isset_bitfield, __SERVER_PORT_ISSET_ID); + } + + public void set_server_port_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SERVER_PORT_ISSET_ID, value); + } + public int get_used_ports_size() { return (this.used_ports == null) ? 0 : this.used_ports.size(); } @@ -548,6 +581,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case SERVER_PORT: + if (value == null) { + unset_server_port(); + } else { + set_server_port((Integer)value); + } + break; + case USED_PORTS: if (value == null) { unset_used_ports(); @@ -610,6 +651,9 @@ public Object getFieldValue(_Fields field) { case ASSIGNMENT_ID: return get_assignment_id(); + case SERVER_PORT: + return get_server_port(); + case USED_PORTS: return get_used_ports(); @@ -645,6 +689,8 @@ public boolean isSet(_Fields field) { return is_set_hostname(); case ASSIGNMENT_ID: return is_set_assignment_id(); + case SERVER_PORT: + return is_set_server_port(); case USED_PORTS: return is_set_used_ports(); case META: @@ -701,6 +747,15 @@ public boolean equals(SupervisorInfo that) { return false; } + boolean this_present_server_port = true && this.is_set_server_port(); + boolean that_present_server_port = true && that.is_set_server_port(); + if (this_present_server_port || that_present_server_port) { + if (!(this_present_server_port && that_present_server_port)) + return false; + if (this.server_port != that.server_port) + return false; + } + boolean this_present_used_ports = true && this.is_set_used_ports(); boolean that_present_used_ports = true && that.is_set_used_ports(); if (this_present_used_ports || that_present_used_ports) { @@ -777,6 +832,11 @@ public int hashCode() { if (present_assignment_id) list.add(assignment_id); + boolean present_server_port = true && (is_set_server_port()); + list.add(present_server_port); + if (present_server_port) + list.add(server_port); + boolean present_used_ports = true && (is_set_used_ports()); list.add(present_used_ports); if (present_used_ports) @@ -848,6 +908,16 @@ public int compareTo(SupervisorInfo other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_server_port()).compareTo(other.is_set_server_port()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_server_port()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.server_port, other.server_port); + if (lastComparison != 0) { + return lastComparison; + } + } lastComparison = Boolean.valueOf(is_set_used_ports()).compareTo(other.is_set_used_ports()); if (lastComparison != 0) { return lastComparison; @@ -949,6 +1019,12 @@ public String toString() { } first = false; } + if (is_set_server_port()) { + if (!first) sb.append(", "); + sb.append("server_port:"); + sb.append(this.server_port); + first = false; + } if (is_set_used_ports()) { if (!first) sb.append(", "); sb.append("used_ports:"); @@ -1082,7 +1158,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 4: // USED_PORTS + case 4: // SERVER_PORT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.server_port = iprot.readI32(); + struct.set_server_port_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // USED_PORTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { org.apache.thrift.protocol.TList _list622 = iprot.readListBegin(); @@ -1100,7 +1184,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 5: // META + case 6: // META if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { org.apache.thrift.protocol.TList _list625 = iprot.readListBegin(); @@ -1118,7 +1202,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 6: // SCHEDULER_META + case 7: // SCHEDULER_META if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { org.apache.thrift.protocol.TMap _map628 = iprot.readMapBegin(); @@ -1138,7 +1222,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 7: // UPTIME_SECS + case 8: // UPTIME_SECS if (schemeField.type == org.apache.thrift.protocol.TType.I64) { struct.uptime_secs = iprot.readI64(); struct.set_uptime_secs_isSet(true); @@ -1146,7 +1230,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 8: // VERSION + case 9: // VERSION if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { struct.version = iprot.readString(); struct.set_version_isSet(true); @@ -1154,7 +1238,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 9: // RESOURCES_MAP + case 10: // RESOURCES_MAP if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { org.apache.thrift.protocol.TMap _map632 = iprot.readMapBegin(); @@ -1202,6 +1286,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo str oprot.writeFieldEnd(); } } + if (struct.is_set_server_port()) { + oprot.writeFieldBegin(SERVER_PORT_FIELD_DESC); + oprot.writeI32(struct.server_port); + oprot.writeFieldEnd(); + } if (struct.used_ports != null) { if (struct.is_set_used_ports()) { oprot.writeFieldBegin(USED_PORTS_FIELD_DESC); @@ -1295,28 +1384,34 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo stru if (struct.is_set_assignment_id()) { optionals.set(0); } - if (struct.is_set_used_ports()) { + if (struct.is_set_server_port()) { optionals.set(1); } - if (struct.is_set_meta()) { + if (struct.is_set_used_ports()) { optionals.set(2); } - if (struct.is_set_scheduler_meta()) { + if (struct.is_set_meta()) { optionals.set(3); } - if (struct.is_set_uptime_secs()) { + if (struct.is_set_scheduler_meta()) { optionals.set(4); } - if (struct.is_set_version()) { + if (struct.is_set_uptime_secs()) { optionals.set(5); } - if (struct.is_set_resources_map()) { + if (struct.is_set_version()) { optionals.set(6); } - oprot.writeBitSet(optionals, 7); + if (struct.is_set_resources_map()) { + optionals.set(7); + } + oprot.writeBitSet(optionals, 8); if (struct.is_set_assignment_id()) { oprot.writeString(struct.assignment_id); } + if (struct.is_set_server_port()) { + oprot.writeI32(struct.server_port); + } if (struct.is_set_used_ports()) { { oprot.writeI32(struct.used_ports.size()); @@ -1370,12 +1465,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struc struct.set_time_secs_isSet(true); struct.hostname = iprot.readString(); struct.set_hostname_isSet(true); - BitSet incoming = iprot.readBitSet(7); + BitSet incoming = iprot.readBitSet(8); if (incoming.get(0)) { struct.assignment_id = iprot.readString(); struct.set_assignment_id_isSet(true); } if (incoming.get(1)) { + struct.server_port = iprot.readI32(); + struct.set_server_port_isSet(true); + } + if (incoming.get(2)) { { 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); @@ -1388,7 +1487,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struc } struct.set_used_ports_isSet(true); } - if (incoming.get(2)) { + if (incoming.get(3)) { { 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); @@ -1401,7 +1500,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struc } struct.set_meta_isSet(true); } - if (incoming.get(3)) { + if (incoming.get(4)) { { 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); @@ -1416,15 +1515,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struc } struct.set_scheduler_meta_isSet(true); } - if (incoming.get(4)) { + if (incoming.get(5)) { struct.uptime_secs = iprot.readI64(); struct.set_uptime_secs_isSet(true); } - if (incoming.get(5)) { + if (incoming.get(6)) { struct.version = iprot.readString(); struct.set_version_isSet(true); } - if (incoming.get(6)) { + if (incoming.get(7)) { { 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); diff --git a/storm-client/src/jvm/org/apache/storm/generated/SupervisorWorkerHeartbeat.java b/storm-client/src/jvm/org/apache/storm/generated/SupervisorWorkerHeartbeat.java new file mode 100644 index 00000000000..0d5c65f0593 --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/generated/SupervisorWorkerHeartbeat.java @@ -0,0 +1,660 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class SupervisorWorkerHeartbeat implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorWorkerHeartbeat"); + + private static final org.apache.thrift.protocol.TField STORM_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storm_id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("executors", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("time_secs", org.apache.thrift.protocol.TType.I32, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SupervisorWorkerHeartbeatStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SupervisorWorkerHeartbeatTupleSchemeFactory()); + } + + private String storm_id; // required + private List executors; // required + private int time_secs; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STORM_ID((short)1, "storm_id"), + EXECUTORS((short)2, "executors"), + TIME_SECS((short)3, "time_secs"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STORM_ID + return STORM_ID; + case 2: // EXECUTORS + return EXECUTORS; + case 3: // TIME_SECS + return TIME_SECS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __TIME_SECS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STORM_ID, new org.apache.thrift.meta_data.FieldMetaData("storm_id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("executors", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class)))); + tmpMap.put(_Fields.TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorWorkerHeartbeat.class, metaDataMap); + } + + public SupervisorWorkerHeartbeat() { + } + + public SupervisorWorkerHeartbeat( + String storm_id, + List executors, + int time_secs) + { + this(); + this.storm_id = storm_id; + this.executors = executors; + this.time_secs = time_secs; + set_time_secs_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public SupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_storm_id()) { + this.storm_id = other.storm_id; + } + if (other.is_set_executors()) { + List __this__executors = new ArrayList(other.executors.size()); + for (ExecutorInfo other_element : other.executors) { + __this__executors.add(new ExecutorInfo(other_element)); + } + this.executors = __this__executors; + } + this.time_secs = other.time_secs; + } + + public SupervisorWorkerHeartbeat deepCopy() { + return new SupervisorWorkerHeartbeat(this); + } + + @Override + public void clear() { + this.storm_id = null; + this.executors = null; + set_time_secs_isSet(false); + this.time_secs = 0; + } + + public String get_storm_id() { + return this.storm_id; + } + + public void set_storm_id(String storm_id) { + this.storm_id = storm_id; + } + + public void unset_storm_id() { + this.storm_id = null; + } + + /** Returns true if field storm_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_storm_id() { + return this.storm_id != null; + } + + public void set_storm_id_isSet(boolean value) { + if (!value) { + this.storm_id = null; + } + } + + public int get_executors_size() { + return (this.executors == null) ? 0 : this.executors.size(); + } + + public java.util.Iterator get_executors_iterator() { + return (this.executors == null) ? null : this.executors.iterator(); + } + + public void add_to_executors(ExecutorInfo elem) { + if (this.executors == null) { + this.executors = new ArrayList(); + } + this.executors.add(elem); + } + + public List get_executors() { + return this.executors; + } + + public void set_executors(List executors) { + this.executors = executors; + } + + public void unset_executors() { + this.executors = null; + } + + /** Returns true if field executors is set (has been assigned a value) and false otherwise */ + public boolean is_set_executors() { + return this.executors != null; + } + + public void set_executors_isSet(boolean value) { + if (!value) { + this.executors = null; + } + } + + public int get_time_secs() { + return this.time_secs; + } + + public void set_time_secs(int time_secs) { + this.time_secs = time_secs; + set_time_secs_isSet(true); + } + + public void unset_time_secs() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_SECS_ISSET_ID); + } + + /** Returns true if field time_secs is set (has been assigned a value) and false otherwise */ + public boolean is_set_time_secs() { + return EncodingUtils.testBit(__isset_bitfield, __TIME_SECS_ISSET_ID); + } + + public void set_time_secs_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_SECS_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STORM_ID: + if (value == null) { + unset_storm_id(); + } else { + set_storm_id((String)value); + } + break; + + case EXECUTORS: + if (value == null) { + unset_executors(); + } else { + set_executors((List)value); + } + break; + + case TIME_SECS: + if (value == null) { + unset_time_secs(); + } else { + set_time_secs((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STORM_ID: + return get_storm_id(); + + case EXECUTORS: + return get_executors(); + + case TIME_SECS: + return get_time_secs(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STORM_ID: + return is_set_storm_id(); + case EXECUTORS: + return is_set_executors(); + case TIME_SECS: + return is_set_time_secs(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof SupervisorWorkerHeartbeat) + return this.equals((SupervisorWorkerHeartbeat)that); + return false; + } + + public boolean equals(SupervisorWorkerHeartbeat that) { + if (that == null) + return false; + + boolean this_present_storm_id = true && this.is_set_storm_id(); + boolean that_present_storm_id = true && that.is_set_storm_id(); + if (this_present_storm_id || that_present_storm_id) { + if (!(this_present_storm_id && that_present_storm_id)) + return false; + if (!this.storm_id.equals(that.storm_id)) + return false; + } + + boolean this_present_executors = true && this.is_set_executors(); + boolean that_present_executors = true && that.is_set_executors(); + if (this_present_executors || that_present_executors) { + if (!(this_present_executors && that_present_executors)) + return false; + if (!this.executors.equals(that.executors)) + return false; + } + + boolean this_present_time_secs = true; + boolean that_present_time_secs = true; + if (this_present_time_secs || that_present_time_secs) { + if (!(this_present_time_secs && that_present_time_secs)) + return false; + if (this.time_secs != that.time_secs) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_storm_id = true && (is_set_storm_id()); + list.add(present_storm_id); + if (present_storm_id) + list.add(storm_id); + + boolean present_executors = true && (is_set_executors()); + list.add(present_executors); + if (present_executors) + list.add(executors); + + boolean present_time_secs = true; + list.add(present_time_secs); + if (present_time_secs) + list.add(time_secs); + + return list.hashCode(); + } + + @Override + public int compareTo(SupervisorWorkerHeartbeat other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_storm_id()).compareTo(other.is_set_storm_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_storm_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storm_id, other.storm_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_executors()).compareTo(other.is_set_executors()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_executors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executors, other.executors); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_time_secs()).compareTo(other.is_set_time_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_time_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_secs, other.time_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("SupervisorWorkerHeartbeat("); + boolean first = true; + + sb.append("storm_id:"); + if (this.storm_id == null) { + sb.append("null"); + } else { + sb.append(this.storm_id); + } + first = false; + if (!first) sb.append(", "); + sb.append("executors:"); + if (this.executors == null) { + sb.append("null"); + } else { + sb.append(this.executors); + } + first = false; + if (!first) sb.append(", "); + sb.append("time_secs:"); + sb.append(this.time_secs); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_storm_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'storm_id' is unset! Struct:" + toString()); + } + + if (!is_set_executors()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'executors' is unset! Struct:" + toString()); + } + + if (!is_set_time_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_secs' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class SupervisorWorkerHeartbeatStandardSchemeFactory implements SchemeFactory { + public SupervisorWorkerHeartbeatStandardScheme getScheme() { + return new SupervisorWorkerHeartbeatStandardScheme(); + } + } + + private static class SupervisorWorkerHeartbeatStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorWorkerHeartbeat struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STORM_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.storm_id = iprot.readString(); + struct.set_storm_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // EXECUTORS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list870 = iprot.readListBegin(); + struct.executors = new ArrayList(_list870.size); + ExecutorInfo _elem871; + for (int _i872 = 0; _i872 < _list870.size; ++_i872) + { + _elem871 = new ExecutorInfo(); + _elem871.read(iprot); + struct.executors.add(_elem871); + } + iprot.readListEnd(); + } + struct.set_executors_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.time_secs = iprot.readI32(); + struct.set_time_secs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorWorkerHeartbeat struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.storm_id != null) { + oprot.writeFieldBegin(STORM_ID_FIELD_DESC); + oprot.writeString(struct.storm_id); + oprot.writeFieldEnd(); + } + if (struct.executors != null) { + oprot.writeFieldBegin(EXECUTORS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size())); + for (ExecutorInfo _iter873 : struct.executors) + { + _iter873.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(TIME_SECS_FIELD_DESC); + oprot.writeI32(struct.time_secs); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class SupervisorWorkerHeartbeatTupleSchemeFactory implements SchemeFactory { + public SupervisorWorkerHeartbeatTupleScheme getScheme() { + return new SupervisorWorkerHeartbeatTupleScheme(); + } + } + + private static class SupervisorWorkerHeartbeatTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorWorkerHeartbeat struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.storm_id); + { + oprot.writeI32(struct.executors.size()); + for (ExecutorInfo _iter874 : struct.executors) + { + _iter874.write(oprot); + } + } + oprot.writeI32(struct.time_secs); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorWorkerHeartbeat struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.storm_id = iprot.readString(); + struct.set_storm_id_isSet(true); + { + org.apache.thrift.protocol.TList _list875 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.executors = new ArrayList(_list875.size); + ExecutorInfo _elem876; + for (int _i877 = 0; _i877 < _list875.size; ++_i877) + { + _elem876 = new ExecutorInfo(); + _elem876.read(iprot); + struct.executors.add(_elem876); + } + } + struct.set_executors_isSet(true); + struct.time_secs = iprot.readI32(); + struct.set_time_secs_isSet(true); + } + } + +} + diff --git a/storm-client/src/jvm/org/apache/storm/generated/SupervisorWorkerHeartbeats.java b/storm-client/src/jvm/org/apache/storm/generated/SupervisorWorkerHeartbeats.java new file mode 100644 index 00000000000..28568691402 --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/generated/SupervisorWorkerHeartbeats.java @@ -0,0 +1,561 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class SupervisorWorkerHeartbeats implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorWorkerHeartbeats"); + + private static final org.apache.thrift.protocol.TField SUPERVISOR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor_id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField WORKER_HEARTBEATS_FIELD_DESC = new org.apache.thrift.protocol.TField("worker_heartbeats", org.apache.thrift.protocol.TType.LIST, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SupervisorWorkerHeartbeatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SupervisorWorkerHeartbeatsTupleSchemeFactory()); + } + + private String supervisor_id; // required + private List worker_heartbeats; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUPERVISOR_ID((short)1, "supervisor_id"), + WORKER_HEARTBEATS((short)2, "worker_heartbeats"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SUPERVISOR_ID + return SUPERVISOR_ID; + case 2: // WORKER_HEARTBEATS + return WORKER_HEARTBEATS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUPERVISOR_ID, new org.apache.thrift.meta_data.FieldMetaData("supervisor_id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.WORKER_HEARTBEATS, new org.apache.thrift.meta_data.FieldMetaData("worker_heartbeats", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorWorkerHeartbeat.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorWorkerHeartbeats.class, metaDataMap); + } + + public SupervisorWorkerHeartbeats() { + } + + public SupervisorWorkerHeartbeats( + String supervisor_id, + List worker_heartbeats) + { + this(); + this.supervisor_id = supervisor_id; + this.worker_heartbeats = worker_heartbeats; + } + + /** + * Performs a deep copy on other. + */ + public SupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats other) { + if (other.is_set_supervisor_id()) { + this.supervisor_id = other.supervisor_id; + } + if (other.is_set_worker_heartbeats()) { + List __this__worker_heartbeats = new ArrayList(other.worker_heartbeats.size()); + for (SupervisorWorkerHeartbeat other_element : other.worker_heartbeats) { + __this__worker_heartbeats.add(new SupervisorWorkerHeartbeat(other_element)); + } + this.worker_heartbeats = __this__worker_heartbeats; + } + } + + public SupervisorWorkerHeartbeats deepCopy() { + return new SupervisorWorkerHeartbeats(this); + } + + @Override + public void clear() { + this.supervisor_id = null; + this.worker_heartbeats = null; + } + + public String get_supervisor_id() { + return this.supervisor_id; + } + + public void set_supervisor_id(String supervisor_id) { + this.supervisor_id = supervisor_id; + } + + public void unset_supervisor_id() { + this.supervisor_id = null; + } + + /** Returns true if field supervisor_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_supervisor_id() { + return this.supervisor_id != null; + } + + public void set_supervisor_id_isSet(boolean value) { + if (!value) { + this.supervisor_id = null; + } + } + + public int get_worker_heartbeats_size() { + return (this.worker_heartbeats == null) ? 0 : this.worker_heartbeats.size(); + } + + public java.util.Iterator get_worker_heartbeats_iterator() { + return (this.worker_heartbeats == null) ? null : this.worker_heartbeats.iterator(); + } + + public void add_to_worker_heartbeats(SupervisorWorkerHeartbeat elem) { + if (this.worker_heartbeats == null) { + this.worker_heartbeats = new ArrayList(); + } + this.worker_heartbeats.add(elem); + } + + public List get_worker_heartbeats() { + return this.worker_heartbeats; + } + + public void set_worker_heartbeats(List worker_heartbeats) { + this.worker_heartbeats = worker_heartbeats; + } + + public void unset_worker_heartbeats() { + this.worker_heartbeats = null; + } + + /** Returns true if field worker_heartbeats is set (has been assigned a value) and false otherwise */ + public boolean is_set_worker_heartbeats() { + return this.worker_heartbeats != null; + } + + public void set_worker_heartbeats_isSet(boolean value) { + if (!value) { + this.worker_heartbeats = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUPERVISOR_ID: + if (value == null) { + unset_supervisor_id(); + } else { + set_supervisor_id((String)value); + } + break; + + case WORKER_HEARTBEATS: + if (value == null) { + unset_worker_heartbeats(); + } else { + set_worker_heartbeats((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUPERVISOR_ID: + return get_supervisor_id(); + + case WORKER_HEARTBEATS: + return get_worker_heartbeats(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUPERVISOR_ID: + return is_set_supervisor_id(); + case WORKER_HEARTBEATS: + return is_set_worker_heartbeats(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof SupervisorWorkerHeartbeats) + return this.equals((SupervisorWorkerHeartbeats)that); + return false; + } + + public boolean equals(SupervisorWorkerHeartbeats that) { + if (that == null) + return false; + + boolean this_present_supervisor_id = true && this.is_set_supervisor_id(); + boolean that_present_supervisor_id = true && that.is_set_supervisor_id(); + if (this_present_supervisor_id || that_present_supervisor_id) { + if (!(this_present_supervisor_id && that_present_supervisor_id)) + return false; + if (!this.supervisor_id.equals(that.supervisor_id)) + return false; + } + + boolean this_present_worker_heartbeats = true && this.is_set_worker_heartbeats(); + boolean that_present_worker_heartbeats = true && that.is_set_worker_heartbeats(); + if (this_present_worker_heartbeats || that_present_worker_heartbeats) { + if (!(this_present_worker_heartbeats && that_present_worker_heartbeats)) + return false; + if (!this.worker_heartbeats.equals(that.worker_heartbeats)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_supervisor_id = true && (is_set_supervisor_id()); + list.add(present_supervisor_id); + if (present_supervisor_id) + list.add(supervisor_id); + + boolean present_worker_heartbeats = true && (is_set_worker_heartbeats()); + list.add(present_worker_heartbeats); + if (present_worker_heartbeats) + list.add(worker_heartbeats); + + return list.hashCode(); + } + + @Override + public int compareTo(SupervisorWorkerHeartbeats other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_supervisor_id()).compareTo(other.is_set_supervisor_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_supervisor_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor_id, other.supervisor_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_worker_heartbeats()).compareTo(other.is_set_worker_heartbeats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_worker_heartbeats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.worker_heartbeats, other.worker_heartbeats); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("SupervisorWorkerHeartbeats("); + boolean first = true; + + sb.append("supervisor_id:"); + if (this.supervisor_id == null) { + sb.append("null"); + } else { + sb.append(this.supervisor_id); + } + first = false; + if (!first) sb.append(", "); + sb.append("worker_heartbeats:"); + if (this.worker_heartbeats == null) { + sb.append("null"); + } else { + sb.append(this.worker_heartbeats); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_supervisor_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisor_id' is unset! Struct:" + toString()); + } + + if (!is_set_worker_heartbeats()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'worker_heartbeats' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class SupervisorWorkerHeartbeatsStandardSchemeFactory implements SchemeFactory { + public SupervisorWorkerHeartbeatsStandardScheme getScheme() { + return new SupervisorWorkerHeartbeatsStandardScheme(); + } + } + + private static class SupervisorWorkerHeartbeatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorWorkerHeartbeats struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SUPERVISOR_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.supervisor_id = iprot.readString(); + struct.set_supervisor_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // WORKER_HEARTBEATS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list878 = iprot.readListBegin(); + struct.worker_heartbeats = new ArrayList(_list878.size); + SupervisorWorkerHeartbeat _elem879; + for (int _i880 = 0; _i880 < _list878.size; ++_i880) + { + _elem879 = new SupervisorWorkerHeartbeat(); + _elem879.read(iprot); + struct.worker_heartbeats.add(_elem879); + } + iprot.readListEnd(); + } + struct.set_worker_heartbeats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorWorkerHeartbeats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.supervisor_id != null) { + oprot.writeFieldBegin(SUPERVISOR_ID_FIELD_DESC); + oprot.writeString(struct.supervisor_id); + oprot.writeFieldEnd(); + } + if (struct.worker_heartbeats != null) { + oprot.writeFieldBegin(WORKER_HEARTBEATS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.worker_heartbeats.size())); + for (SupervisorWorkerHeartbeat _iter881 : struct.worker_heartbeats) + { + _iter881.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class SupervisorWorkerHeartbeatsTupleSchemeFactory implements SchemeFactory { + public SupervisorWorkerHeartbeatsTupleScheme getScheme() { + return new SupervisorWorkerHeartbeatsTupleScheme(); + } + } + + private static class SupervisorWorkerHeartbeatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorWorkerHeartbeats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.supervisor_id); + { + oprot.writeI32(struct.worker_heartbeats.size()); + for (SupervisorWorkerHeartbeat _iter882 : struct.worker_heartbeats) + { + _iter882.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorWorkerHeartbeats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.supervisor_id = iprot.readString(); + struct.set_supervisor_id_isSet(true); + { + org.apache.thrift.protocol.TList _list883 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.worker_heartbeats = new ArrayList(_list883.size); + SupervisorWorkerHeartbeat _elem884; + for (int _i885 = 0; _i885 < _list883.size; ++_i885) + { + _elem884 = new SupervisorWorkerHeartbeat(); + _elem884.read(iprot); + struct.worker_heartbeats.add(_elem884); + } + } + struct.set_worker_heartbeats_isSet(true); + } + } + +} + diff --git a/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricList.java b/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricList.java index 0bf95b5f61f..764a4f954d2 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricList.java +++ b/storm-client/src/jvm/org/apache/storm/generated/WorkerMetricList.java @@ -367,14 +367,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WorkerMetricList st case 1: // METRICS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list870 = iprot.readListBegin(); - struct.metrics = new ArrayList(_list870.size); - WorkerMetricPoint _elem871; - for (int _i872 = 0; _i872 < _list870.size; ++_i872) + org.apache.thrift.protocol.TList _list896 = iprot.readListBegin(); + struct.metrics = new ArrayList(_list896.size); + WorkerMetricPoint _elem897; + for (int _i898 = 0; _i898 < _list896.size; ++_i898) { - _elem871 = new WorkerMetricPoint(); - _elem871.read(iprot); - struct.metrics.add(_elem871); + _elem897 = new WorkerMetricPoint(); + _elem897.read(iprot); + struct.metrics.add(_elem897); } iprot.readListEnd(); } @@ -400,9 +400,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WorkerMetricList s oprot.writeFieldBegin(METRICS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.metrics.size())); - for (WorkerMetricPoint _iter873 : struct.metrics) + for (WorkerMetricPoint _iter899 : struct.metrics) { - _iter873.write(oprot); + _iter899.write(oprot); } oprot.writeListEnd(); } @@ -433,9 +433,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, WorkerMetricList st if (struct.is_set_metrics()) { { oprot.writeI32(struct.metrics.size()); - for (WorkerMetricPoint _iter874 : struct.metrics) + for (WorkerMetricPoint _iter900 : struct.metrics) { - _iter874.write(oprot); + _iter900.write(oprot); } } } @@ -447,14 +447,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, WorkerMetricList str BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list875 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.metrics = new ArrayList(_list875.size); - WorkerMetricPoint _elem876; - for (int _i877 = 0; _i877 < _list875.size; ++_i877) + org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.metrics = new ArrayList(_list901.size); + WorkerMetricPoint _elem902; + for (int _i903 = 0; _i903 < _list901.size; ++_i903) { - _elem876 = new WorkerMetricPoint(); - _elem876.read(iprot); - struct.metrics.add(_elem876); + _elem902 = new WorkerMetricPoint(); + _elem902.read(iprot); + struct.metrics.add(_elem902); } } struct.set_metrics_isSet(true); diff --git a/storm-client/src/jvm/org/apache/storm/generated/WorkerTokenServiceType.java b/storm-client/src/jvm/org/apache/storm/generated/WorkerTokenServiceType.java index 99c26a85f15..3602c9755ad 100644 --- a/storm-client/src/jvm/org/apache/storm/generated/WorkerTokenServiceType.java +++ b/storm-client/src/jvm/org/apache/storm/generated/WorkerTokenServiceType.java @@ -30,7 +30,8 @@ public enum WorkerTokenServiceType implements org.apache.thrift.TEnum { NIMBUS(0), - DRPC(1); + DRPC(1), + SUPERVISOR(2); private final int value; @@ -55,6 +56,8 @@ public static WorkerTokenServiceType findByValue(int value) { return NIMBUS; case 1: return DRPC; + case 2: + return SUPERVISOR; default: return null; } diff --git a/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java index f1aebd3641a..f8dcf56c96a 100644 --- a/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java +++ b/storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java @@ -295,7 +295,13 @@ private Scope calculateScope(Map taskToNodePort, Map getHostToRackMapping(Map taskToNodePort) { Set hosts = new HashSet(); for (int task: targetTasks) { - hosts.add(taskToNodePort.get(task).get_node()); + //if this task containing worker will be killed by a assignments sync, + //taskToNodePort will be an empty map which is refreshed by WorkerState + if (taskToNodePort.containsKey(task)) { + hosts.add(taskToNodePort.get(task).get_node()); + } else { + LOG.error("Could not find task NodeInfo from local cache."); + } } hosts.add(sourceNodeInfo.get_node()); return dnsToSwitchMapping.resolve(new ArrayList<>(hosts)); diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/IAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/IAuthorizer.java index 70f9e3295e9..d83cc723b58 100644 --- a/storm-client/src/jvm/org/apache/storm/security/auth/IAuthorizer.java +++ b/storm-client/src/jvm/org/apache/storm/security/auth/IAuthorizer.java @@ -37,10 +37,10 @@ */ public interface IAuthorizer { /** - * Invoked once immediately after construction - * @param topoConf Storm configuration + * Invoked once immediately after construction. + * @param conf Storm cluster configuration */ - void prepare(Map topoConf); + void prepare(Map conf); /** * permit() method is invoked for each incoming Thrift request. diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java index 398dbeb3f04..91d280c2956 100644 --- a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java +++ b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java @@ -29,8 +29,11 @@ */ public enum ThriftConnectionType { NIMBUS(Config.NIMBUS_THRIFT_TRANSPORT_PLUGIN, Config.NIMBUS_THRIFT_PORT, Config.NIMBUS_QUEUE_SIZE, - Config.NIMBUS_THRIFT_THREADS, Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE, Config.STORM_THRIFT_SOCKET_TIMEOUT_MS, + Config.NIMBUS_THRIFT_THREADS, Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE, Config.STORM_THRIFT_SOCKET_TIMEOUT_MS, WorkerTokenServiceType.NIMBUS), + SUPERVISOR(Config.SUPERVISOR_THRIFT_TRANSPORT_PLUGIN, Config.SUPERVISOR_THRIFT_PORT, Config.SUPERVISOR_QUEUE_SIZE, + Config.SUPERVISOR_THRIFT_THREADS, Config.SUPERVISOR_THRIFT_MAX_BUFFER_SIZE, + Config.SUPERVISOR_THRIFT_SOCKET_TIMEOUT_MS, WorkerTokenServiceType.SUPERVISOR), //A DRPC token only works for the invocations transport, not for the basic thrift transport. DRPC(Config.DRPC_THRIFT_TRANSPORT_PLUGIN, Config.DRPC_PORT, Config.DRPC_QUEUE_SIZE, Config.DRPC_WORKER_THREADS, Config.DRPC_MAX_BUFFER_SIZE, null, null), diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java index 05247f9dfe1..fcd32014298 100644 --- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java +++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java @@ -51,7 +51,9 @@ public class SimpleACLAuthorizer implements IAuthorizer { "getOwnerResourceSummaries")); protected Set supervisorCommands = new HashSet<>(Arrays.asList( "fileDownload", - "processWorkerMetrics")); + "processWorkerMetrics", + "getSupervisorAssignments", + "sendSupervisorWorkerHeartbeats")); protected Set topoReadOnlyCommands = new HashSet<>(Arrays.asList( "getTopologyConf", "getTopology", @@ -75,7 +77,8 @@ public class SimpleACLAuthorizer implements IAuthorizer { "dumpProfile", "dumpJstack", "dumpHeap", - "debug")); + "debug", + "sendSupervisorWorkerHeartbeat")); { topoCommands.addAll(topoReadOnlyCommands); diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SupervisorSimpleACLAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SupervisorSimpleACLAuthorizer.java new file mode 100644 index 00000000000..3ddb5734964 --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SupervisorSimpleACLAuthorizer.java @@ -0,0 +1,157 @@ +/** + * 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.security.auth.authorizer; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.apache.storm.Config; +import org.apache.storm.security.auth.AuthUtils; +import org.apache.storm.security.auth.IAuthorizer; +import org.apache.storm.security.auth.IGroupMappingServiceProvider; +import org.apache.storm.security.auth.IPrincipalToLocal; +import org.apache.storm.security.auth.ReqContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An authorization implementation that simply checks if a user is allowed to perform specific + * operations. + */ +public class SupervisorSimpleACLAuthorizer implements IAuthorizer { + private static final Logger LOG = LoggerFactory.getLogger(SupervisorSimpleACLAuthorizer.class); + + protected Set topoCommands = new HashSet<>(Arrays.asList( + "getLocalAssignmentForStorm", + "sendSupervisorWorkerHeartbeat")); + protected Set nimbusCommands = new HashSet<>(Arrays.asList( + "sendSupervisorAssignments")); + + protected Set admins; + protected Set adminsGroups; + protected Set nimbus; + protected IPrincipalToLocal ptol; + protected IGroupMappingServiceProvider groupMappingServiceProvider; + + /** + * Invoked once immediately after construction. + * @param conf Storm configuration + */ + @Override + public void prepare(Map conf) { + admins = new HashSet<>(); + adminsGroups = new HashSet<>(); + nimbus = new HashSet<>(); + + if (conf.containsKey(Config.NIMBUS_ADMINS)) { + admins.addAll((Collection)conf.get(Config.NIMBUS_ADMINS)); + } + + if (conf.containsKey(Config.NIMBUS_ADMINS_GROUPS)) { + adminsGroups.addAll((Collection)conf.get(Config.NIMBUS_ADMINS_GROUPS)); + } + + if (conf.containsKey(Config.NIMBUS_DAEMON_USERS)) { + nimbus.addAll((Collection)conf.get(Config.NIMBUS_DAEMON_USERS)); + } else if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) { + LOG.warn("{} is not set falling back to using {}.", Config.NIMBUS_DAEMON_USERS, Config.NIMBUS_SUPERVISOR_USERS); + //In almost all cases these should be the same, but warn the user just in case something goes wrong... + nimbus.addAll((Collection)conf.get(Config.NIMBUS_SUPERVISOR_USERS)); + } else { + //If it is not set a lot of things are not really going to work all that well + LOG.error("Could not find {} things might now work correctly...", Config.NIMBUS_DAEMON_USERS); + } + + ptol = AuthUtils.GetPrincipalToLocalPlugin(conf); + groupMappingServiceProvider = AuthUtils.GetGroupMappingServiceProviderPlugin(conf); + } + + /** + * permit() method is invoked for each incoming Thrift request. + * @param context request context includes info about + * @param operation operation name + * @param topoConf configuration of targeted topology + * @return true if the request is authorized, false if reject + */ + @Override + public boolean permit(ReqContext context, String operation, Map topoConf) { + String principal = context.principal().getName(); + String user = ptol.toLocal(context.principal()); + Set userGroups = new HashSet<>(); + + if (groupMappingServiceProvider != null) { + try { + userGroups = groupMappingServiceProvider.getGroups(user); + } catch (IOException e) { + LOG.warn("Error while trying to fetch user groups",e); + } + } + + if (admins.contains(principal) || admins.contains(user) || checkUserGroupAllowed(userGroups, adminsGroups)) { + return true; + } + + if (nimbus.contains(principal) || nimbus.contains(user)) { + return nimbusCommands.contains(operation); + } + + if (topoCommands.contains(operation)) { + if (topoConf != null) { + if (checkTopoPermission(principal, user, userGroups, topoConf, Config.TOPOLOGY_USERS, Config.TOPOLOGY_GROUPS)) { + return true; + } + } + } + return false; + } + + private Boolean checkTopoPermission(String principal, String user, Set userGroups, + Map topoConf, String userConfigKey, String groupConfigKey) { + Set configuredUsers = new HashSet<>(); + + if (topoConf.containsKey(userConfigKey)) { + configuredUsers.addAll((Collection)topoConf.get(userConfigKey)); + } + + if (configuredUsers.contains(principal) || configuredUsers.contains(user)) { + return true; + } + + Set configuredGroups = new HashSet<>(); + if (topoConf.containsKey(groupConfigKey) && topoConf.get(groupConfigKey) != null) { + configuredGroups.addAll((Collection)topoConf.get(groupConfigKey)); + } + + return checkUserGroupAllowed(userGroups, configuredGroups); + } + + private Boolean checkUserGroupAllowed(Set userGroups, Set configuredGroups) { + if (userGroups.size() > 0 && configuredGroups.size() > 0) { + for (String tgroup : configuredGroups) { + if (userGroups.contains(tgroup)) { + return true; + } + } + } + return false; + } +} \ No newline at end of file diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ServerCallbackHandler.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ServerCallbackHandler.java index d3157c8aebd..4da8652f579 100644 --- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ServerCallbackHandler.java +++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ServerCallbackHandler.java @@ -83,7 +83,8 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException { if (ac != null) { String authenticationID = ac.getAuthenticationID(); - LOG.info("Successfully authenticated client: authenticationID=" + authenticationID + " authorizationID= " + ac.getAuthorizationID()); + LOG.debug("Successfully authenticated client: authenticationID={} authorizationID= {}", authenticationID, + ac.getAuthorizationID()); //if authorizationId is not set, set it to authenticationId. if (ac.getAuthorizationID() == null) { diff --git a/storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java index e712804559d..965084f263e 100644 --- a/storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java +++ b/storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java @@ -19,6 +19,16 @@ import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + import org.apache.storm.cluster.ExecutorBeat; import org.apache.storm.cluster.IStormClusterState; import org.apache.storm.generated.Bolt; @@ -40,6 +50,7 @@ import org.apache.storm.generated.SpoutAggregateStats; import org.apache.storm.generated.SpoutStats; import org.apache.storm.generated.StormTopology; +import org.apache.storm.generated.SupervisorWorkerHeartbeat; import org.apache.storm.generated.TopologyPageInfo; import org.apache.storm.generated.TopologyStats; import org.apache.storm.generated.WorkerResources; @@ -50,15 +61,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - @SuppressWarnings("unchecked") public class StatsUtil { private static final Logger LOG = LoggerFactory.getLogger(StatsUtil.class); @@ -1255,11 +1257,11 @@ public static ComponentPageInfo aggCompExecsStats( /** * aggregate statistics per worker for a topology. Optionally filtering on specific supervisors * - * @param topologyId topology id - * @param topology storm topology - * @param task2component a Map of {task id -> component} + * @param stormId topology id + * @param stormName storm topology + * @param task2Component a Map of {task id -> component} * @param beats a converted HashMap of executor heartbeats, {executor -> heartbeat} - * @param exec2hostPort a Map of {executor -> host+port} + * @param exec2NodePort a Map of {executor -> host+port} * @param includeSys whether to include system streams * @param userAuthorized whether the user is authorized to view topology info * @param filterSupervisor if not null, only return WorkerSummaries for that supervisor @@ -1318,10 +1320,7 @@ public static List aggWorkerStats(String stormId, String stormNam int hbeatSecs = 0; if (beats != null) { Map beat = beats.get(convertExecutor(exec)); - if (beat != null) { - Map hbeat = (Map)beat.get("heartbeat"); - hbeatSecs = hbeat == null ? 0 : (int) hbeat.get("uptime"); - } + hbeatSecs = beat == null ? 0 : (int) beat.get("uptime"); } ws.set_uptime_secs(hbeatSecs); ws.set_num_executors(ws.get_num_executors() + 1); @@ -1360,11 +1359,11 @@ public static List aggWorkerStats(String stormId, String stormNam * Convenience overload when called from the topology page code (in that case we want data * for all workers in the topology, not filtered by supervisor) * - * @param topologyId topology id - * @param topology storm topology - * @param task2component a Map of {task id -> component} + * @param stormId topology id + * @param stormName storm topology + * @param task2Component a Map of {task id -> component} * @param beats a converted HashMap of executor heartbeats, {executor -> heartbeat} - * @param exec2hostPort a Map of {executor -> host+port} + * @param exec2NodePort a Map of {executor -> host+port} * @param includeSys whether to include system streams * @param userAuthorized whether the user is authorized to view topology info * @@ -1401,6 +1400,23 @@ public static Map, Map> convertExecutorBeats(Map, Map> convertWorkerBeats(SupervisorWorkerHeartbeat workerHeartbeat) { + Map, Map> ret = new HashMap<>(); + for(ExecutorInfo executorInfo: workerHeartbeat.get_executors()) { + Map reportBeat = new HashMap<>(); + reportBeat.put(TIME_SECS, workerHeartbeat.get_time_secs()); + ret.put(Lists.newArrayList(executorInfo.get_task_start(), executorInfo.get_task_end()), + reportBeat); + } + + return ret; + } + /** * convert thrift ExecutorBeat into a java HashMap */ @@ -1532,14 +1548,8 @@ public static List> extractNodeInfosFromHbForComp( return ret; } - - // ===================================================================================== - // heartbeats related - // ===================================================================================== - /** - * update all executor heart beats - * TODO: should move this method to nimbus when nimbus.clj is translated + * update all executor heart beats (legacy ZK heartbeat compatibility) * * @param cache existing heart beats cache * @param executorBeats new heart beats @@ -1547,9 +1557,8 @@ public static List> extractNodeInfosFromHbForComp( * @param timeout timeout * @return a HashMap of updated executor heart beats */ - public static Map, Map> updateHeartbeatCache(Map, Map> cache, - Map, Map> executorBeats, - Set> executors, Integer timeout) { + public static Map, Map> updateHeartbeatCacheFromZkHeartbeat(Map, Map> cache, + Map, Map> executorBeats, Set> executors, Integer timeout) { Map, Map> ret = new HashMap<>(); if (cache == null && executorBeats == null) { return ret; @@ -1569,6 +1578,36 @@ public static Map, Map> updateHeartbeatCache(Map, Map> cache, + Map, Map> executorBeats, Set> executors, Integer timeout) { + //if not executor beats, refresh is-timed-out of the cache which is done by master + if (executorBeats == null) { + for (Map.Entry, Map> executorbeat: cache.entrySet()) { + Map beat = executorbeat.getValue(); + beat.put("is-timed-out", Time.deltaSecs((Integer)beat.get("nimbus-time")) >= timeout); + } + return; + } + //else refresh nimbus-time and executor-reported-time by heartbeats reporting + for (List executor : executors) { + cache.put(executor, updateExecutorCache(cache.get(executor), executorBeats.get(executor), timeout)); + } + } + // TODO: should move this method to nimbus when nimbus.clj is translated public static Map updateExecutorCache( Map currBeat, Map newBeat, Integer timeout) { @@ -1589,7 +1628,7 @@ public static Map updateExecutorCache( if (lastReportedTime != null) { reportedTime = lastReportedTime; } else { - reportedTime = 0; + reportedTime = lastReportedTime = 0; } } @@ -1600,7 +1639,6 @@ public static Map updateExecutorCache( ret.put("is-timed-out", Time.deltaSecs(lastNimbusTime) >= timeout); ret.put("nimbus-time", lastNimbusTime); ret.put("executor-reported-time", reportedTime); - ret.put(HEARTBEAT, newBeat); return ret; } @@ -1647,20 +1685,17 @@ public static List> extractDataFromHb(Map executor2hostPort, putKV(m, HOST, host); putKV(m, PORT, port); - Map hb = getMapByKey(beat, HEARTBEAT); - if (hb != null) { - Map stats = getMapByKey(hb, STATS); - putKV(m, UPTIME, hb.get(UPTIME)); - putKV(m, STATS, stats); - - String type = componentType(topology, compId); - if (type != null) { - putKV(m, TYPE, type); - } else { - putKV(m, TYPE, stats.get(TYPE)); - } - ret.add(m); + Map stats = getMapByKey(beat, STATS); + putKV(m, UPTIME, beat.get(UPTIME)); + putKV(m, STATS, stats); + + String type = componentType(topology, compId); + if (type != null) { + putKV(m, TYPE, type); + } else { + putKV(m, TYPE, stats.get(TYPE)); } + ret.add(m); } } return ret; @@ -1815,7 +1850,7 @@ private static Map mergeMaps(Map m1, Map m2) { /** * filter system streams from stats * - * @param stats { stream id -> value } + * @param stream2stat { stream id -> value } * @param includeSys whether to filter system streams * @return filtered stats */ @@ -2018,6 +2053,18 @@ public static ClusterWorkerHeartbeat thriftifyZkWorkerHb(Map hea return ret; } + /** + * Used for local test. + * @return + */ + public static SupervisorWorkerHeartbeat thriftifyRPCWorkerHb(String stormId, List executorId) { + SupervisorWorkerHeartbeat supervisorWorkerHeartbeat = new SupervisorWorkerHeartbeat(); + supervisorWorkerHeartbeat.set_storm_id(stormId); + supervisorWorkerHeartbeat.set_executors(Collections.singletonList(new ExecutorInfo(executorId.get(0).intValue(), executorId.get(1).intValue()))); + supervisorWorkerHeartbeat.set_time_secs(Time.currentTimeSecs()); + return supervisorWorkerHeartbeat; + } + private static ComponentAggregateStats thriftifySpoutAggStats(Map m) { ComponentAggregateStats stats = new ComponentAggregateStats(); stats.set_type(ComponentType.SPOUT); 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 e10bc440bec..ccde5057207 100644 --- a/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java +++ b/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java @@ -15,8 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.topology; +import java.io.NotSerializableException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + import org.apache.storm.Config; import org.apache.storm.generated.Bolt; import org.apache.storm.generated.ComponentCommon; @@ -24,10 +34,10 @@ import org.apache.storm.generated.GlobalStreamId; import org.apache.storm.generated.Grouping; import org.apache.storm.generated.NullStruct; +import org.apache.storm.generated.SharedMemory; import org.apache.storm.generated.SpoutSpec; import org.apache.storm.generated.StateSpoutSpec; import org.apache.storm.generated.StormTopology; -import org.apache.storm.generated.SharedMemory; import org.apache.storm.grouping.CustomStreamGrouping; import org.apache.storm.grouping.PartialKeyGrouping; import org.apache.storm.hooks.IWorkerHook; @@ -47,15 +57,6 @@ import org.apache.storm.windowing.TupleWindow; import org.json.simple.JSONValue; -import java.io.NotSerializableException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_COMPONENT_ID; import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID; import static org.apache.storm.utils.Utils.parseJson; diff --git a/storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java b/storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java new file mode 100644 index 00000000000..469e57f7870 --- /dev/null +++ b/storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java @@ -0,0 +1,88 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.utils; + +import org.apache.storm.Config; +import org.apache.storm.generated.Supervisor; +import org.apache.storm.security.auth.ThriftClient; +import org.apache.storm.security.auth.ThriftConnectionType; +import org.apache.thrift.transport.TTransportException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** + * Client for interacting with Supervisor server, now we use supervisor server mainly for cases below: + *

    + *
  • worker <- supervisor: get worker local assignment for a storm.
  • + *
  • nimbus -> supervisor: assign assignments for a node.
  • + *
+ */ +public class SupervisorClient extends ThriftClient { + private Supervisor.Client client; + private static final Logger LOG = LoggerFactory.getLogger(SupervisorClient.class); + + public static SupervisorClient getConfiguredClient(Map conf, String host) { + //use the default server port. + int port = Integer.parseInt(conf.get(Config.SUPERVISOR_THRIFT_PORT).toString()); + return getConfiguredClientAs(conf, host, port, null); + } + + public static SupervisorClient getConfiguredClient(Map conf, String host, int port) { + return getConfiguredClientAs(conf, host, port, null); + } + + public static SupervisorClient getConfiguredClientAs(Map conf, String host, int port, String asUser) { + if (conf.containsKey(Config.STORM_DO_AS_USER)) { + if (asUser != null && !asUser.isEmpty()) { + LOG.warn("You have specified a doAsUser as param {} and a doAsParam as config, config will take precedence." + , asUser, conf.get(Config.STORM_DO_AS_USER)); + } + asUser = (String) conf.get(Config.STORM_DO_AS_USER); + } + try { + return new SupervisorClient(conf, host, port, null, asUser); + } catch (TTransportException e) { + throw new RuntimeException("Failed to create a supervisor client for host " + host); + } + } + + public SupervisorClient(Map conf, String host, int port) throws TTransportException { + this(conf, host, port, null, null); + } + + public SupervisorClient(Map conf, String host, int port, Integer timeout) throws TTransportException { + super(conf, ThriftConnectionType.SUPERVISOR, host, port, timeout, null); + client = new Supervisor.Client(_protocol); + } + + public SupervisorClient(Map conf, String host, Integer port, Integer timeout, String asUser) throws TTransportException { + super(conf, ThriftConnectionType.SUPERVISOR, host, port, timeout, asUser); + client = new Supervisor.Client(_protocol); + } + + public SupervisorClient(Map conf, String host) throws TTransportException { + super(conf, ThriftConnectionType.SUPERVISOR, host, null, null, null); + client = new Supervisor.Client(_protocol); + } + + public Supervisor.Client getClient() { + return client; + } +} diff --git a/storm-client/src/py/storm/Nimbus-remote b/storm-client/src/py/storm/Nimbus-remote index 1ce91e462ee..603b62890a9 100644 --- a/storm-client/src/py/storm/Nimbus-remote +++ b/storm-client/src/py/storm/Nimbus-remote @@ -88,6 +88,9 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help': print(' StormTopology getUserTopology(string id)') print(' TopologyHistoryInfo getTopologyHistory(string user)') print(' getOwnerResourceSummaries(string owner)') + print(' SupervisorAssignments getSupervisorAssignments(string node)') + print(' void sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats)') + print(' void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heatbeat)') print(' void processWorkerMetrics(WorkerMetrics metrics)') print('') sys.exit(0) @@ -421,6 +424,24 @@ elif cmd == 'getOwnerResourceSummaries': sys.exit(1) pp.pprint(client.getOwnerResourceSummaries(args[0],)) +elif cmd == 'getSupervisorAssignments': + if len(args) != 1: + print('getSupervisorAssignments requires 1 args') + sys.exit(1) + pp.pprint(client.getSupervisorAssignments(args[0],)) + +elif cmd == 'sendSupervisorWorkerHeartbeats': + if len(args) != 1: + print('sendSupervisorWorkerHeartbeats requires 1 args') + sys.exit(1) + pp.pprint(client.sendSupervisorWorkerHeartbeats(eval(args[0]),)) + +elif cmd == 'sendSupervisorWorkerHeartbeat': + if len(args) != 1: + print('sendSupervisorWorkerHeartbeat requires 1 args') + sys.exit(1) + pp.pprint(client.sendSupervisorWorkerHeartbeat(eval(args[0]),)) + elif cmd == 'processWorkerMetrics': if len(args) != 1: print('processWorkerMetrics requires 1 args') diff --git a/storm-client/src/py/storm/Nimbus.py b/storm-client/src/py/storm/Nimbus.py index 5f9f324c58c..1e9460ded4e 100644 --- a/storm-client/src/py/storm/Nimbus.py +++ b/storm-client/src/py/storm/Nimbus.py @@ -383,6 +383,33 @@ def getOwnerResourceSummaries(self, owner): """ pass + def getSupervisorAssignments(self, node): + """ + Get assigned assignments for a specific supervisor + + Parameters: + - node + """ + pass + + def sendSupervisorWorkerHeartbeats(self, heartbeats): + """ + Send supervisor worker heartbeats for a specific supervisor + + Parameters: + - heartbeats + """ + pass + + def sendSupervisorWorkerHeartbeat(self, heatbeat): + """ + Send supervisor local worker heartbeat when a supervisor is unreachable + + Parameters: + - heatbeat + """ + pass + def processWorkerMetrics(self, metrics): """ Parameters: @@ -1974,6 +2001,107 @@ def recv_getOwnerResourceSummaries(self): raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "getOwnerResourceSummaries failed: unknown result") + def getSupervisorAssignments(self, node): + """ + Get assigned assignments for a specific supervisor + + Parameters: + - node + """ + self.send_getSupervisorAssignments(node) + return self.recv_getSupervisorAssignments() + + def send_getSupervisorAssignments(self, node): + self._oprot.writeMessageBegin('getSupervisorAssignments', TMessageType.CALL, self._seqid) + args = getSupervisorAssignments_args() + args.node = node + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getSupervisorAssignments(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getSupervisorAssignments_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.aze is not None: + raise result.aze + raise TApplicationException(TApplicationException.MISSING_RESULT, "getSupervisorAssignments failed: unknown result") + + def sendSupervisorWorkerHeartbeats(self, heartbeats): + """ + Send supervisor worker heartbeats for a specific supervisor + + Parameters: + - heartbeats + """ + self.send_sendSupervisorWorkerHeartbeats(heartbeats) + self.recv_sendSupervisorWorkerHeartbeats() + + def send_sendSupervisorWorkerHeartbeats(self, heartbeats): + self._oprot.writeMessageBegin('sendSupervisorWorkerHeartbeats', TMessageType.CALL, self._seqid) + args = sendSupervisorWorkerHeartbeats_args() + args.heartbeats = heartbeats + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_sendSupervisorWorkerHeartbeats(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = sendSupervisorWorkerHeartbeats_result() + result.read(iprot) + iprot.readMessageEnd() + if result.aze is not None: + raise result.aze + return + + def sendSupervisorWorkerHeartbeat(self, heatbeat): + """ + Send supervisor local worker heartbeat when a supervisor is unreachable + + Parameters: + - heatbeat + """ + self.send_sendSupervisorWorkerHeartbeat(heatbeat) + self.recv_sendSupervisorWorkerHeartbeat() + + def send_sendSupervisorWorkerHeartbeat(self, heatbeat): + self._oprot.writeMessageBegin('sendSupervisorWorkerHeartbeat', TMessageType.CALL, self._seqid) + args = sendSupervisorWorkerHeartbeat_args() + args.heatbeat = heatbeat + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_sendSupervisorWorkerHeartbeat(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = sendSupervisorWorkerHeartbeat_result() + result.read(iprot) + iprot.readMessageEnd() + if result.aze is not None: + raise result.aze + return + def processWorkerMetrics(self, metrics): """ Parameters: @@ -2054,6 +2182,9 @@ def __init__(self, handler): self._processMap["getUserTopology"] = Processor.process_getUserTopology self._processMap["getTopologyHistory"] = Processor.process_getTopologyHistory self._processMap["getOwnerResourceSummaries"] = Processor.process_getOwnerResourceSummaries + self._processMap["getSupervisorAssignments"] = Processor.process_getSupervisorAssignments + self._processMap["sendSupervisorWorkerHeartbeats"] = Processor.process_sendSupervisorWorkerHeartbeats + self._processMap["sendSupervisorWorkerHeartbeat"] = Processor.process_sendSupervisorWorkerHeartbeat self._processMap["processWorkerMetrics"] = Processor.process_processWorkerMetrics def process(self, iprot, oprot): @@ -3152,6 +3283,72 @@ def process_getOwnerResourceSummaries(self, seqid, iprot, oprot): oprot.writeMessageEnd() oprot.trans.flush() + def process_getSupervisorAssignments(self, seqid, iprot, oprot): + args = getSupervisorAssignments_args() + args.read(iprot) + iprot.readMessageEnd() + result = getSupervisorAssignments_result() + try: + result.success = self._handler.getSupervisorAssignments(args.node) + msg_type = TMessageType.REPLY + except (TTransport.TTransportException, KeyboardInterrupt, SystemExit): + raise + except AuthorizationException as aze: + msg_type = TMessageType.REPLY + result.aze = aze + except Exception as ex: + msg_type = TMessageType.EXCEPTION + logging.exception(ex) + result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error') + oprot.writeMessageBegin("getSupervisorAssignments", msg_type, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_sendSupervisorWorkerHeartbeats(self, seqid, iprot, oprot): + args = sendSupervisorWorkerHeartbeats_args() + args.read(iprot) + iprot.readMessageEnd() + result = sendSupervisorWorkerHeartbeats_result() + try: + self._handler.sendSupervisorWorkerHeartbeats(args.heartbeats) + msg_type = TMessageType.REPLY + except (TTransport.TTransportException, KeyboardInterrupt, SystemExit): + raise + except AuthorizationException as aze: + msg_type = TMessageType.REPLY + result.aze = aze + except Exception as ex: + msg_type = TMessageType.EXCEPTION + logging.exception(ex) + result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error') + oprot.writeMessageBegin("sendSupervisorWorkerHeartbeats", msg_type, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_sendSupervisorWorkerHeartbeat(self, seqid, iprot, oprot): + args = sendSupervisorWorkerHeartbeat_args() + args.read(iprot) + iprot.readMessageEnd() + result = sendSupervisorWorkerHeartbeat_result() + try: + self._handler.sendSupervisorWorkerHeartbeat(args.heatbeat) + msg_type = TMessageType.REPLY + except (TTransport.TTransportException, KeyboardInterrupt, SystemExit): + raise + except AuthorizationException as aze: + msg_type = TMessageType.REPLY + result.aze = aze + except Exception as ex: + msg_type = TMessageType.EXCEPTION + logging.exception(ex) + result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error') + oprot.writeMessageBegin("sendSupervisorWorkerHeartbeat", msg_type, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + def process_processWorkerMetrics(self, seqid, iprot, oprot): args = processWorkerMetrics_args() args.read(iprot) @@ -5033,11 +5230,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype801, _size798) = iprot.readListBegin() - for _i802 in xrange(_size798): - _elem803 = ProfileRequest() - _elem803.read(iprot) - self.success.append(_elem803) + (_etype824, _size821) = iprot.readListBegin() + for _i825 in xrange(_size821): + _elem826 = ProfileRequest() + _elem826.read(iprot) + self.success.append(_elem826) iprot.readListEnd() else: iprot.skip(ftype) @@ -5054,8 +5251,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 iter804 in self.success: - iter804.write(oprot) + for iter827 in self.success: + iter827.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -10126,11 +10323,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype808, _size805) = iprot.readListBegin() - for _i809 in xrange(_size805): - _elem810 = OwnerResourceSummary() - _elem810.read(iprot) - self.success.append(_elem810) + (_etype831, _size828) = iprot.readListBegin() + for _i832 in xrange(_size828): + _elem833 = OwnerResourceSummary() + _elem833.read(iprot) + self.success.append(_elem833) iprot.readListEnd() else: iprot.skip(ftype) @@ -10153,8 +10350,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 iter811 in self.success: - iter811.write(oprot) + for iter834 in self.success: + iter834.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.aze is not None: @@ -10185,6 +10382,414 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) +class getSupervisorAssignments_args: + """ + Attributes: + - node + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'node', None, None, ), # 1 + ) + + def __init__(self, node=None,): + self.node = node + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.node = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getSupervisorAssignments_args') + if self.node is not None: + oprot.writeFieldBegin('node', TType.STRING, 1) + oprot.writeString(self.node.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.node) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getSupervisorAssignments_result: + """ + Attributes: + - success + - aze + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (SupervisorAssignments, SupervisorAssignments.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, aze=None,): + self.success = success + self.aze = aze + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = SupervisorAssignments() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getSupervisorAssignments_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.aze) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class sendSupervisorWorkerHeartbeats_args: + """ + Attributes: + - heartbeats + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'heartbeats', (SupervisorWorkerHeartbeats, SupervisorWorkerHeartbeats.thrift_spec), None, ), # 1 + ) + + def __init__(self, heartbeats=None,): + self.heartbeats = heartbeats + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.heartbeats = SupervisorWorkerHeartbeats() + self.heartbeats.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('sendSupervisorWorkerHeartbeats_args') + if self.heartbeats is not None: + oprot.writeFieldBegin('heartbeats', TType.STRUCT, 1) + self.heartbeats.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.heartbeats) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class sendSupervisorWorkerHeartbeats_result: + """ + Attributes: + - aze + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 + ) + + def __init__(self, aze=None,): + self.aze = aze + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('sendSupervisorWorkerHeartbeats_result') + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.aze) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class sendSupervisorWorkerHeartbeat_args: + """ + Attributes: + - heatbeat + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'heatbeat', (SupervisorWorkerHeartbeat, SupervisorWorkerHeartbeat.thrift_spec), None, ), # 1 + ) + + def __init__(self, heatbeat=None,): + self.heatbeat = heatbeat + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.heatbeat = SupervisorWorkerHeartbeat() + self.heatbeat.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('sendSupervisorWorkerHeartbeat_args') + if self.heatbeat is not None: + oprot.writeFieldBegin('heatbeat', TType.STRUCT, 1) + self.heatbeat.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.heatbeat) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class sendSupervisorWorkerHeartbeat_result: + """ + Attributes: + - aze + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 + ) + + def __init__(self, aze=None,): + self.aze = aze + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('sendSupervisorWorkerHeartbeat_result') + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.aze) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + class processWorkerMetrics_args: """ Attributes: diff --git a/storm-client/src/py/storm/Supervisor-remote b/storm-client/src/py/storm/Supervisor-remote new file mode 100644 index 00000000000..f0b6f98e9d8 --- /dev/null +++ b/storm-client/src/py/storm/Supervisor-remote @@ -0,0 +1,126 @@ +#!/usr/bin/env python +# +# 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. + +#!/usr/bin/env python +# +# Autogenerated by Thrift Compiler (0.9.3) +# +# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING +# +# options string: py:utf8strings +# + +import sys +import pprint +from urlparse import urlparse +from thrift.transport import TTransport +from thrift.transport import TSocket +from thrift.transport import TSSLSocket +from thrift.transport import THttpClient +from thrift.protocol import TBinaryProtocol + +from storm import Supervisor +from storm.ttypes import * + +if len(sys.argv) <= 1 or sys.argv[1] == '--help': + print('') + print('Usage: ' + sys.argv[0] + ' [-h host[:port]] [-u url] [-f[ramed]] [-s[sl]] function [arg1 [arg2...]]') + print('') + print('Functions:') + print(' void sendSupervisorAssignments(SupervisorAssignments assignments)') + print(' Assignment getLocalAssignmentForStorm(string id)') + print(' void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heartbeat)') + print('') + sys.exit(0) + +pp = pprint.PrettyPrinter(indent = 2) +host = 'localhost' +port = 9090 +uri = '' +framed = False +ssl = False +http = False +argi = 1 + +if sys.argv[argi] == '-h': + parts = sys.argv[argi+1].split(':') + host = parts[0] + if len(parts) > 1: + port = int(parts[1]) + argi += 2 + +if sys.argv[argi] == '-u': + url = urlparse(sys.argv[argi+1]) + parts = url[1].split(':') + host = parts[0] + if len(parts) > 1: + port = int(parts[1]) + else: + port = 80 + uri = url[2] + if url[4]: + uri += '?%s' % url[4] + http = True + argi += 2 + +if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': + framed = True + argi += 1 + +if sys.argv[argi] == '-s' or sys.argv[argi] == '-ssl': + ssl = True + argi += 1 + +cmd = sys.argv[argi] +args = sys.argv[argi+1:] + +if http: + transport = THttpClient.THttpClient(host, port, uri) +else: + socket = TSSLSocket.TSSLSocket(host, port, validate=False) if ssl else TSocket.TSocket(host, port) + if framed: + transport = TTransport.TFramedTransport(socket) + else: + transport = TTransport.TBufferedTransport(socket) +protocol = TBinaryProtocol.TBinaryProtocol(transport) +client = Supervisor.Client(protocol) +transport.open() + +if cmd == 'sendSupervisorAssignments': + if len(args) != 1: + print('sendSupervisorAssignments requires 1 args') + sys.exit(1) + pp.pprint(client.sendSupervisorAssignments(eval(args[0]),)) + +elif cmd == 'getLocalAssignmentForStorm': + if len(args) != 1: + print('getLocalAssignmentForStorm requires 1 args') + sys.exit(1) + pp.pprint(client.getLocalAssignmentForStorm(args[0],)) + +elif cmd == 'sendSupervisorWorkerHeartbeat': + if len(args) != 1: + print('sendSupervisorWorkerHeartbeat requires 1 args') + sys.exit(1) + pp.pprint(client.sendSupervisorWorkerHeartbeat(eval(args[0]),)) + +else: + print('Unrecognized method %s' % cmd) + sys.exit(1) + +transport.close() diff --git a/storm-client/src/py/storm/Supervisor.py b/storm-client/src/py/storm/Supervisor.py new file mode 100644 index 00000000000..fbc573f65e4 --- /dev/null +++ b/storm-client/src/py/storm/Supervisor.py @@ -0,0 +1,694 @@ +#!/usr/bin/env python +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# +# Autogenerated by Thrift Compiler (0.9.3) +# +# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING +# +# options string: py:utf8strings +# + +from thrift.Thrift import TType, TMessageType, TException, TApplicationException +import logging +from ttypes import * +from thrift.Thrift import TProcessor +from thrift.transport import TTransport +from thrift.protocol import TBinaryProtocol, TProtocol +try: + from thrift.protocol import fastbinary +except: + fastbinary = None + + +class Iface: + def sendSupervisorAssignments(self, assignments): + """ + Send node specific assignments to supervisor + + Parameters: + - assignments + """ + pass + + def getLocalAssignmentForStorm(self, id): + """ + Get local assignment for a storm + + Parameters: + - id + """ + pass + + def sendSupervisorWorkerHeartbeat(self, heartbeat): + """ + Send worker heartbeat to local supervisor + + Parameters: + - heartbeat + """ + pass + + +class Client(Iface): + def __init__(self, iprot, oprot=None): + self._iprot = self._oprot = iprot + if oprot is not None: + self._oprot = oprot + self._seqid = 0 + + def sendSupervisorAssignments(self, assignments): + """ + Send node specific assignments to supervisor + + Parameters: + - assignments + """ + self.send_sendSupervisorAssignments(assignments) + self.recv_sendSupervisorAssignments() + + def send_sendSupervisorAssignments(self, assignments): + self._oprot.writeMessageBegin('sendSupervisorAssignments', TMessageType.CALL, self._seqid) + args = sendSupervisorAssignments_args() + args.assignments = assignments + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_sendSupervisorAssignments(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = sendSupervisorAssignments_result() + result.read(iprot) + iprot.readMessageEnd() + if result.aze is not None: + raise result.aze + return + + def getLocalAssignmentForStorm(self, id): + """ + Get local assignment for a storm + + Parameters: + - id + """ + self.send_getLocalAssignmentForStorm(id) + return self.recv_getLocalAssignmentForStorm() + + def send_getLocalAssignmentForStorm(self, id): + self._oprot.writeMessageBegin('getLocalAssignmentForStorm', TMessageType.CALL, self._seqid) + args = getLocalAssignmentForStorm_args() + args.id = id + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getLocalAssignmentForStorm(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getLocalAssignmentForStorm_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + if result.aze is not None: + raise result.aze + raise TApplicationException(TApplicationException.MISSING_RESULT, "getLocalAssignmentForStorm failed: unknown result") + + def sendSupervisorWorkerHeartbeat(self, heartbeat): + """ + Send worker heartbeat to local supervisor + + Parameters: + - heartbeat + """ + self.send_sendSupervisorWorkerHeartbeat(heartbeat) + self.recv_sendSupervisorWorkerHeartbeat() + + def send_sendSupervisorWorkerHeartbeat(self, heartbeat): + self._oprot.writeMessageBegin('sendSupervisorWorkerHeartbeat', TMessageType.CALL, self._seqid) + args = sendSupervisorWorkerHeartbeat_args() + args.heartbeat = heartbeat + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_sendSupervisorWorkerHeartbeat(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = sendSupervisorWorkerHeartbeat_result() + result.read(iprot) + iprot.readMessageEnd() + if result.aze is not None: + raise result.aze + return + + +class Processor(Iface, TProcessor): + def __init__(self, handler): + self._handler = handler + self._processMap = {} + self._processMap["sendSupervisorAssignments"] = Processor.process_sendSupervisorAssignments + self._processMap["getLocalAssignmentForStorm"] = Processor.process_getLocalAssignmentForStorm + self._processMap["sendSupervisorWorkerHeartbeat"] = Processor.process_sendSupervisorWorkerHeartbeat + + def process(self, iprot, oprot): + (name, type, seqid) = iprot.readMessageBegin() + if name not in self._processMap: + iprot.skip(TType.STRUCT) + iprot.readMessageEnd() + x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name)) + oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid) + x.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + return + else: + self._processMap[name](self, seqid, iprot, oprot) + return True + + def process_sendSupervisorAssignments(self, seqid, iprot, oprot): + args = sendSupervisorAssignments_args() + args.read(iprot) + iprot.readMessageEnd() + result = sendSupervisorAssignments_result() + try: + self._handler.sendSupervisorAssignments(args.assignments) + msg_type = TMessageType.REPLY + except (TTransport.TTransportException, KeyboardInterrupt, SystemExit): + raise + except AuthorizationException as aze: + msg_type = TMessageType.REPLY + result.aze = aze + except Exception as ex: + msg_type = TMessageType.EXCEPTION + logging.exception(ex) + result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error') + oprot.writeMessageBegin("sendSupervisorAssignments", msg_type, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getLocalAssignmentForStorm(self, seqid, iprot, oprot): + args = getLocalAssignmentForStorm_args() + args.read(iprot) + iprot.readMessageEnd() + result = getLocalAssignmentForStorm_result() + try: + result.success = self._handler.getLocalAssignmentForStorm(args.id) + msg_type = TMessageType.REPLY + except (TTransport.TTransportException, KeyboardInterrupt, SystemExit): + raise + except NotAliveException as e: + msg_type = TMessageType.REPLY + result.e = e + except AuthorizationException as aze: + msg_type = TMessageType.REPLY + result.aze = aze + except Exception as ex: + msg_type = TMessageType.EXCEPTION + logging.exception(ex) + result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error') + oprot.writeMessageBegin("getLocalAssignmentForStorm", msg_type, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_sendSupervisorWorkerHeartbeat(self, seqid, iprot, oprot): + args = sendSupervisorWorkerHeartbeat_args() + args.read(iprot) + iprot.readMessageEnd() + result = sendSupervisorWorkerHeartbeat_result() + try: + self._handler.sendSupervisorWorkerHeartbeat(args.heartbeat) + msg_type = TMessageType.REPLY + except (TTransport.TTransportException, KeyboardInterrupt, SystemExit): + raise + except AuthorizationException as aze: + msg_type = TMessageType.REPLY + result.aze = aze + except Exception as ex: + msg_type = TMessageType.EXCEPTION + logging.exception(ex) + result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error') + oprot.writeMessageBegin("sendSupervisorWorkerHeartbeat", msg_type, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + +# HELPER FUNCTIONS AND STRUCTURES + +class sendSupervisorAssignments_args: + """ + Attributes: + - assignments + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'assignments', (SupervisorAssignments, SupervisorAssignments.thrift_spec), None, ), # 1 + ) + + def __init__(self, assignments=None,): + self.assignments = assignments + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.assignments = SupervisorAssignments() + self.assignments.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('sendSupervisorAssignments_args') + if self.assignments is not None: + oprot.writeFieldBegin('assignments', TType.STRUCT, 1) + self.assignments.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.assignments) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class sendSupervisorAssignments_result: + """ + Attributes: + - aze + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 + ) + + def __init__(self, aze=None,): + self.aze = aze + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('sendSupervisorAssignments_result') + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.aze) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getLocalAssignmentForStorm_args: + """ + Attributes: + - id + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + ) + + def __init__(self, id=None,): + self.id = id + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getLocalAssignmentForStorm_args') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getLocalAssignmentForStorm_result: + """ + Attributes: + - success + - e + - aze + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (Assignment, Assignment.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 + ) + + def __init__(self, success=None, e=None, aze=None,): + self.success = success + self.e = e + self.aze = aze + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = Assignment() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getLocalAssignmentForStorm_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + value = (value * 31) ^ hash(self.aze) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class sendSupervisorWorkerHeartbeat_args: + """ + Attributes: + - heartbeat + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'heartbeat', (SupervisorWorkerHeartbeat, SupervisorWorkerHeartbeat.thrift_spec), None, ), # 1 + ) + + def __init__(self, heartbeat=None,): + self.heartbeat = heartbeat + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.heartbeat = SupervisorWorkerHeartbeat() + self.heartbeat.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('sendSupervisorWorkerHeartbeat_args') + if self.heartbeat is not None: + oprot.writeFieldBegin('heartbeat', TType.STRUCT, 1) + self.heartbeat.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.heartbeat) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class sendSupervisorWorkerHeartbeat_result: + """ + Attributes: + - aze + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 + ) + + def __init__(self, aze=None,): + self.aze = aze + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('sendSupervisorWorkerHeartbeat_result') + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.aze) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) diff --git a/storm-client/src/py/storm/__init__.py b/storm-client/src/py/storm/__init__.py index 9ecdc2bb8cb..af102f23875 100644 --- a/storm-client/src/py/storm/__init__.py +++ b/storm-client/src/py/storm/__init__.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__all__ = ['ttypes', 'constants', 'Nimbus', 'DistributedRPC', 'DistributedRPCInvocations'] +__all__ = ['ttypes', 'constants', 'Nimbus', 'DistributedRPC', 'DistributedRPCInvocations', 'Supervisor'] diff --git a/storm-client/src/py/storm/ttypes.py b/storm-client/src/py/storm/ttypes.py index 2ae36053dd7..f6c98b08a46 100644 --- a/storm-client/src/py/storm/ttypes.py +++ b/storm-client/src/py/storm/ttypes.py @@ -244,15 +244,18 @@ class HBServerMessageType: class WorkerTokenServiceType: NIMBUS = 0 DRPC = 1 + SUPERVISOR = 2 _VALUES_TO_NAMES = { 0: "NIMBUS", 1: "DRPC", + 2: "SUPERVISOR", } _NAMES_TO_VALUES = { "NIMBUS": 0, "DRPC": 1, + "SUPERVISOR": 2, } @@ -9165,6 +9168,7 @@ class SupervisorInfo: - time_secs - hostname - assignment_id + - server_port - used_ports - meta - scheduler_meta @@ -9178,18 +9182,20 @@ class SupervisorInfo: (1, TType.I64, 'time_secs', None, None, ), # 1 (2, TType.STRING, 'hostname', None, None, ), # 2 (3, TType.STRING, 'assignment_id', None, None, ), # 3 - (4, TType.LIST, 'used_ports', (TType.I64,None), None, ), # 4 - (5, TType.LIST, 'meta', (TType.I64,None), None, ), # 5 - (6, TType.MAP, 'scheduler_meta', (TType.STRING,None,TType.STRING,None), None, ), # 6 - (7, TType.I64, 'uptime_secs', None, None, ), # 7 - (8, TType.STRING, 'version', None, None, ), # 8 - (9, TType.MAP, 'resources_map', (TType.STRING,None,TType.DOUBLE,None), None, ), # 9 + (4, TType.I32, 'server_port', None, None, ), # 4 + (5, TType.LIST, 'used_ports', (TType.I64,None), None, ), # 5 + (6, TType.LIST, 'meta', (TType.I64,None), None, ), # 6 + (7, TType.MAP, 'scheduler_meta', (TType.STRING,None,TType.STRING,None), None, ), # 7 + (8, TType.I64, 'uptime_secs', None, None, ), # 8 + (9, TType.STRING, 'version', None, None, ), # 9 + (10, TType.MAP, 'resources_map', (TType.STRING,None,TType.DOUBLE,None), None, ), # 10 ) - def __init__(self, time_secs=None, hostname=None, assignment_id=None, used_ports=None, meta=None, scheduler_meta=None, uptime_secs=None, version=None, resources_map=None,): + def __init__(self, time_secs=None, hostname=None, assignment_id=None, server_port=None, used_ports=None, meta=None, scheduler_meta=None, uptime_secs=None, version=None, resources_map=None,): self.time_secs = time_secs self.hostname = hostname self.assignment_id = assignment_id + self.server_port = server_port self.used_ports = used_ports self.meta = meta self.scheduler_meta = scheduler_meta @@ -9222,6 +9228,11 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 4: + if ftype == TType.I32: + self.server_port = iprot.readI32() + else: + iprot.skip(ftype) + elif fid == 5: if ftype == TType.LIST: self.used_ports = [] (_etype559, _size556) = iprot.readListBegin() @@ -9231,7 +9242,7 @@ def read(self, iprot): iprot.readListEnd() else: iprot.skip(ftype) - elif fid == 5: + elif fid == 6: if ftype == TType.LIST: self.meta = [] (_etype565, _size562) = iprot.readListBegin() @@ -9241,7 +9252,7 @@ def read(self, iprot): iprot.readListEnd() else: iprot.skip(ftype) - elif fid == 6: + elif fid == 7: if ftype == TType.MAP: self.scheduler_meta = {} (_ktype569, _vtype570, _size568 ) = iprot.readMapBegin() @@ -9252,17 +9263,17 @@ def read(self, iprot): iprot.readMapEnd() else: iprot.skip(ftype) - elif fid == 7: + elif fid == 8: if ftype == TType.I64: self.uptime_secs = iprot.readI64() else: iprot.skip(ftype) - elif fid == 8: + elif fid == 9: if ftype == TType.STRING: self.version = iprot.readString().decode('utf-8') else: iprot.skip(ftype) - elif fid == 9: + elif fid == 10: if ftype == TType.MAP: self.resources_map = {} (_ktype576, _vtype577, _size575 ) = iprot.readMapBegin() @@ -9295,22 +9306,26 @@ def write(self, oprot): oprot.writeFieldBegin('assignment_id', TType.STRING, 3) oprot.writeString(self.assignment_id.encode('utf-8')) oprot.writeFieldEnd() + if self.server_port is not None: + oprot.writeFieldBegin('server_port', TType.I32, 4) + oprot.writeI32(self.server_port) + oprot.writeFieldEnd() if self.used_ports is not None: - oprot.writeFieldBegin('used_ports', TType.LIST, 4) + oprot.writeFieldBegin('used_ports', TType.LIST, 5) oprot.writeListBegin(TType.I64, len(self.used_ports)) 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.writeFieldBegin('meta', TType.LIST, 6) oprot.writeListBegin(TType.I64, len(self.meta)) 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.writeFieldBegin('scheduler_meta', TType.MAP, 7) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.scheduler_meta)) for kiter584,viter585 in self.scheduler_meta.items(): oprot.writeString(kiter584.encode('utf-8')) @@ -9318,15 +9333,15 @@ def write(self, oprot): oprot.writeMapEnd() oprot.writeFieldEnd() if self.uptime_secs is not None: - oprot.writeFieldBegin('uptime_secs', TType.I64, 7) + oprot.writeFieldBegin('uptime_secs', TType.I64, 8) oprot.writeI64(self.uptime_secs) oprot.writeFieldEnd() if self.version is not None: - oprot.writeFieldBegin('version', TType.STRING, 8) + oprot.writeFieldBegin('version', TType.STRING, 9) oprot.writeString(self.version.encode('utf-8')) oprot.writeFieldEnd() if self.resources_map is not None: - oprot.writeFieldBegin('resources_map', TType.MAP, 9) + oprot.writeFieldBegin('resources_map', TType.MAP, 10) oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.resources_map)) for kiter586,viter587 in self.resources_map.items(): oprot.writeString(kiter586.encode('utf-8')) @@ -9349,6 +9364,7 @@ def __hash__(self): value = (value * 31) ^ hash(self.time_secs) value = (value * 31) ^ hash(self.hostname) value = (value * 31) ^ hash(self.assignment_id) + value = (value * 31) ^ hash(self.server_port) value = (value * 31) ^ hash(self.used_ports) value = (value * 31) ^ hash(self.meta) value = (value * 31) ^ hash(self.scheduler_meta) @@ -11838,6 +11854,283 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) +class SupervisorWorkerHeartbeat: + """ + Attributes: + - storm_id + - executors + - time_secs + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'storm_id', None, None, ), # 1 + (2, TType.LIST, 'executors', (TType.STRUCT,(ExecutorInfo, ExecutorInfo.thrift_spec)), None, ), # 2 + (3, TType.I32, 'time_secs', None, None, ), # 3 + ) + + def __init__(self, storm_id=None, executors=None, time_secs=None,): + self.storm_id = storm_id + self.executors = executors + self.time_secs = time_secs + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.storm_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.executors = [] + (_etype780, _size777) = iprot.readListBegin() + for _i781 in xrange(_size777): + _elem782 = ExecutorInfo() + _elem782.read(iprot) + self.executors.append(_elem782) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.time_secs = iprot.readI32() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('SupervisorWorkerHeartbeat') + if self.storm_id is not None: + oprot.writeFieldBegin('storm_id', TType.STRING, 1) + oprot.writeString(self.storm_id.encode('utf-8')) + oprot.writeFieldEnd() + if self.executors is not None: + oprot.writeFieldBegin('executors', TType.LIST, 2) + oprot.writeListBegin(TType.STRUCT, len(self.executors)) + for iter783 in self.executors: + iter783.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.time_secs is not None: + oprot.writeFieldBegin('time_secs', TType.I32, 3) + oprot.writeI32(self.time_secs) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.storm_id is None: + raise TProtocol.TProtocolException(message='Required field storm_id is unset!') + if self.executors is None: + raise TProtocol.TProtocolException(message='Required field executors is unset!') + if self.time_secs is None: + raise TProtocol.TProtocolException(message='Required field time_secs is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.storm_id) + value = (value * 31) ^ hash(self.executors) + value = (value * 31) ^ hash(self.time_secs) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class SupervisorWorkerHeartbeats: + """ + Attributes: + - supervisor_id + - worker_heartbeats + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'supervisor_id', None, None, ), # 1 + (2, TType.LIST, 'worker_heartbeats', (TType.STRUCT,(SupervisorWorkerHeartbeat, SupervisorWorkerHeartbeat.thrift_spec)), None, ), # 2 + ) + + def __init__(self, supervisor_id=None, worker_heartbeats=None,): + self.supervisor_id = supervisor_id + self.worker_heartbeats = worker_heartbeats + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.supervisor_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.worker_heartbeats = [] + (_etype787, _size784) = iprot.readListBegin() + for _i788 in xrange(_size784): + _elem789 = SupervisorWorkerHeartbeat() + _elem789.read(iprot) + self.worker_heartbeats.append(_elem789) + iprot.readListEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('SupervisorWorkerHeartbeats') + if self.supervisor_id is not None: + oprot.writeFieldBegin('supervisor_id', TType.STRING, 1) + oprot.writeString(self.supervisor_id.encode('utf-8')) + oprot.writeFieldEnd() + if self.worker_heartbeats is not None: + oprot.writeFieldBegin('worker_heartbeats', TType.LIST, 2) + oprot.writeListBegin(TType.STRUCT, len(self.worker_heartbeats)) + for iter790 in self.worker_heartbeats: + iter790.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.supervisor_id is None: + raise TProtocol.TProtocolException(message='Required field supervisor_id is unset!') + if self.worker_heartbeats is None: + raise TProtocol.TProtocolException(message='Required field worker_heartbeats is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.supervisor_id) + value = (value * 31) ^ hash(self.worker_heartbeats) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class SupervisorAssignments: + """ + Attributes: + - storm_assignment + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'storm_assignment', (TType.STRING,None,TType.STRUCT,(Assignment, Assignment.thrift_spec)), { + }, ), # 1 + ) + + def __init__(self, storm_assignment=thrift_spec[1][4],): + if storm_assignment is self.thrift_spec[1][4]: + storm_assignment = { + } + self.storm_assignment = storm_assignment + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.storm_assignment = {} + (_ktype792, _vtype793, _size791 ) = iprot.readMapBegin() + for _i795 in xrange(_size791): + _key796 = iprot.readString().decode('utf-8') + _val797 = Assignment() + _val797.read(iprot) + self.storm_assignment[_key796] = _val797 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('SupervisorAssignments') + if self.storm_assignment is not None: + oprot.writeFieldBegin('storm_assignment', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.storm_assignment)) + for kiter798,viter799 in self.storm_assignment.items(): + oprot.writeString(kiter798.encode('utf-8')) + viter799.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.storm_assignment) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + class WorkerMetricPoint: """ Attributes: @@ -12006,11 +12299,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.metrics = [] - (_etype780, _size777) = iprot.readListBegin() - for _i781 in xrange(_size777): - _elem782 = WorkerMetricPoint() - _elem782.read(iprot) - self.metrics.append(_elem782) + (_etype803, _size800) = iprot.readListBegin() + for _i804 in xrange(_size800): + _elem805 = WorkerMetricPoint() + _elem805.read(iprot) + self.metrics.append(_elem805) iprot.readListEnd() else: iprot.skip(ftype) @@ -12027,8 +12320,8 @@ def write(self, oprot): if self.metrics is not None: oprot.writeFieldBegin('metrics', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.metrics)) - for iter783 in self.metrics: - iter783.write(oprot) + for iter806 in self.metrics: + iter806.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -12438,11 +12731,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.pulses = [] - (_etype787, _size784) = iprot.readListBegin() - for _i788 in xrange(_size784): - _elem789 = HBPulse() - _elem789.read(iprot) - self.pulses.append(_elem789) + (_etype810, _size807) = iprot.readListBegin() + for _i811 in xrange(_size807): + _elem812 = HBPulse() + _elem812.read(iprot) + self.pulses.append(_elem812) iprot.readListEnd() else: iprot.skip(ftype) @@ -12459,8 +12752,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 iter790 in self.pulses: - iter790.write(oprot) + for iter813 in self.pulses: + iter813.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -12512,10 +12805,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.pulseIds = [] - (_etype794, _size791) = iprot.readListBegin() - for _i795 in xrange(_size791): - _elem796 = iprot.readString().decode('utf-8') - self.pulseIds.append(_elem796) + (_etype817, _size814) = iprot.readListBegin() + for _i818 in xrange(_size814): + _elem819 = iprot.readString().decode('utf-8') + self.pulseIds.append(_elem819) iprot.readListEnd() else: iprot.skip(ftype) @@ -12532,8 +12825,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 iter797 in self.pulseIds: - oprot.writeString(iter797.encode('utf-8')) + for iter820 in self.pulseIds: + oprot.writeString(iter820.encode('utf-8')) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() diff --git a/storm-client/src/storm.thrift b/storm-client/src/storm.thrift index 81b71f3ea49..e0935bd9526 100644 --- a/storm-client/src/storm.thrift +++ b/storm-client/src/storm.thrift @@ -483,6 +483,7 @@ struct SupervisorInfo { 7: optional i64 uptime_secs; 8: optional string version; 9: optional map resources_map; + 10: optional i32 server_port; } struct NodeInfo { @@ -672,6 +673,21 @@ struct OwnerResourceSummary { 18: optional double assigned_off_heap_memory; } +struct SupervisorWorkerHeartbeat { + 1: required string storm_id; + 2: required list executors + 3: required i32 time_secs; +} + +struct SupervisorWorkerHeartbeats { + 1: required string supervisor_id; + 2: required list worker_heartbeats; +} + +struct SupervisorAssignments { + 1: optional map storm_assignment = {} +} + struct WorkerMetricPoint { 1: required string metricName; 2: required i64 timestamp; @@ -768,6 +784,18 @@ service Nimbus { StormTopology getUserTopology(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze); TopologyHistoryInfo getTopologyHistory(1: string user) throws (1: AuthorizationException aze); list getOwnerResourceSummaries (1: string owner) throws (1: AuthorizationException aze); + /** + * Get assigned assignments for a specific supervisor + */ + SupervisorAssignments getSupervisorAssignments(1: string node) throws (1: AuthorizationException aze); + /** + * Send supervisor worker heartbeats for a specific supervisor + */ + void sendSupervisorWorkerHeartbeats(1: SupervisorWorkerHeartbeats heartbeats) throws (1: AuthorizationException aze); + /** + * Send supervisor local worker heartbeat when a supervisor is unreachable + */ + void sendSupervisorWorkerHeartbeat(1: SupervisorWorkerHeartbeat heatbeat) throws (1: AuthorizationException aze, 2: NotAliveException e); void processWorkerMetrics(1: WorkerMetrics metrics); } @@ -858,10 +886,26 @@ exception HBExecutionException { 1: required string msg; } +service Supervisor { + /** + * Send node specific assignments to supervisor + */ + void sendSupervisorAssignments(1: SupervisorAssignments assignments) throws (1: AuthorizationException aze); + /** + * Get local assignment for a storm + */ + Assignment getLocalAssignmentForStorm(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze); + /** + * Send worker heartbeat to local supervisor + */ + void sendSupervisorWorkerHeartbeat(1: SupervisorWorkerHeartbeat heartbeat) throws (1: AuthorizationException aze); +} + # WorkerTokens are used as credentials that allow a Worker to authenticate with DRPC, Nimbus, or other storm processes that we add in here. enum WorkerTokenServiceType { NIMBUS, - DRPC + DRPC, + SUPERVISOR } #This is information that we want to be sure users do not modify in any way... diff --git a/storm-client/test/jvm/org/apache/storm/assignments/LocalAssignmentsBackendTest.java b/storm-client/test/jvm/org/apache/storm/assignments/LocalAssignmentsBackendTest.java new file mode 100644 index 00000000000..69777a83dc5 --- /dev/null +++ b/storm-client/test/jvm/org/apache/storm/assignments/LocalAssignmentsBackendTest.java @@ -0,0 +1,102 @@ +/* + * 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.assignments; + +import org.apache.commons.collections.map.HashedMap; +import org.apache.storm.generated.Assignment; +import org.apache.storm.generated.NodeInfo; +import org.apache.storm.utils.ConfigUtils; +import org.junit.Test; +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.HashMap; +import java.util.Set; +import java.util.HashSet; + + +public class LocalAssignmentsBackendTest { + + @Test + public void testLocalAssignment(){ + Map stormToAssignment = new HashMap<>(); + String storm1 = "storm1"; + String storm2 = "storm2"; + Assignment ass1 = mockedAssignment(1); + Assignment ass2 = mockedAssignment(2); + + ILocalAssignmentsBackend backend = LocalAssignmentsBackendFactory.getBackend(ConfigUtils.readStormConfig()); + assertEquals(null, backend.getAssignment(storm1)); + backend.keepOrUpdateAssignment(storm1, ass1); + backend.keepOrUpdateAssignment(storm2, ass2); + assertEquals(ass1, backend.getAssignment(storm1)); + assertEquals(ass2, backend.getAssignment(storm2)); + backend.clearStateForStorm(storm1); + assertEquals(null, backend.getAssignment(storm1)); + backend.keepOrUpdateAssignment(storm1, ass1); + backend.keepOrUpdateAssignment(storm1, ass2); + assertEquals(ass2, backend.getAssignment(storm1)); + } + + @Test + public void testLocalIdInfo() { + String name1 = "name1"; + String name2 = "name2"; + String name3 = "name3"; + + String id1 = "id1"; + String id2 = "id2"; + String id3 = "id3"; + + ILocalAssignmentsBackend backend = LocalAssignmentsBackendFactory.getBackend(ConfigUtils.readStormConfig()); + assertEquals(null, backend.getStormId(name3)); + backend.keepStormId(name1, id1); + backend.keepStormId(name2, id2); + assertEquals(id1, backend.getStormId(name1)); + assertEquals(id2, backend.getStormId(name2)); + backend.deleteStormId(name1); + assertEquals(null, backend.getStormId(name1)); + backend.clearStateForStorm(id2); + assertEquals(null, backend.getStormId(name2)); + backend.keepStormId(name1, id1); + backend.keepStormId(name1, id3); + assertEquals(id3, backend.getStormId(name1)); + } + + private Assignment mockedAssignment(int i) { + Assignment ass = new Assignment(); + ass.set_master_code_dir("master_code_dir" + i); + HashMap node_to_host = new HashMap(); + node_to_host.put("node" + i, "host" + i); + ass.set_node_host(node_to_host); + Map, NodeInfo> executor_node_port = new HashMap<>(); + Set nodePorts = new HashSet<>(); + nodePorts.add(9723L); + executor_node_port.put(Arrays.asList(i + 0L), new NodeInfo("node" + i, nodePorts)); + ass.set_executor_node_port(executor_node_port); + Map, Long> executor_start_time_secs = new HashMap<>(); + executor_start_time_secs.put(Arrays.asList(1L), 12345L); + ass.set_executor_start_time_secs(executor_start_time_secs); + ass.set_worker_resources(new HashedMap()); + ass.set_total_shared_off_heap(new HashedMap()); + ass.set_owner("o"); + return ass; + } +} diff --git a/storm-client/test/jvm/org/apache/storm/cluster/StormClusterStateImplTest.java b/storm-client/test/jvm/org/apache/storm/cluster/StormClusterStateImplTest.java index e2ef41ee12a..86647017dc7 100644 --- a/storm-client/test/jvm/org/apache/storm/cluster/StormClusterStateImplTest.java +++ b/storm-client/test/jvm/org/apache/storm/cluster/StormClusterStateImplTest.java @@ -18,9 +18,8 @@ package org.apache.storm.cluster; +import org.apache.storm.assignments.LocalAssignmentsBackendFactory; import org.apache.storm.callback.ZKStateChangedCallback; -import org.apache.zookeeper.KeeperException; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.mockito.Matchers; @@ -48,7 +47,7 @@ public class StormClusterStateImplTest { public void init() throws Exception { storage = Mockito.mock(IStateStorage.class); context = new ClusterStateContext(); - state = new StormClusterStateImpl(storage, context, false /*solo*/); + state = new StormClusterStateImpl(storage, LocalAssignmentsBackendFactory.getDefault(), context, false /*solo*/); } diff --git a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj index 351404001c5..48057290ca4 100644 --- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj +++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj @@ -32,7 +32,7 @@ root (.get conf Config/STORM_ZOOKEEPER_ROOT) zk (ClientZookeeper/mkClient conf servers port root (DefaultWatcherCallBack.) conf) ; since this is not a purpose to add to leader lock queue, passing nil as blob-store and topo cache is ok - zk-leader-elector (Zookeeper/zkLeaderElector conf zk nil nil) + zk-leader-elector (Zookeeper/zkLeaderElector conf zk nil nil nil nil) leader-nimbus (.getLeader zk-leader-elector) host (.getHost leader-nimbus) port (.getPort leader-nimbus) diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj index 37f5c7416f4..d2066e83dd9 100644 --- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj +++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj @@ -46,7 +46,8 @@ (:import [org.apache.commons.io FileUtils]) (:import [org.json.simple JSONValue]) (:import [org.apache.storm.daemon StormCommon]) - (:import [org.apache.storm.cluster IStormClusterState StormClusterStateImpl ClusterStateContext ClusterUtils]) + (:import [org.apache.storm.cluster IStormClusterState StormClusterStateImpl ClusterStateContext ClusterUtils] + [org.apache.storm.assignments LocalAssignmentsBackendFactory]) (:use [org.apache.storm util daemon-config config log]) (:require [conjure.core]) @@ -168,7 +169,8 @@ (log-warn "merged:" stats) (.workerHeartbeat state storm-id node port - (StatsUtil/thriftifyZkWorkerHb (StatsUtil/mkZkWorkerHb storm-id stats (int 10)))))) + (StatsUtil/thriftifyZkWorkerHb (StatsUtil/mkZkWorkerHb storm-id stats (int 10)))) + (.sendSupervisorWorkerHeartbeat (.getNimbus cluster) (StatsUtil/thriftifyRPCWorkerHb storm-id executor)))) (defn slot-assignments [cluster storm-id] (let [state (.getClusterState cluster) @@ -686,11 +688,12 @@ (.submitTopology cluster "test3" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 5} topology) (bind storm-id3 (StormCommon/getStormId state "test3")) (.advanceClusterTime cluster 11) - (.removeStorm state storm-id3) + ;; this guarantees an immediate kill notification + (.killTopology (.getNimbus cluster) "test3") + (.advanceClusterTime cluster 41) (is (nil? (.stormBase state storm-id3 nil))) (is (nil? (.assignmentInfo state storm-id3 nil))) - (.advanceClusterTime cluster 11) (is (= 0 (count (.heartbeatStorms state)))) ;; this guarantees that monitor thread won't trigger for 10 more seconds @@ -801,6 +804,7 @@ (.advanceClusterTime cluster 31) (is (not= ass1 (executor-assignment cluster storm-id executor-id1))) (is (= ass2 (executor-assignment cluster storm-id executor-id2))) ; tests launch timeout + (check-consistency cluster "test") @@ -879,7 +883,8 @@ (.advanceClusterTime cluster 13) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (is (= ass2 (executor-assignment cluster storm-id executor-id2))) - (.killSupervisor cluster "b") + ;; with rpc reporting mode, only heartbeats from killed supervisor will time out + (.killSupervisor cluster (.get_node ass2)) (do-executor-heartbeat cluster storm-id executor-id1) (.advanceClusterTime cluster 11) @@ -1311,7 +1316,7 @@ (with-open [zk (InProcessZookeeper. )] (with-open [tmp-nimbus-dir (TmpPath.) _ (MockedZookeeper. (proxy [Zookeeper] [] - (zkLeaderElectorImpl [conf zk blob-store tc] (MockLeaderElector. ))))] + (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. ))))] (let [nimbus-dir (.getPath tmp-nimbus-dir)] (letlocals (bind conf (merge (clojurify-structure (ConfigUtils/readStormConfig)) @@ -1319,8 +1324,9 @@ STORM-CLUSTER-MODE "local" STORM-ZOOKEEPER-PORT (.getPort zk) STORM-LOCAL-DIR nimbus-dir})) - (bind cluster-state (ClusterUtils/mkStormClusterState conf (ClusterStateContext.))) - (bind nimbus (mk-nimbus conf (Nimbus$StandaloneINimbus.) nil nil nil nil)) + (bind ass-backend (LocalAssignmentsBackendFactory/getDefault)) + (bind cluster-state (ClusterUtils/mkStormClusterState conf ass-backend (ClusterStateContext.))) + (bind nimbus (mk-nimbus conf (Nimbus$StandaloneINimbus.) nil nil nil cluster-state)) (.launchServer nimbus) (bind topology (Thrift/buildTopology {"1" (Thrift/prepareSpoutDetails @@ -1328,11 +1334,11 @@ {})) (with-open [_ (MockedZookeeper. (proxy [Zookeeper] [] - (zkLeaderElectorImpl [conf zk blob-store tc] (MockLeaderElector. false))))] + (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. false))))] (letlocals - (bind non-leader-cluster-state (ClusterUtils/mkStormClusterState conf (ClusterStateContext.))) - (bind non-leader-nimbus (mk-nimbus conf (Nimbus$StandaloneINimbus.) nil nil nil nil)) + (bind non-leader-cluster-state (ClusterUtils/mkStormClusterState conf ass-backend (ClusterStateContext.))) + (bind non-leader-nimbus (mk-nimbus conf (Nimbus$StandaloneINimbus.) nil nil nil non-leader-cluster-state)) (.launchServer non-leader-nimbus) ;first we verify that the master nimbus can perform all actions, even with another nimbus present. @@ -1374,7 +1380,8 @@ (deftest test-nimbus-iface-submitTopologyWithOpts-checks-authorization (with-open [cluster (.build (doto (LocalCluster$Builder. ) (.withDaemonConf {NIMBUS-AUTHORIZER - "org.apache.storm.security.auth.authorizer.DenyAuthorizer"})))] + "org.apache.storm.security.auth.authorizer.DenyAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer"})))] (let [ topology (Thrift/buildTopology {} {}) ] @@ -1395,7 +1402,8 @@ (.withClusterState cluster-state) (.withBlobStore blob-store) (.withTopoCache tc) - (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer"})))] + (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer"})))] (let [nimbus (.getNimbus cluster) topology-name "test" topology-id "test-id"] @@ -1417,7 +1425,8 @@ (.withBlobStore blob-store) (.withTopoCache tc) (.withNimbusWrapper (reify UnaryOperator (apply [this nimbus] (Mockito/spy nimbus)))) - (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] + (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] (let [nimbus (.getNimbus cluster) topology-name "test-nimbus-check-autho-params" topology-id "fake-id" @@ -1476,7 +1485,8 @@ (.withBlobStore blob-store) (.withTopoCache tc) (.withNimbusWrapper (reify UnaryOperator (apply [this nimbus] (Mockito/spy nimbus)))) - (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] + (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] (let [nimbus (.getNimbus cluster) expected-name "test-nimbus-check-autho-params" expected-conf {TOPOLOGY-NAME expected-name @@ -1506,7 +1516,7 @@ (.thenReturn (Mockito/when (.allSupervisorInfo cluster-state)) all-supervisors) (.thenReturn (Mockito/when (.readTopoConf tc (Mockito/any String) (Mockito/any Subject))) expected-conf) (.thenReturn (Mockito/when (.readTopology tc (Mockito/any String) (Mockito/any Subject))) topology) - (.thenReturn (Mockito/when (.topologyAssignments cluster-state)) topo-assignment) + (.thenReturn (Mockito/when (.assignmentsInfo cluster-state)) topo-assignment) (.getSupervisorPageInfo nimbus "super1" nil true) ;; afterwards, it should get called twice @@ -1630,7 +1640,8 @@ (.withClusterState cluster-state) (.withBlobStore blob-store) (.withTopoCache tc) - (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] + (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] (.thenReturn (Mockito/when (.getTopoId cluster-state "test")) (Optional/empty)) (let [topology (Thrift/buildTopology {} {}) bad-config {"topology.isolate.machines" "2"}] @@ -1676,7 +1687,7 @@ (with-open [zk (InProcessZookeeper. )] (with-open [tmp-nimbus-dir (TmpPath.) _ (MockedZookeeper. (proxy [Zookeeper] [] - (zkLeaderElectorImpl [conf zk blob-store tc] (MockLeaderElector. ))))] + (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. ))))] (let [nimbus-dir (.getPath tmp-nimbus-dir)] (letlocals (bind conf (merge (clojurify-structure (ConfigUtils/readStormConfig)) @@ -1745,7 +1756,8 @@ (.withClusterState cluster-state) (.withBlobStore blob-store) (.withTopoCache tc) - (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] + (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] (let [nimbus (.getNimbus cluster) previous-config (LogConfig.) mock-config (LogConfig.) @@ -1776,7 +1788,8 @@ (.withClusterState cluster-state) (.withBlobStore blob-store) (.withTopoCache tc) - (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] + (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer"})))] (let [nimbus (.getNimbus cluster) previous-config (LogConfig.) mock-config (LogConfig.) @@ -1874,7 +1887,7 @@ mock-blob-store (Mockito/mock BlobStore) conf {NIMBUS-MONITOR-FREQ-SECS 10}] (with-open [_ (MockedZookeeper. (proxy [Zookeeper] [] - (zkLeaderElectorImpl [conf zk blob-store tc] (MockLeaderElector. ))))] + (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. ))))] (let [nimbus (Mockito/spy (Nimbus. conf nil mock-state nil mock-blob-store nil nil))] (.set (.getHeartbeatsCache nimbus) hb-cache) (.thenReturn (Mockito/when (.storedTopoIds mock-blob-store)) (HashSet. inactive-topos)) @@ -1914,7 +1927,7 @@ mock-blob-store (Mockito/mock BlobStore) conf {NIMBUS-MONITOR-FREQ-SECS 10}] (with-open [_ (MockedZookeeper. (proxy [Zookeeper] [] - (zkLeaderElectorImpl [conf zk blob-store tc] (MockLeaderElector. ))))] + (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. ))))] (let [nimbus (Mockito/spy (Nimbus. conf nil mock-state nil mock-blob-store nil nil))] (.set (.getHeartbeatsCache nimbus) hb-cache) (.thenReturn (Mockito/when (.storedTopoIds mock-blob-store)) (set inactive-topos)) diff --git a/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj b/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj index abc157918a2..148afd897de 100644 --- a/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj +++ b/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj @@ -38,6 +38,7 @@ (defn to-conf [nimbus-port login-cfg aznClass transportPluginClass] (let [conf {NIMBUS-AUTHORIZER aznClass + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer" NIMBUS-THRIFT-PORT nimbus-port STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass } conf (if login-cfg (merge conf {"java.security.auth.login.config" login-cfg}) conf)] @@ -78,6 +79,7 @@ (.withNimbusDaemon) (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer" NIMBUS-THRIFT-PORT 0 STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.SimpleTransportPlugin"})))] (let [storm-conf (merge (clojurify-structure (ConfigUtils/readStormConfig)) @@ -106,6 +108,7 @@ (.withNimbusDaemon) (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer" NIMBUS-THRIFT-PORT 0 STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.SimpleTransportPlugin"})))] (let [storm-conf (merge (clojurify-structure (ConfigUtils/readStormConfig)) @@ -170,6 +173,7 @@ (.withNimbusDaemon) (.withDaemonConf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer" + SUPERVISOR-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer" NIMBUS-THRIFT-PORT 0 "java.security.auth.login.config" "test/clj/org/apache/storm/security/auth/jaas_digest.conf" STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.digest.DigestSaslTransportPlugin"})))] diff --git a/storm-core/test/jvm/org/apache/storm/stats/TestStatsUtil.java b/storm-core/test/jvm/org/apache/storm/stats/TestStatsUtil.java index 78b032c238f..1ca793b41cd 100644 --- a/storm-core/test/jvm/org/apache/storm/stats/TestStatsUtil.java +++ b/storm-core/test/jvm/org/apache/storm/stats/TestStatsUtil.java @@ -70,14 +70,8 @@ public void makeTopoInfo() { HashMap exec2Beat = new HashMap(); exec2Beat.put("uptime", 200); - Map beat1 = new HashMap(); - beat1.put("heartbeat", exec1Beat); - - Map beat2 = new HashMap(); - beat2.put("heartbeat", exec2Beat); - - beats.put(exec1, beat1); - beats.put(exec2, beat2); + beats.put(exec1, exec1Beat); + beats.put(exec2, exec2Beat); task2Component.put(1, "my-component"); task2Component.put(2, "__sys1"); diff --git a/storm-server/src/main/java/org/apache/storm/DaemonConfig.java b/storm-server/src/main/java/org/apache/storm/DaemonConfig.java index f7388f5da6c..ec1815dfbc8 100644 --- a/storm-server/src/main/java/org/apache/storm/DaemonConfig.java +++ b/storm-server/src/main/java/org/apache/storm/DaemonConfig.java @@ -38,6 +38,7 @@ import org.apache.storm.scheduler.blacklist.strategies.IBlacklistStrategy; import org.apache.storm.scheduler.resource.strategies.priority.ISchedulingPriorityStrategy; import org.apache.storm.scheduler.resource.strategies.scheduling.IStrategy; +import org.apache.storm.security.auth.IAuthorizer; import org.apache.storm.validation.ConfigValidation; import org.apache.storm.validation.Validated; @@ -263,6 +264,13 @@ public class DaemonConfig implements Validated { @isString public static final String NIMBUS_AUTHORIZER = "nimbus.authorizer"; + /** + * Class name for authorization plugin for supervisor. + */ + @isImplementationOfClass(implementsClass = IAuthorizer.class) + @isString + public static final String SUPERVISOR_AUTHORIZER = "supervisor.authorizer"; + /** * Impersonation user ACL config entries. */ @@ -282,6 +290,24 @@ public class DaemonConfig implements Validated { @isImplementationOfClass(implementsClass = ITopologyActionNotifierPlugin.class) public static final String NIMBUS_TOPOLOGY_ACTION_NOTIFIER_PLUGIN = "nimbus.topology.action.notifier.plugin.class"; + /** + * This controls the number of working threads for distributing master assignments to supervisors. + */ + @isInteger + public static final String NIMBUS_ASSIGNMENTS_SERVICE_THREADS = "nimbus.assignments.service.threads"; + + /** + * This controls the number of working thread queue size of assignment service. + */ + @isInteger + public static final String NIMBUS_ASSIGNMENTS_SERVICE_THREAD_QUEUE_SIZE = "nimbus.assignments.service.thread.queue.size"; + + /** + * class controls heartbeats recovery strategy + */ + @isString + public static final String NIMBUS_WORKER_HEARTBEATS_RECOVERY_STRATEGY_CLASS = "nimbus.worker.heartbeats.recovery.strategy.class"; + /** * Storm UI binds to this host/interface. */ diff --git a/storm-server/src/main/java/org/apache/storm/LocalCluster.java b/storm-server/src/main/java/org/apache/storm/LocalCluster.java index 39c8d5734c0..7db41b58d85 100644 --- a/storm-server/src/main/java/org/apache/storm/LocalCluster.java +++ b/storm-server/src/main/java/org/apache/storm/LocalCluster.java @@ -75,6 +75,9 @@ import org.apache.storm.generated.StormTopology; import org.apache.storm.generated.SubmitOptions; import org.apache.storm.generated.SupervisorPageInfo; +import org.apache.storm.generated.SupervisorAssignments; +import org.apache.storm.generated.SupervisorWorkerHeartbeats; +import org.apache.storm.generated.SupervisorWorkerHeartbeat; import org.apache.storm.generated.TopologyHistoryInfo; import org.apache.storm.generated.TopologyInfo; import org.apache.storm.generated.TopologyPageInfo; @@ -808,6 +811,8 @@ public String generateSupervisorId() { Supervisor s = new Supervisor(superConf, sharedContext, isuper); s.launch(); + s.setLocalNimbus(this.nimbus); + this.nimbus.addSupervisor(s); supervisors.add(s); return s; } @@ -1138,6 +1143,20 @@ public List getOwnerResourceSummaries(String owner) throws } @Override + public SupervisorAssignments getSupervisorAssignments(String node) throws AuthorizationException, TException { + return null; + } + + @Override + public void sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats) throws AuthorizationException, TException { + + } + + @Override + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heatbeat) throws AuthorizationException, TException { + + } + public void processWorkerMetrics(WorkerMetrics metrics) throws org.apache.thrift.TException { getNimbus().processWorkerMetrics(metrics); } 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 f5d08ec0ce8..deeffc2be03 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 @@ -22,6 +22,7 @@ import com.codahale.metrics.Histogram; import com.codahale.metrics.Meter; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; import java.io.File; @@ -48,11 +49,13 @@ import java.util.Map.Entry; import java.util.NavigableMap; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.UnaryOperator; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; import javax.security.auth.Subject; import org.apache.curator.framework.CuratorFramework; @@ -115,9 +118,12 @@ import org.apache.storm.generated.StormBase; import org.apache.storm.generated.StormTopology; import org.apache.storm.generated.SubmitOptions; +import org.apache.storm.generated.SupervisorAssignments; import org.apache.storm.generated.SupervisorInfo; import org.apache.storm.generated.SupervisorPageInfo; import org.apache.storm.generated.SupervisorSummary; +import org.apache.storm.generated.SupervisorWorkerHeartbeat; +import org.apache.storm.generated.SupervisorWorkerHeartbeats; import org.apache.storm.generated.TopologyActionOptions; import org.apache.storm.generated.TopologyHistoryInfo; import org.apache.storm.generated.TopologyInfo; @@ -138,6 +144,7 @@ import org.apache.storm.metric.api.DataPoint; import org.apache.storm.metric.api.IClusterMetricsConsumer; import org.apache.storm.metric.api.IClusterMetricsConsumer.ClusterInfo; +import org.apache.storm.nimbus.AssignmentDistributionService; import org.apache.storm.metricstore.AggLevel; import org.apache.storm.metricstore.Metric; import org.apache.storm.metricstore.MetricStore; @@ -146,7 +153,9 @@ import org.apache.storm.nimbus.ILeaderElector; import org.apache.storm.nimbus.ITopologyActionNotifierPlugin; import org.apache.storm.nimbus.ITopologyValidator; +import org.apache.storm.nimbus.IWorkerHeartbeatsRecoveryStrategy; import org.apache.storm.nimbus.NimbusInfo; +import org.apache.storm.nimbus.WorkerHeartbeatsRecoveryStrategyFactory; import org.apache.storm.scheduler.Cluster; import org.apache.storm.scheduler.DefaultScheduler; import org.apache.storm.scheduler.ExecutorDetails; @@ -194,6 +203,8 @@ import org.apache.storm.zookeeper.ClientZookeeper; import org.apache.storm.zookeeper.Zookeeper; import org.apache.thrift.TException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.data.ACL; import org.json.simple.JSONValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -244,6 +255,20 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon { private static final String STORM_VERSION = VersionInfo.getVersion(); + @VisibleForTesting + public static final List ZK_ACLS = Arrays.asList(ZooDefs.Ids.CREATOR_ALL_ACL.get(0), + new ACL(ZooDefs.Perms.READ | ZooDefs.Perms.CREATE, ZooDefs.Ids.ANYONE_ID_UNSAFE)); + + public static final SimpleVersion MIN_VERSION_SUPPORT_RPC_HEARTBEAT = new SimpleVersion("2.0.0"); + + private static List getNimbusAcls(Map conf) { + List acls = null; + if (Utils.isZkAuthenticationConfiguredStormServer(conf)) { + acls = ZK_ACLS; + } + return acls; + } + private static final Subject NIMBUS_SUBJECT = new Subject(); static { @@ -320,7 +345,9 @@ private static StormBase make(TopologyStatus status) { private static final TopologyStateTransition REMOVE_TRANSITION = (args, nimbus, topoId, base) -> { LOG.info("Killing topology: {}", topoId); IStormClusterState state = nimbus.getStormClusterState(); + Assignment oldAssignment = state.assignmentInfo(topoId, null); state.removeStorm(topoId); + notifySupervisorsAsKilled(state, oldAssignment, nimbus.getAssignmentsDistributer()); BlobStore store = nimbus.getBlobStore(); if (store instanceof LocalFsBlobStore) { for (String key: Nimbus.getKeyListFromId(nimbus.getConf(), topoId)) { @@ -328,6 +355,7 @@ private static StormBase make(TopologyStatus status) { state.removeKeyVersion(key); } } + nimbus.getHeartbeatsCache().getAndUpdate(new Dissoc<>(topoId)); return null; }; @@ -443,7 +471,7 @@ public IScheduler getForcedScheduler() { return null; } - }; + } private static class CommonTopoInfo { public Map topoConf; @@ -552,7 +580,7 @@ private static ITopologyActionNotifierPlugin createTopologyActionNotifier(Map makeClusterMetricsConsumerExecutors(Map conf) { Collection> consumers = (Collection>) conf.get( - DaemonConfig.STORM_CLUSTER_METRICS_CONSUMER_REGISTER); + DaemonConfig.STORM_CLUSTER_METRICS_CONSUMER_REGISTER); List ret = new ArrayList<>(); if (consumers != null) { for (Map consumer : consumers) { @@ -566,7 +594,8 @@ private static Subject getSubject() { return ReqContext.context().subject(); } - static Map readTopoConf(String topoId, TopoCache tc) throws KeyNotFoundException, AuthorizationException, IOException { + static Map readTopoConf(String topoId, TopoCache tc) throws KeyNotFoundException, + AuthorizationException, IOException { return tc.readTopoConf(topoId, getSubject()); } @@ -587,24 +616,24 @@ private static int getVersionForKey(String key, NimbusInfo nimbusInfo, } private static StormTopology readStormTopology(String topoId, TopoCache tc) throws KeyNotFoundException, AuthorizationException, - IOException { + IOException { return tc.readTopology(topoId, getSubject()); } private static Map readTopoConfAsNimbus(String topoId, TopoCache tc) throws KeyNotFoundException, - AuthorizationException, IOException { + AuthorizationException, IOException { return tc.readTopoConf(topoId, NIMBUS_SUBJECT); } private static StormTopology readStormTopologyAsNimbus(String topoId, TopoCache tc) throws KeyNotFoundException, - AuthorizationException, IOException { + AuthorizationException, IOException { return tc.readTopology(topoId, NIMBUS_SUBJECT); } /** * convert {topology-id -> SchedulerAssignment} to * {topology-id -> {executor [node port]}}. - * @return + * @return {topology-id -> {executor [node port]}} mapping */ private static Map, List>> computeTopoToExecToNodePort(Map schedAssignments) { Map, List>> ret = new HashMap<>(); @@ -653,7 +682,7 @@ private static Map> computeTopoToNodePo } private static Map, List>> computeNewTopoToExecToNodePort( - Map schedAssignments, Map existingAssignments) { + Map schedAssignments, Map existingAssignments) { Map, List>> ret = computeTopoToExecToNodePort(schedAssignments); // Print some useful information if (existingAssignments != null && !existingAssignments.isEmpty()) { @@ -687,8 +716,8 @@ private static Map, List>> computeNewTopoToExecTo return ret; } - private static List> changedExecutors(Map, NodeInfo> map, - Map, List> newExecToNodePort) { + private static List> changedExecutors(Map, NodeInfo> map, Map, + List> newExecToNodePort) { HashMap>> tmpSlotAssigned = map == null ? new HashMap<>() : Utils.reverseMap(map); HashMap, List>> slotAssigned = new HashMap<>(); for (Entry>> entry: tmpSlotAssigned.entrySet()) { @@ -732,8 +761,8 @@ private static Map basicSupervisorDetailsMap(IStormCl for (Entry entry: state.allSupervisorInfo().entrySet()) { String id = entry.getKey(); SupervisorInfo info = entry.getValue(); - ret.put(id, new SupervisorDetails(id, info.get_hostname(), info.get_scheduler_meta(), null, - info.get_resources_map())); + ret.put(id, new SupervisorDetails(id, info.get_server_port(), info.get_hostname(), + info.get_scheduler_meta(), null, info.get_resources_map())); } return ret; } @@ -742,8 +771,8 @@ private static boolean isTopologyActive(IStormClusterState state, String topoNam return state.getTopoId(topoName).isPresent(); } - private static Map tryReadTopoConf(String topoId, TopoCache tc) throws NotAliveException, AuthorizationException, - IOException { + private static Map tryReadTopoConf(String topoId, TopoCache tc) + throws NotAliveException, AuthorizationException, IOException { try { return readTopoConfAsNimbus(topoId, tc); //Was a try-cause but I looked at the code around this and key not found is not wrapped in runtime, @@ -782,7 +811,8 @@ private static String extractStatusStr(StormBase base) { return ret; } - private static StormTopology normalizeTopology(Map topoConf, StormTopology topology) throws InvalidTopologyException { + private static StormTopology normalizeTopology(Map topoConf, StormTopology topology) + throws InvalidTopologyException { StormTopology ret = topology.deepCopy(); for (Object comp: StormCommon.allComponents(ret).values()) { Map mergedConf = StormCommon.componentConf(comp); @@ -865,7 +895,7 @@ public static void cleanInbox(String dirLoc, int seconds) { final long now = Time.currentTimeMillis(); final long ms = Time.secsToMillis(seconds); File dir = new File(dirLoc); - for (File f : dir.listFiles((f) -> f.isFile() && ((f.lastModified() + ms) <= now))) { + for (File f : dir.listFiles((file) -> file.isFile() && ((file.lastModified() + ms) <= now))) { if (f.delete()) { LOG.info("Cleaning inbox ... deleted: {}", f.getName()); } else { @@ -879,13 +909,14 @@ private static ExecutorInfo toExecInfo(List exec) { } private static final Pattern TOPOLOGY_NAME_REGEX = Pattern.compile("^[^/.:\\\\]+$"); + private static void validateTopologyName(String name) throws InvalidTopologyException { Matcher m = TOPOLOGY_NAME_REGEX.matcher(name); if (!m.matches()) { throw new InvalidTopologyException("Topology name must match " + TOPOLOGY_NAME_REGEX); } } - + private static StormTopology tryReadTopology(String topoId, TopoCache tc) throws NotAliveException, AuthorizationException, IOException { try { @@ -895,7 +926,8 @@ private static StormTopology tryReadTopology(String topoId, TopoCache tc) } } - private static void validateTopologySize(Map topoConf, Map nimbusConf, StormTopology topology) throws InvalidTopologyException { + private static void validateTopologySize(Map topoConf, Map nimbusConf, + StormTopology topology) throws InvalidTopologyException { int workerCount = ObjectReader.getInt(topoConf.get(Config.TOPOLOGY_WORKERS), 1); Integer allowedWorkers = ObjectReader.getInt(nimbusConf.get(DaemonConfig.NIMBUS_SLOTS_PER_TOPOLOGY), null); int executorsCount = 0; @@ -1047,6 +1079,8 @@ public static void main(String[] args) throws Exception { private final Object schedLock = new Object(); private final Object credUpdateLock = new Object(); private final AtomicReference, Map>>> heartbeatsCache; + private final AtomicBoolean heartbeatsReadyFlag; + private final IWorkerHeartbeatsRecoveryStrategy heartbeatsRecoveryStrategy; @SuppressWarnings("deprecation") private final TimeCacheMap downloaders; @SuppressWarnings("deprecation") @@ -1065,6 +1099,7 @@ public static void main(String[] args) throws Exception { private final IScheduler scheduler; private final IScheduler underlyingScheduler; private final ILeaderElector leaderElector; + private final AssignmentDistributionService assignmentsDistributer; private final AtomicReference> idToSchedStatus; private final AtomicReference> nodeIdToResources; private final AtomicReference> idToResources; @@ -1134,6 +1169,8 @@ public Nimbus(Map conf, INimbus inimbus, IStormClusterState stor } this.stormClusterState = stormClusterState; this.heartbeatsCache = new AtomicReference<>(new HashMap<>()); + this.heartbeatsReadyFlag = new AtomicBoolean(false); + this.heartbeatsRecoveryStrategy = WorkerHeartbeatsRecoveryStrategyFactory.getStrategy(conf); this.downloaders = fileCacheMap(conf); this.uploaders = fileCacheMap(conf); if (blobStore == null) { @@ -1157,9 +1194,10 @@ public Nimbus(Map conf, INimbus inimbus, IStormClusterState stor this.scheduler = wrapAsBlacklistScheduler(conf, underlyingScheduler); this.zkClient = makeZKClient(conf); if (leaderElector == null) { - leaderElector = Zookeeper.zkLeaderElector(conf, zkClient, blobStore, topoCache); + leaderElector = Zookeeper.zkLeaderElector(conf, zkClient, blobStore, topoCache, stormClusterState, getNimbusAcls(conf)); } this.leaderElector = leaderElector; + this.assignmentsDistributer = AssignmentDistributionService.getInstance(conf); this.idToSchedStatus = new AtomicReference<>(new HashMap<>()); this.nodeIdToResources = new AtomicReference<>(new HashMap<>()); this.idToResources = new AtomicReference<>(new HashMap<>()); @@ -1190,6 +1228,10 @@ public void setAuthorizationHandler(IAuthorizer authorizationHandler) { private IStormClusterState getStormClusterState() { return stormClusterState; } + + private AssignmentDistributionService getAssignmentsDistributer() { + return assignmentsDistributer; + } @VisibleForTesting public AtomicReference,Map>>> getHeartbeatsCache() { @@ -1225,6 +1267,14 @@ private void assertIsLeader() throws Exception { private String getInbox() throws IOException { return ServerConfigUtils.masterInbox(conf); } + + /** + * Used for local cluster. + * @param supervisor {@link org.apache.storm.daemon.supervisor.Supervisor} + */ + public void addSupervisor(org.apache.storm.daemon.supervisor.Supervisor supervisor) { + assignmentsDistributer.addLocalSupervisor(supervisor); + } void delayEvent(String topoId, int delaySecs, TopologyActions event, Object args) { LOG.info("Delaying event {} for {} secs for {}", event, delaySecs, topoId); @@ -1268,7 +1318,8 @@ private void transition(String topoId, TopologyActions event, Object eventArg) t transition(topoId, event, eventArg, false); } - private void transition(String topoId, TopologyActions event, Object eventArg, boolean errorOnNoTransition) throws Exception { + private void transition(String topoId, TopologyActions event, Object eventArg, boolean errorOnNoTransition) + throws Exception { LOG.info("TRANSITION: {} {} {} {}", topoId, event, eventArg, errorOnNoTransition); assertIsLeader(); synchronized(submitLock) { @@ -1299,8 +1350,8 @@ private void transition(String topoId, TopologyActions event, Object eventArg, b } } - private void setupStormCode(Map conf, String topoId, String tmpJarLocation, - Map topoConf, StormTopology topology) throws Exception { + private void setupStormCode(Map conf, String topoId, String tmpJarLocation, + Map topoConf, StormTopology topology) throws Exception { Subject subject = getSubject(); IStormClusterState clusterState = stormClusterState; BlobStore store = blobStore; @@ -1404,7 +1455,7 @@ private void waitForDesiredCodeReplication(Map topoConf, String } private TopologyDetails readTopologyDetails(String topoId, StormBase base) throws KeyNotFoundException, - AuthorizationException, IOException, InvalidTopologyException { + AuthorizationException, IOException, InvalidTopologyException { assert (base != null); assert (topoId != null); @@ -1414,7 +1465,7 @@ private TopologyDetails readTopologyDetails(String topoId, StormBase base) throw fixupBase(base, topoConf); stormClusterState.updateStorm(topoId, base); } - Map, String> rawExecToComponent = computeExecutorToComponent(topoId, base); + Map, String> rawExecToComponent = computeExecutorToComponent(topoId, base, topoConf, topo); Map executorsToComponent = new HashMap<>(); for (Entry, String> entry: rawExecToComponent.entrySet()) { List execs = entry.getKey(); @@ -1425,31 +1476,104 @@ private TopologyDetails readTopologyDetails(String topoId, StormBase base) throw return new TopologyDetails(topoId, topoConf, topo, base.get_num_workers(), executorsToComponent, base.get_launch_time_secs(), base.get_owner()); } - - private void updateHeartbeats(String topoId, Set> allExecutors, Assignment existingAssignment) { - LOG.debug("Updating heartbeats for {} {}", topoId, allExecutors); + + private void updateHeartbeatsFromZkHeartbeat(String topoId, Set> allExecutors, Assignment existingAssignment) { + LOG.debug("Updating heartbeats for {} {} (from ZK heartbeat)", topoId, allExecutors); IStormClusterState state = stormClusterState; Map, Map> executorBeats = StatsUtil.convertExecutorBeats(state.executorBeats(topoId, existingAssignment.get_executor_node_port())); - Map, Map> cache = StatsUtil.updateHeartbeatCache(heartbeatsCache.get().get(topoId), - executorBeats, allExecutors, ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS))); + Map, Map> cache = StatsUtil.updateHeartbeatCacheFromZkHeartbeat(heartbeatsCache.get().get(topoId), + executorBeats, allExecutors, ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS))); heartbeatsCache.getAndUpdate(new Assoc<>(topoId, cache)); } - + + private void updateHeartbeats(String topoId, Set> allExecutors, Assignment existingAssignment) { + LOG.debug("Updating heartbeats for {} {}", topoId, allExecutors); + Map, Map> cache = heartbeatsCache.get().get(topoId); + if(cache == null) { + cache = new HashMap<>(); + heartbeatsCache.getAndUpdate(new Assoc<>(topoId, cache)); + } + StatsUtil.updateHeartbeatCache(heartbeatsCache.get().get(topoId), + null, allExecutors, ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS))); + } + /** * Update all the heartbeats for all the topologies' executors. * @param existingAssignments current assignments (thrift) * @param topologyToExecutors topology ID to executors. */ - private void updateAllHeartbeats(Map existingAssignments, Map>> topologyToExecutors) { + private void updateAllHeartbeats(Map existingAssignments, + Map>> topologyToExecutors, Set zkHeartbeatTopologies) { for (Entry entry: existingAssignments.entrySet()) { String topoId = entry.getKey(); - updateHeartbeats(topoId, topologyToExecutors.get(topoId), entry.getValue()); + if (zkHeartbeatTopologies.contains(topoId)) { + updateHeartbeatsFromZkHeartbeat(topoId, topologyToExecutors.get(topoId), entry.getValue()); + } else { + updateHeartbeats(topoId, topologyToExecutors.get(topoId), entry.getValue()); + } } } + + private void updateCachedHeartbeatsFromWorker(SupervisorWorkerHeartbeat workerHeartbeat) { + Map, Map> executorBeats = StatsUtil.convertWorkerBeats(workerHeartbeat); + String topoId = workerHeartbeat.get_storm_id(); + Map, Map> cache = heartbeatsCache.get().get(topoId); + if(cache == null) { + cache = new HashMap<>(); + heartbeatsCache.getAndUpdate(new Assoc<>(topoId, cache)); + } + Set> executors = new HashSet<>(); + for(ExecutorInfo executorInfo : workerHeartbeat.get_executors()) { + executors.add(Arrays.asList(executorInfo.get_task_start(), executorInfo.get_task_end())); + } + + StatsUtil.updateHeartbeatCache(heartbeatsCache.get().get(topoId), executorBeats, executors, + ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS))); + + } + + private void updateCachedHeartbeatsFromSupervisor(SupervisorWorkerHeartbeats workerHeartbeats) { + workerHeartbeats.get_worker_heartbeats().forEach(this::updateCachedHeartbeatsFromWorker); + if(!heartbeatsReadyFlag.get() && !Strings.isNullOrEmpty(workerHeartbeats.get_supervisor_id())) { + heartbeatsRecoveryStrategy.reportNodeId(workerHeartbeats.get_supervisor_id()); + } + } + + /** + * Decide if the heartbeats is recovered for a master, will wait for all the assignments nodes to recovery, + * every node will take care its node heartbeats reporting. + * @return true if all nodes have reported heartbeats or exceeds max-time-out + */ + private boolean isHeartbeatsRecovered() { + if(heartbeatsReadyFlag.get()) { + return true; + } + Set allNodes = new HashSet<>(); + for(Map.Entry assignmentEntry: stormClusterState.assignmentsInfo().entrySet()) { + allNodes.addAll(assignmentEntry.getValue().get_node_host().keySet()); + } + boolean isReady = heartbeatsRecoveryStrategy.isReady(allNodes); + if(isReady) { + heartbeatsReadyFlag.getAndSet(true); + } + return isReady; + } + + /** + * Decide if the assignments is synchronized. + * @return true if assignments have been synchronized from remote state store + */ + private boolean isAssignmentsRecovered() { + return stormClusterState.isAssignmentsBackendSynchronized(); + } private Set> aliveExecutors(TopologyDetails td, Set> allExecutors, Assignment assignment) { String topoId = td.getId(); Map, Map> hbCache = heartbeatsCache.get().get(topoId); + //in case that no workers report any heartbeats yet. + if (null == hbCache) { + hbCache = new HashMap<>(); + } LOG.debug("NEW Computing alive executors for {}\nExecutors: {}\nAssignment: {}\nHeartbeat cache: {}", topoId, allExecutors, assignment, hbCache); @@ -1464,9 +1588,11 @@ private Set> aliveExecutors(TopologyDetails td, Set> } Long startTime = execToStartTimes.get(longExec); - Boolean isTimedOut = (Boolean)hbCache.get(StatsUtil.convertExecutor(longExec)).get("is-timed-out"); + Map executorCache = hbCache.get(StatsUtil.convertExecutor(longExec)); + //null isTimedOut means worker never reported any heartbeat + Boolean isTimedOut = executorCache == null ? null : (Boolean)executorCache.get("is-timed-out"); Integer delta = startTime == null ? null : Time.deltaSecs(startTime.intValue()); - if (startTime != null && ((delta < taskLaunchSecs) || !isTimedOut)) { + if (startTime != null && ((delta < taskLaunchSecs) || (isTimedOut != null && !isTimedOut))) { ret.add(exec); } else { LOG.info("Executor {}:{} not alive", topoId, exec); @@ -1475,12 +1601,12 @@ private Set> aliveExecutors(TopologyDetails td, Set> return ret; } - private List> computeExecutors(String topoId, StormBase base) throws KeyNotFoundException, AuthorizationException, IOException, InvalidTopologyException { + private List> computeExecutors(String topoId, StormBase base, Map topoConf, + StormTopology topology) + throws KeyNotFoundException, AuthorizationException, IOException, InvalidTopologyException { assert (base != null); Map compToExecutors = base.get_component_executors(); - Map topoConf = readTopoConfAsNimbus(topoId, topoCache); - StormTopology topology = readStormTopologyAsNimbus(topoId, topoCache); List> ret = new ArrayList<>(); if (compToExecutors != null) { Map taskInfo = StormCommon.stormTaskInfo(topology, topoConf); @@ -1500,10 +1626,10 @@ private List> computeExecutors(String topoId, StormBase base) thro return ret; } - private Map, String> computeExecutorToComponent(String topoId, StormBase base) throws KeyNotFoundException, AuthorizationException, InvalidTopologyException, IOException { - List> executors = computeExecutors(topoId, base); - StormTopology topology = readStormTopologyAsNimbus(topoId, topoCache); - Map topoConf = readTopoConfAsNimbus(topoId, topoCache); + private Map, String> computeExecutorToComponent(String topoId, StormBase base, + Map topoConf, StormTopology topology) + throws KeyNotFoundException, AuthorizationException, InvalidTopologyException, IOException { + List> executors = computeExecutors(topoId, base, topoConf, topology); Map taskToComponent = StormCommon.stormTaskInfo(topology, topoConf); Map, String> ret = new HashMap<>(); for (List executor: executors) { @@ -1512,27 +1638,30 @@ private Map, String> computeExecutorToComponent(String topoId, Sto return ret; } - private Map>> computeTopologyToExecutors(Map bases) throws KeyNotFoundException, AuthorizationException, InvalidTopologyException, IOException { + private Map>> computeTopologyToExecutors(Map bases) + throws KeyNotFoundException, AuthorizationException, InvalidTopologyException, IOException { Map>> ret = new HashMap<>(); if (bases != null) { for (Entry entry: bases.entrySet()) { String topoId = entry.getKey(); - ret.put(topoId, new HashSet<>(computeExecutors(topoId, entry.getValue()))); + Map topoConf = readTopoConfAsNimbus(topoId, topoCache); + StormTopology topology = readStormTopologyAsNimbus(topoId, topoCache); + ret.put(topoId, new HashSet<>(computeExecutors(topoId, entry.getValue(), topoConf, topology))); } } return ret; } /** - * compute a topology-id -> alive executors map + * compute a topology-id -> alive executors map. * @param existingAssignment the current assignments * @param topologies the current topologies * @param topologyToExecutors the executors for the current topologies * @param scratchTopologyId the topology being rebalanced and should be excluded * @return the map of topology id to alive executors */ - private Map>> computeTopologyToAliveExecutors(Map existingAssignment, Topologies topologies, - Map>> topologyToExecutors, String scratchTopologyId) { + private Map>> computeTopologyToAliveExecutors(Map existingAssignment, + Topologies topologies, Map>> topologyToExecutors, String scratchTopologyId) { Map>> ret = new HashMap<>(); for (Entry entry: existingAssignment.entrySet()) { String topoId = entry.getKey(); @@ -1557,8 +1686,8 @@ private static List asIntExec(List exec) { return ret; } - private Map> computeSupervisorToDeadPorts(Map existingAssignments, Map>> topologyToExecutors, - Map>> topologyToAliveExecutors) { + private Map> computeSupervisorToDeadPorts(Map existingAssignments, + Map>> topologyToExecutors, Map>> topologyToAliveExecutors) { Map> ret = new HashMap<>(); for (Entry entry: existingAssignments.entrySet()) { String topoId = entry.getKey(); @@ -1591,7 +1720,7 @@ private Map> computeSupervisorToDeadPorts(Map computeTopologyToSchedulerAssignment(Map existingAssignments, - Map>> topologyToAliveExecutors) { + Map>> topologyToAliveExecutors) { Map ret = new HashMap<>(); for (Entry entry: existingAssignments.entrySet()) { String topoId = entry.getKey(); @@ -1622,6 +1751,7 @@ private Map computeTopologyToSchedulerAssignmen } /** + * Read supervisor details/exclude the dead slots. * @param superToDeadPorts dead ports on the supervisor * @param topologies all of the topologies * @param missingAssignmentTopologies topologies that need assignments @@ -1667,7 +1797,7 @@ private Map readAllSupervisorDetails(Map isFragmented(x) == true).mapToDouble(SupervisorResources::getAvailableMem).filter(x -> x > 0).sum(); + Double res = nodeIdToResources.get().values().parallelStream().filter(x -> isFragmented(x) == true) + .mapToDouble(SupervisorResources::getAvailableMem).filter(x -> x > 0).sum(); return res.intValue(); } private int fragmentedCpu() { - Double res = nodeIdToResources.get().values().parallelStream().filter(x -> isFragmented(x) == true).mapToDouble(SupervisorResources::getAvailableCpu).filter(x -> x > 0).sum(); + Double res = nodeIdToResources.get().values().parallelStream().filter(x -> isFragmented(x) == true) + .mapToDouble(SupervisorResources::getAvailableCpu).filter(x -> x > 0).sum(); return res.intValue(); } private Map computeNewSchedulerAssignments(Map existingAssignments, - Topologies topologies, Map bases, String scratchTopologyId) throws KeyNotFoundException, AuthorizationException, InvalidTopologyException, IOException { + Topologies topologies, Map bases, String scratchTopologyId) + throws KeyNotFoundException, AuthorizationException, InvalidTopologyException, IOException { + Map>> topoToExec = computeTopologyToExecutors(bases); - - updateAllHeartbeats(existingAssignments, topoToExec); + + Set zkHeartbeatTopologies = topologies.getTopologies().stream() + .filter(topo -> !supportRpcHeartbeat(topo)) + .map(TopologyDetails::getId) + .collect(Collectors.toSet()); + + updateAllHeartbeats(existingAssignments, topoToExec, zkHeartbeatTopologies); Map>> topoToAliveExecutors = computeTopologyToAliveExecutors(existingAssignments, topologies, topoToExec, scratchTopologyId); @@ -1723,7 +1862,7 @@ private Map computeNewSchedulerAssignments(Map computeNewSchedulerAssignments(Map= 0; + } + + /** + * Diff old/new assignment to find nodes which assigned assignments has changed. + * @param oldAss old assigned assignment + * @param newAss new assigned assignment + * @return nodeId -> host map of assignments changed nodes + */ + private static Map assignmentChangedNodes(Assignment oldAss, Assignment newAss) { + Map, NodeInfo> oldExecutorNodePort = null; + Map, NodeInfo> newExecutorNodePort = null; + Map allNodeHost = new HashMap<>(); + if(oldAss != null) { + oldExecutorNodePort = oldAss.get_executor_node_port(); + allNodeHost.putAll(oldAss.get_node_host()); + } + if(newAss != null) { + newExecutorNodePort = newAss.get_executor_node_port(); + allNodeHost.putAll(newAss.get_node_host()); + } + //kill or newly submit + if (oldAss == null || newAss == null) { + return allNodeHost; + } else { + // rebalance + Map ret = new HashMap(); + for(Map.Entry, NodeInfo> entry: newExecutorNodePort.entrySet()) { + NodeInfo newNodeInfo = entry.getValue(); + NodeInfo oldNodeInfo = oldExecutorNodePort.get(entry.getKey()); + if(null != oldNodeInfo) { + if(!oldNodeInfo.equals(newNodeInfo)){ + ret.put(oldNodeInfo.get_node(), allNodeHost.get(oldNodeInfo.get_node())); + ret.put(newNodeInfo.get_node(), allNodeHost.get(newNodeInfo.get_node())); + } + }else { + ret.put(newNodeInfo.get_node(), allNodeHost.get(newNodeInfo.get_node())); + } + } + + return ret; + } + } + + /** + * Pick out assignments for specific node from all assignments. + * @param assignmentMap stormId -> assignment map + * @param nodeId supervisor/node id + * @return stormId -> assignment map for the node + */ + private static Map assignmentsForNode(Map assignmentMap, String nodeId) { + Map ret = new HashMap<>(); + assignmentMap.entrySet().stream().filter(assignmentEntry -> assignmentEntry.getValue().get_node_host().keySet() + .contains(nodeId)).forEach(assignmentEntry -> { ret.put(assignmentEntry.getKey(), assignmentEntry.getValue()); }); + + return ret; + } + + /** + * Notify supervisors/nodes assigned assignments. + * @param assignments assignments map for nodes + * @param service {@link AssignmentDistributionService} for distributing assignments asynchronous + * @param nodeHost node -> host map + * @param supervisorDetails nodeId -> {@link SupervisorDetails} map + */ + private static void notifySupervisorsAssignments(Map assignments, + AssignmentDistributionService service, Map nodeHost, + Map supervisorDetails) { + for(Map.Entry nodeEntry: nodeHost.entrySet()) { + try{ + String nodeId = nodeEntry.getKey(); + SupervisorAssignments supervisorAssignments = new SupervisorAssignments(); + supervisorAssignments.set_storm_assignment(assignmentsForNode(assignments, nodeEntry.getKey())); + SupervisorDetails details = supervisorDetails.get(nodeId); + Integer serverPort = details != null ? details.getServerPort() : null; + service.addAssignmentsForNode(nodeId, nodeEntry.getValue(), serverPort, supervisorAssignments); + } catch (Throwable tr1) { + //just skip when any error happens wait for next round assignments reassign + LOG.error("Exception when add assignments distribution task for node {}", nodeEntry.getKey()); + } + } + } + + private static void notifySupervisorsAsKilled(IStormClusterState clusterState, Assignment oldAss, + AssignmentDistributionService service) { + Map nodeHost = assignmentChangedNodes(oldAss, null); + notifySupervisorsAssignments(clusterState.assignmentsInfo(), service, nodeHost, + basicSupervisorDetailsMap(clusterState)); + } - private TopologyResources getResourcesForTopology(String topoId, StormBase base) throws NotAliveException, AuthorizationException, InvalidTopologyException, IOException { + private TopologyResources getResourcesForTopology(String topoId, StormBase base) + throws NotAliveException, AuthorizationException, InvalidTopologyException, IOException { TopologyResources ret = idToResources.get().get(topoId); if (ret == null) { try { @@ -1784,13 +2023,27 @@ private Map getWorkerResourcesForTopology(String to return ret; } + private boolean isReadyForMKAssignments() throws Exception { + if (isLeader()) { + if (isHeartbeatsRecovered()) { + if (isAssignmentsRecovered()) { + return true; + } + LOG.warn("waiting for assignments recovery, skipping assignments"); + } + LOG.warn("waiting for worker heartbeats recovery, skipping assignments"); + } else { + LOG.info("not a leader, skipping assignments"); + } + return false; + } + private void mkAssignments() throws Exception { mkAssignments(null); } private void mkAssignments(String scratchTopoId) throws Exception { - if (!isLeader()) { - LOG.info("not a leader, skipping assignments"); + if (!isReadyForMKAssignments()) { return; } // get existing assignment (just the topologyToExecutorToNodePort map) -> default to {} @@ -1947,6 +2200,19 @@ private void mkAssignments(String scratchTopoId) throws Exception { } } + //grouping assignment by node to see the nodes diff, then notify nodes/supervisors to synchronize its owned assignment + //because the number of existing assignments is small for every scheduling round, + //we expect to notify supervisors at almost the same time + Map totalAssignmentsChangedNodes = new HashMap<>(); + for(Entry entry: newAssignments.entrySet()) { + String topoId = entry.getKey(); + Assignment assignment = entry.getValue(); + Assignment existingAssignment = existingAssignments.get(topoId); + totalAssignmentsChangedNodes.putAll(assignmentChangedNodes(existingAssignment, assignment)); + } + notifySupervisorsAssignments(newAssignments, assignmentsDistributer, totalAssignmentsChangedNodes, + basicSupervisorDetailsMap); + Map> addedSlots = new HashMap<>(); for (Entry entry: newAssignments.entrySet()) { String topoId = entry.getKey(); @@ -2016,7 +2282,8 @@ private void assertTopoActive(String topoName, boolean expectActive) throws NotA } } - private Map tryReadTopoConfFromName(final String topoName) throws NotAliveException, AuthorizationException, IOException { + private Map tryReadTopoConfFromName(final String topoName) throws NotAliveException, + AuthorizationException, IOException { IStormClusterState state = stormClusterState; String topoId = state.getTopoId(topoName) .orElseThrow(() -> new NotAliveException(topoName + " is not alive")); @@ -2024,12 +2291,14 @@ private Map tryReadTopoConfFromName(final String topoName) throw } @VisibleForTesting - public void checkAuthorization(String topoName, Map topoConf, String operation) throws AuthorizationException { + public void checkAuthorization(String topoName, Map topoConf, String operation) + throws AuthorizationException { checkAuthorization(topoName, topoConf, operation, null); } @VisibleForTesting - public void checkAuthorization(String topoName, Map topoConf, String operation, ReqContext context) throws AuthorizationException { + public void checkAuthorization(String topoName, Map topoConf, String operation, ReqContext context) + throws AuthorizationException { IAuthorizer aclHandler = authorizationHandler; IAuthorizer impersonationAuthorizer = impersonationAuthorizationHandler; if (context == null) { @@ -2046,7 +2315,8 @@ public void checkAuthorization(String topoName, Map topoConf, St LOG.warn("principal: {} is trying to impersonate principal: {}", context.realPrincipal(), context.principal()); if (impersonationAuthorizer == null) { LOG.warn("impersonation attempt but {} has no authorizer configured. potential security risk, " - + "please see SECURITY.MD to learn how to configure impersonation authorizer.", DaemonConfig.NIMBUS_IMPERSONATION_AUTHORIZER); + + "please see SECURITY.MD to learn how to configure impersonation authorizer.", + DaemonConfig.NIMBUS_IMPERSONATION_AUTHORIZER); } else { if (!impersonationAuthorizer.permit(context, operation, checkConf)) { ThriftAccessLogger.logAccess(context.requestID(), context.remoteAddress(), @@ -2085,7 +2355,8 @@ private boolean isAuthorized(String operation, String topoId) throws NotAliveExc } @VisibleForTesting - public Set filterAuthorized(String operation, Collection topoIds) throws NotAliveException, AuthorizationException, IOException { + public Set filterAuthorized(String operation, Collection topoIds) throws NotAliveException, + AuthorizationException, IOException { Set ret = new HashSet<>(); for (String topoId : topoIds) { if (isAuthorized(operation, topoId)) { @@ -2440,7 +2711,8 @@ private void sendClusterMetricsToExecutors() throws Exception { } } - private CommonTopoInfo getCommonTopoInfo(String topoId, String operation) throws NotAliveException, AuthorizationException, IOException, InvalidTopologyException { + private CommonTopoInfo getCommonTopoInfo(String topoId, String operation) throws NotAliveException, + AuthorizationException, IOException, InvalidTopologyException { IStormClusterState state = stormClusterState; CommonTopoInfo ret = new CommonTopoInfo(); ret.topoConf = tryReadTopoConf(topoId, topoCache); @@ -2455,7 +2727,9 @@ private CommonTopoInfo getCommonTopoInfo(String topoId, String operation) throws ret.launchTimeSecs = 0; } ret.assignment = state.assignmentInfo(topoId, null); - ret.beats = Utils.OR(heartbeatsCache.get().get(topoId), Collections.emptyMap()); + //get it from cluster state/zookeeper every time to collect the UI stats, may replace it with other StateStore later + ret.beats = ret.assignment != null? StatsUtil.convertExecutorBeats(state.executorBeats(topoId, + ret.assignment.get_executor_node_port())) : Collections.emptyMap(); ret.allComponents = new HashSet<>(ret.taskToComponent.values()); return ret; } @@ -2593,7 +2867,7 @@ public void launchServer() throws Exception { @Override public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) - throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, TException { + throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, TException { submitTopologyCalls.mark(); submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, new SubmitOptions(TopologyInitialStatus.ACTIVE)); } @@ -2678,9 +2952,9 @@ private void upsertWorkerTokensInCreds(Map creds, String user, S } @Override - public void submitTopologyWithOpts(String topoName, String uploadedJarLocation, String jsonConf, StormTopology topology, - SubmitOptions options) - throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, TException { + public void submitTopologyWithOpts(String topoName, String uploadedJarLocation, String jsonConf, + StormTopology topology, SubmitOptions options) + throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, TException { try { submitTopologyWithOptsCalls.mark(); assertIsLeader(); @@ -2764,6 +3038,10 @@ public void submitTopologyWithOpts(String topoName, String uploadedJarLocation, IStormClusterState state = stormClusterState; + if (creds == null && workerTokenManager != null) { + //Make sure we can store the worker tokens even if no creds are provided. + creds = new HashMap<>(); + } if (creds != null) { Map finalConf = Collections.unmodifiableMap(topoConf); for (INimbusCredentialPlugin autocred: nimbusAutocredPlugins) { @@ -2833,7 +3111,7 @@ public void killTopology(String name) throws NotAliveException, AuthorizationExc @Override public void killTopologyWithOpts(final String topoName, final KillOptions options) - throws NotAliveException, AuthorizationException, TException { + throws NotAliveException, AuthorizationException, TException { killTopologyWithOptsCalls.mark(); assertTopoActive(topoName, true); try { @@ -2897,7 +3175,7 @@ public void deactivate(String topoName) throws NotAliveException, AuthorizationE @Override public void rebalance(String topoName, RebalanceOptions options) - throws NotAliveException, InvalidTopologyException, AuthorizationException, TException { + throws NotAliveException, InvalidTopologyException, AuthorizationException, TException { rebalanceCalls.mark(); assertTopoActive(topoName, true); try { @@ -3024,7 +3302,7 @@ public LogConfig getLogConfig(String topoId) throws TException { @Override public void debug(String topoName, String componentId, boolean enable, double samplingPercentage) - throws NotAliveException, AuthorizationException, TException { + throws NotAliveException, AuthorizationException, TException { debugCalls.mark(); try { IStormClusterState state = stormClusterState; @@ -3087,7 +3365,7 @@ public void setWorkerProfiler(String topoId, ProfileRequest profileRequest) thro @Override public List getComponentPendingProfileActions(String id, String componentId, ProfileAction action) - throws TException { + throws TException { try { getComponentPendingProfileActionsCalls.mark(); CommonTopoInfo info = getCommonTopoInfo(id, "getComponentPendingProfileActions"); @@ -3134,7 +3412,7 @@ public List getComponentPendingProfileActions(String id, String @Override public void uploadNewCredentials(String topoName, Credentials credentials) - throws NotAliveException, InvalidTopologyException, AuthorizationException, TException { + throws NotAliveException, InvalidTopologyException, AuthorizationException, TException { try { uploadNewCredentialsCalls.mark(); IStormClusterState state = stormClusterState; @@ -3171,7 +3449,7 @@ public void uploadNewCredentials(String topoName, Credentials credentials) @SuppressWarnings("deprecation") @Override public String beginCreateBlob(String key, SettableBlobMeta meta) - throws AuthorizationException, KeyAlreadyExistsException, TException { + throws AuthorizationException, KeyAlreadyExistsException, TException { try { String sessionId = Utils.uuid(); blobUploaders.put(sessionId, blobStore.createBlob(key, meta, getSubject())); @@ -3281,7 +3559,7 @@ public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, K @Override public void setBlobMeta(String key, SettableBlobMeta meta) - throws AuthorizationException, KeyNotFoundException, TException { + throws AuthorizationException, KeyNotFoundException, TException { try { blobStore.setBlobMeta(key, meta, getSubject()); } catch (Exception e) { @@ -3296,7 +3574,7 @@ public void setBlobMeta(String key, SettableBlobMeta meta) @SuppressWarnings("deprecation") @Override public BeginDownloadResult beginBlobDownload(String key) - throws AuthorizationException, KeyNotFoundException, TException { + throws AuthorizationException, KeyNotFoundException, TException { try { InputStreamWithMeta is = blobStore.getBlob(key, getSubject()); String sessionId = Utils.uuid(); @@ -3416,7 +3694,7 @@ public int getBlobReplication(String key) throws AuthorizationException, KeyNotF @Override public int updateBlobReplication(String key, int replication) - throws AuthorizationException, KeyNotFoundException, TException { + throws AuthorizationException, KeyNotFoundException, TException { try { return blobStore.updateBlobReplication(key, replication, getSubject()); } catch (Exception e) { @@ -3587,7 +3865,7 @@ public TopologyInfo getTopologyInfo(String id) throws NotAliveException, Authori @Override public TopologyInfo getTopologyInfoWithOpts(String topoId, GetInfoOptions options) - throws NotAliveException, AuthorizationException, TException { + throws NotAliveException, AuthorizationException, TException { try { getTopologyInfoWithOptsCalls.mark(); CommonTopoInfo common = getCommonTopoInfo(topoId, "getTopologyInfo"); @@ -3630,18 +3908,16 @@ public TopologyInfo getTopologyInfoWithOpts(String topoId, GetInfoOptions option if (heartbeat == null) { heartbeat = Collections.emptyMap(); } - ExecutorSummary summ = new ExecutorSummary(execInfo, common.taskToComponent.get(execInfo.get_task_start()), + ExecutorSummary summ = new ExecutorSummary(execInfo, + common.taskToComponent.get(execInfo.get_task_start()), nodeToHost.get(ni.get_node()), ni.get_port_iterator().next().intValue(), (Integer) heartbeat.getOrDefault("uptime", 0)); //heartbeats "stats" - Map hb = (Map)heartbeat.get("heartbeat"); - if (hb != null) { - Map ex = (Map) hb.get("stats"); - if (ex != null) { - ExecutorStats stats = StatsUtil.thriftifyExecutorStats(ex); - summ.set_stats(stats); - } + Map ex = (Map) heartbeat.get("stats"); + if (ex != null) { + ExecutorStats stats = StatsUtil.thriftifyExecutorStats(ex); + summ.set_stats(stats); } summaries.add(summ); } @@ -3684,7 +3960,7 @@ public TopologyInfo getTopologyInfoWithOpts(String topoId, GetInfoOptions option @Override public TopologyPageInfo getTopologyPageInfo(String topoId, String window, boolean includeSys) - throws NotAliveException, AuthorizationException, TException { + throws NotAliveException, AuthorizationException, TException { try { getTopologyPageInfoCalls.mark(); CommonTopoInfo common = getCommonTopoInfo(topoId, "getTopologyPageInfo"); @@ -3805,7 +4081,7 @@ public TopologyPageInfo getTopologyPageInfo(String topoId, String window, boolea @Override public SupervisorPageInfo getSupervisorPageInfo(String superId, String host, boolean includeSys) - throws NotAliveException, AuthorizationException, TException { + throws NotAliveException, AuthorizationException, TException { try { getSupervisorPageInfoCalls.mark(); IStormClusterState state = stormClusterState; @@ -3827,7 +4103,7 @@ public SupervisorPageInfo getSupervisorPageInfo(String superId, String host, boo supervisorIds = Arrays.asList(superId); } SupervisorPageInfo pageInfo = new SupervisorPageInfo(); - Map topoToAssignment = state.topologyAssignments(); + Map topoToAssignment = state.assignmentsInfo(); for (String sid: supervisorIds) { SupervisorInfo info = superInfos.get(sid); LOG.info("SIDL {} SI: {} ALL: {}", sid, info, superInfos); @@ -3873,8 +4149,8 @@ public SupervisorPageInfo getSupervisorPageInfo(String superId, String host, boo } @Override - public ComponentPageInfo getComponentPageInfo(String topoId, String componentId, String window, - boolean includeSys) throws NotAliveException, AuthorizationException, TException { + public ComponentPageInfo getComponentPageInfo(String topoId, String componentId, String window, boolean includeSys) + throws NotAliveException, AuthorizationException, TException { try { getComponentPageInfoCalls.mark(); CommonTopoInfo info = getCommonTopoInfo(topoId, "getComponentPageInfo"); @@ -4099,7 +4375,7 @@ public List getOwnerResourceSummaries(String owner) throws getOwnerResourceSummariesCalls.mark(); checkAuthorization(null, null, "getOwnerResourceSummaries"); IStormClusterState state = stormClusterState; - Map topoIdToAssignments = state.topologyAssignments(); + Map topoIdToAssignments = state.assignmentsInfo(); Map topoIdToBases = state.topologyBases(); Map clusterSchedulerConfig = scheduler.config(); @@ -4216,7 +4492,67 @@ public List getOwnerResourceSummaries(String owner) throws throw new RuntimeException(e); } } - + + @Override + public SupervisorAssignments getSupervisorAssignments(String node) throws AuthorizationException, TException { + checkAuthorization(null, null, "getSupervisorAssignments"); + try { + if (isLeader() && isAssignmentsRecovered()) { + SupervisorAssignments supervisorAssignments = new SupervisorAssignments(); + supervisorAssignments.set_storm_assignment(assignmentsForNode(stormClusterState.assignmentsInfo(), node)); + return supervisorAssignments; + } + } catch (Exception e) { + LOG.debug("Exception when node {} fetching assignments", node); + if (e instanceof TException) { + throw (TException)e; + } + // When this master is not leader and get a sync request from node, + // just return nil which will cause client/node to get an unknown error, + // the node/supervisor will sync it as a timer task. + LOG.debug("Exception when node {} fetching assignments", node); + } + return null; + } + + @Override + public void sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats) + throws AuthorizationException, TException { + checkAuthorization(null, null, "sendSupervisorWorkerHeartbeats"); + try { + if (isLeader()) { + updateCachedHeartbeatsFromSupervisor(heartbeats); + } + } catch (Exception e) { + LOG.debug("Exception when update heartbeats for node {} heartbeats report.", + heartbeats.get_supervisor_id()); + if (e instanceof TException) { + throw (TException)e; + } + // When this master is not leader and get heartbeats report from supervisor/node, just ignore it. + } + } + + @Override + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat hb) throws AuthorizationException, TException { + String id = hb.get_storm_id(); + try { + Map topoConf = tryReadTopoConf(id, topoCache); + topoConf = Utils.merge(conf, topoConf); + String topoName = (String) topoConf.get(Config.TOPOLOGY_NAME); + checkAuthorization(topoName, topoConf, "sendSupervisorWorkerHeartbeat"); + if (isLeader()) { + updateCachedHeartbeatsFromWorker(hb); + } + } catch (Exception e) { + LOG.warn("Send HB exception. (topology id='{}')", id, e); + if (e instanceof TException) { + throw (TException)e; + } + throw new RuntimeException(e); + } + } + // Shutdownable methods @SuppressWarnings("deprecation") @@ -4234,6 +4570,7 @@ public void shutdown() { blobListers.cleanup(); blobStore.shutdown(); leaderElector.close(); + assignmentsDistributer.close(); ITopologyActionNotifierPlugin actionNotifier = nimbusTopologyActionNotifier; if (actionNotifier != null) { actionNotifier.cleanup(); diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java index f21c818bf87..e2ecbfb3514 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java @@ -18,6 +18,7 @@ package org.apache.storm.daemon.supervisor; +import static org.apache.storm.daemon.nimbus.Nimbus.MIN_VERSION_SUPPORT_RPC_HEARTBEAT; import static org.apache.storm.utils.Utils.OR; import com.google.common.base.Joiner; @@ -95,27 +96,30 @@ public void call(int exitCode) { } /** - * Create a new BasicContainer + * Create a new BasicContainer. * @param type the type of container being made. * @param conf the supervisor config * @param supervisorId the ID of the supervisor this is a part of. + * @param supervisorPort the thrift server port of the supervisor this is a part of. * @param port the port the container is on. Should be <= 0 if only a partial recovery * @param assignment the assignment for this container. Should be null if only a partial recovery. * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used. * @param localState the local state of the supervisor. May be null if partial recovery * @param workerId the id of the worker to use. Must not be null if doing a partial recovery. */ - public BasicContainer(ContainerType type, Map conf, String supervisorId, int port, - LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, + public BasicContainer(ContainerType type, Map conf, String supervisorId, int supervisorPort, + int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, LocalState localState, String workerId) throws IOException { - this(type, conf, supervisorId, port, assignment, resourceIsolationManager, localState, workerId, null, null, null); + this(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, localState, + workerId, null, null, null); } /** - * Create a new BasicContainer + * Create a new BasicContainer. * @param type the type of container being made. * @param conf the supervisor config * @param supervisorId the ID of the supervisor this is a part of. + * @param supervisorPort the thrift server port of the supervisor this is a part of. * @param port the port the container is on. Should be <= 0 if only a partial recovery * @param assignment the assignment for this container. Should be null if only a partial recovery. * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used. @@ -128,11 +132,11 @@ public BasicContainer(ContainerType type, Map conf, String super * @throws IOException on any error * @throws ContainerRecoveryException if the Container could not be recovered. */ - BasicContainer(ContainerType type, Map conf, String supervisorId, int port, + BasicContainer(ContainerType type, Map conf, String supervisorId, int supervisorPort, int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, LocalState localState, String workerId, Map topoConf, AdvancedFSOps ops, String profileCmd) throws IOException { - super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops); + super(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, workerId, topoConf, ops); assert(localState != null); _localState = localState; @@ -151,7 +155,7 @@ public BasicContainer(ContainerType type, Map conf, String super LOG.info("Recovered Worker {}", wid); _workerId = wid; } - } else if (_workerId == null){ + } else if (_workerId == null) { createNewWorkerId(); } @@ -237,7 +241,7 @@ public boolean didMainProcessExit() { } /** - * Run the given command for profiling + * Run the given command for profiling. * * @param command * the command to run @@ -291,7 +295,7 @@ public boolean runProfiling(ProfileRequest request, boolean stop) throws IOExcep } /** - * Get the command to run when doing profiling + * Get the command to run when doing profiling. * @param action the profiling action to perform * @param stop if this is meant to stop the profiling or start it * @param workerPid the PID of the process to profile @@ -422,7 +426,7 @@ private List asStringList(Object o) { } /** - * Compute the classpath for the worker process + * Compute the classpath for the worker process. * @param stormJar the topology jar * @param dependencyLocations any dependencies from the topology * @param topoVersion the version of the storm framework to use @@ -485,7 +489,7 @@ protected List substituteChildopts(Object value, int memOnheap) { } /** - * Launch the worker process (non-blocking) + * Launch the worker process (non-blocking). * * @param command * the command to run @@ -624,7 +628,7 @@ public static String getStormVersionFor(final Map conf, final St /** * Get parameters for the class path of the worker process. Also used by the - * log Writer + * log Writer. * @param stormRoot the root dist dir for the topology * @return the classpath for the topology as command line arguments. * @throws IOException on any error. @@ -700,7 +704,7 @@ protected String javaCmd(String cmd) { } /** - * Create the command to launch the worker process + * Create the command to launch the worker process. * @param memOnheap the on heap memory for the worker * @param stormRoot the root dist dir for the topology * @param jlp java library path for the topology @@ -751,6 +755,14 @@ private List mkLaunchCommand(final int memOnheap, final String stormRoot commandList.add(getWorkerMain(topoVersion)); commandList.add(_topologyId); commandList.add(_supervisorId); + + // supervisor port should be only presented to worker which supports RPC heartbeat + // unknown version should be treated as "current version", which supports RPC heartbeat + if ((topoVersion.getMajor() == -1 && topoVersion.getMinor() == -1) || + topoVersion.compareTo(MIN_VERSION_SUPPORT_RPC_HEARTBEAT) >= 0) { + commandList.add(String.valueOf(_supervisorPort)); + } + commandList.add(String.valueOf(_port)); commandList.add(_workerId); diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainerLauncher.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainerLauncher.java index 4915650217c..e2043e8ecce 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainerLauncher.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainerLauncher.java @@ -31,18 +31,21 @@ public class BasicContainerLauncher extends ContainerLauncher { private final Map _conf; private final String _supervisorId; + private final int _supervisorPort; protected final ResourceIsolationInterface _resourceIsolationManager; - public BasicContainerLauncher(Map conf, String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException { + public BasicContainerLauncher(Map conf, String supervisorId, int supervisorPort, + ResourceIsolationInterface resourceIsolationManager) throws IOException { _conf = conf; _supervisorId = supervisorId; + _supervisorPort = supervisorPort; _resourceIsolationManager = resourceIsolationManager; } @Override public Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException { - Container container = new BasicContainer(ContainerType.LAUNCH, _conf, _supervisorId, port, assignment, - _resourceIsolationManager, state, null); + Container container = new BasicContainer(ContainerType.LAUNCH, _conf, _supervisorId, _supervisorPort, port, + assignment, _resourceIsolationManager, state, null); container.setup(); container.launch(); return container; @@ -50,13 +53,13 @@ public Container launchContainer(int port, LocalAssignment assignment, LocalStat @Override public Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException { - return new BasicContainer(ContainerType.RECOVER_FULL, _conf, _supervisorId, port, assignment, + return new BasicContainer(ContainerType.RECOVER_FULL, _conf, _supervisorId, _supervisorPort, port, assignment, _resourceIsolationManager, state, null); } @Override public Killable recoverContainer(String workerId, LocalState localState) throws IOException { - return new BasicContainer(ContainerType.RECOVER_PARTIAL, _conf, _supervisorId, -1, null, + return new BasicContainer(ContainerType.RECOVER_PARTIAL, _conf, _supervisorId, _supervisorPort, -1, null, _resourceIsolationManager, localState, workerId); } } diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java index 9958f1bc971..10d81e28c81 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java @@ -145,6 +145,7 @@ public String toString() { protected String _workerId; protected final String _topologyId; //Not set if RECOVER_PARTIAL protected final String _supervisorId; + protected final int _supervisorPort; protected final int _port; //Not set if RECOVER_PARTIAL protected final LocalAssignment _assignment; //Not set if RECOVER_PARTIAL protected final AdvancedFSOps _ops; @@ -158,6 +159,7 @@ public String toString() { * @param type the type of container being made. * @param conf the supervisor config * @param supervisorId the ID of the supervisor this is a part of. + * @param supervisorPort the thrift server port of the supervisor this is a part of. * @param port the port the container is on. Should be <= 0 if only a partial recovery * @param assignment the assignment for this container. Should be null if only a partial recovery. * @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used. @@ -167,7 +169,7 @@ public String toString() { * @param ops file system operations (mostly for testing) if null a new one is made * @throws IOException on any error. */ - protected Container(ContainerType type, Map conf, String supervisorId, + protected Container(ContainerType type, Map conf, String supervisorId, int supervisorPort, int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, String workerId, Map topoConf, AdvancedFSOps ops) throws IOException { assert(type != null); @@ -186,6 +188,7 @@ protected Container(ContainerType type, Map conf, String supervi _ops = ops; _conf = conf; _supervisorId = supervisorId; + _supervisorPort = supervisorPort; _resourceIsolationManager = resourceIsolationManager; _assignment = assignment; diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerLauncher.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerLauncher.java index 70a8a522280..527b321120b 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerLauncher.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerLauncher.java @@ -43,13 +43,15 @@ public abstract class ContainerLauncher { * for the config and the environment. * @param conf the config * @param supervisorId the ID of the supervisor + * @param supervisorPort the parent supervisor thrift server port * @param sharedContext Used in local mode to let workers talk together without netty * @return the proper container launcher * @throws IOException on any error */ - public static ContainerLauncher make(Map conf, String supervisorId, IContext sharedContext) throws IOException { + public static ContainerLauncher make(Map conf, String supervisorId, int supervisorPort, + IContext sharedContext) throws IOException { if (ConfigUtils.isLocalMode(conf)) { - return new LocalContainerLauncher(conf, supervisorId, sharedContext); + return new LocalContainerLauncher(conf, supervisorId, supervisorPort, sharedContext); } ResourceIsolationInterface resourceIsolationManager = null; @@ -60,9 +62,9 @@ public static ContainerLauncher make(Map conf, String supervisor } if (ObjectReader.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) { - return new RunAsUserContainerLauncher(conf, supervisorId, resourceIsolationManager); + return new RunAsUserContainerLauncher(conf, supervisorId, supervisorPort, resourceIsolationManager); } - return new BasicContainerLauncher(conf, supervisorId, resourceIsolationManager); + return new BasicContainerLauncher(conf, supervisorId, supervisorPort, resourceIsolationManager); } protected ContainerLauncher() { diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainer.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainer.java index dbc71e9d567..4afaffe469b 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainer.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainer.java @@ -34,8 +34,9 @@ public class LocalContainer extends Container { private volatile boolean _isAlive = false; private final IContext _sharedContext; - public LocalContainer(Map conf, String supervisorId, int port, LocalAssignment assignment, IContext sharedContext) throws IOException { - super(ContainerType.LAUNCH, conf, supervisorId, port, assignment, null, null, null, null); + public LocalContainer(Map conf, String supervisorId, int supervisorPort, int port, + LocalAssignment assignment, IContext sharedContext) throws IOException { + super(ContainerType.LAUNCH, conf, supervisorId, supervisorPort, port, assignment, null, null, null, null); _sharedContext = sharedContext; _workerId = Utils.uuid(); } @@ -52,7 +53,7 @@ protected void createBlobstoreLinks() { @Override public void launch() throws IOException { - Worker worker = new Worker(_conf, _sharedContext, _topologyId, _supervisorId, _port, _workerId); + Worker worker = new Worker(_conf, _sharedContext, _topologyId, _supervisorId, _supervisorPort, _port, _workerId); try { worker.start(); } catch (Exception e) { diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainerLauncher.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainerLauncher.java index c25bc49e0a1..1c4312865e3 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainerLauncher.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainerLauncher.java @@ -30,17 +30,20 @@ public class LocalContainerLauncher extends ContainerLauncher { private final Map _conf; private final String _supervisorId; + private final int _supervisorPort; private final IContext _sharedContext; - public LocalContainerLauncher(Map conf, String supervisorId, IContext sharedContext) { + public LocalContainerLauncher(Map conf, String supervisorId, int supervisorPort, + IContext sharedContext) { _conf = conf; _supervisorId = supervisorId; + _supervisorPort = supervisorPort; _sharedContext = sharedContext; } @Override public Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException { - LocalContainer ret = new LocalContainer(_conf, _supervisorId, port, assignment, _sharedContext); + LocalContainer ret = new LocalContainer(_conf, _supervisorId, _supervisorPort, port, assignment, _sharedContext); ret.setup(); ret.launch(); return ret; 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 23b8b64f973..e065f38ea44 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 @@ -31,10 +31,8 @@ import org.apache.storm.DaemonConfig; import org.apache.storm.cluster.IStormClusterState; -import org.apache.storm.cluster.VersionedData; import org.apache.storm.daemon.supervisor.Slot.MachineState; import org.apache.storm.daemon.supervisor.Slot.TopoProfileAction; -import org.apache.storm.event.EventManager; import org.apache.storm.generated.Assignment; import org.apache.storm.generated.ExecutorInfo; import org.apache.storm.generated.LocalAssignment; @@ -56,11 +54,10 @@ public class ReadClusterState implements Runnable, AutoCloseable { private final Map superConf; private final IStormClusterState stormClusterState; - private final EventManager syncSupEventManager; - private final AtomicReference>> assignmentVersions; private final Map slots = new HashMap<>(); private final AtomicInteger readRetry = new AtomicInteger(0); private final String assignmentId; + private final int supervisorPort; private final ISupervisor iSuper; private final AsyncLocalizer localizer; private final ContainerLauncher launcher; @@ -73,17 +70,16 @@ public class ReadClusterState implements Runnable, AutoCloseable { public ReadClusterState(Supervisor supervisor) throws Exception { this.superConf = supervisor.getConf(); this.stormClusterState = supervisor.getStormClusterState(); - this.syncSupEventManager = supervisor.getEventManger(); - this.assignmentVersions = new AtomicReference<>(new HashMap<>()); this.assignmentId = supervisor.getAssignmentId(); + this.supervisorPort = supervisor.getThriftServerPort(); this.iSuper = supervisor.getiSupervisor(); this.localizer = supervisor.getAsyncLocalizer(); this.host = supervisor.getHostName(); this.localState = supervisor.getLocalState(); this.cachedAssignments = supervisor.getCurrAssignment(); this.metricsExec = new OnlyLatestExecutor<>(supervisor.getHeartbeatExecutor()); - - this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisor.getSharedContext()); + + this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisorPort, supervisor.getSharedContext()); this.metricsProcessor = null; try { @@ -92,7 +88,7 @@ public ReadClusterState(Supervisor supervisor) throws Exception { // the metrics processor is not critical to the operation of the cluster, allow Supervisor to come up LOG.error("Failed to initialize metric processor", e); } - + @SuppressWarnings("unchecked") List ports = (List)superConf.get(DaemonConfig.SUPERVISOR_SLOTS_PORTS); for (Number port: ports) { @@ -127,13 +123,10 @@ private Slot mkSlot(int port) throws Exception { @Override public synchronized void run() { try { - Runnable syncCallback = new EventManagerPushCallback(this, syncSupEventManager); - List stormIds = stormClusterState.assignments(syncCallback); - Map> assignmentsSnapshot = - getAssignmentsSnapshot(stormClusterState, stormIds, assignmentVersions.get(), syncCallback); + List stormIds = stormClusterState.assignments(null); + Map assignmentsSnapshot = getAssignmentsSnapshot(stormClusterState); - Map allAssignments = - readAssignments(assignmentsSnapshot); + Map allAssignments = readAssignments(assignmentsSnapshot); if (allAssignments == null) { //Something odd happened try again later return; @@ -189,26 +182,8 @@ public synchronized void run() { } } - protected Map> getAssignmentsSnapshot(IStormClusterState stormClusterState, List topoIds, - Map> localAssignmentVersion, Runnable callback) throws Exception { - Map> updateAssignmentVersion = new HashMap<>(); - for (String topoId : topoIds) { - Integer recordedVersion = -1; - Integer version = stormClusterState.assignmentVersion(topoId, callback); - VersionedData locAssignment = localAssignmentVersion.get(topoId); - if (locAssignment != null) { - recordedVersion = locAssignment.getVersion(); - } - if (version == null) { - // ignore - } else if (version.equals(recordedVersion)) { - updateAssignmentVersion.put(topoId, locAssignment); - } else { - VersionedData assignmentVersion = stormClusterState.assignmentInfoWithVersion(topoId, callback); - updateAssignmentVersion.put(topoId, assignmentVersion); - } - } - return updateAssignmentVersion; + protected Map getAssignmentsSnapshot(IStormClusterState stormClusterState) throws Exception { + return stormClusterState.assignmentsInfo(); } protected Map> getProfileActions(IStormClusterState stormClusterState, List stormIds) throws Exception { @@ -220,12 +195,12 @@ protected Map> getProfileActions(IStormClusterState return ret; } - protected Map readAssignments(Map> assignmentsSnapshot) { + protected Map readAssignments(Map assignmentsSnapshot) { try { Map portLA = new HashMap<>(); - for (Map.Entry> assignEntry : assignmentsSnapshot.entrySet()) { + for (Map.Entry assignEntry : assignmentsSnapshot.entrySet()) { String topoId = assignEntry.getKey(); - Assignment assignment = assignEntry.getValue().getData(); + Assignment assignment = assignEntry.getValue(); Map portTasks = readMyExecutors(topoId, assignmentId, assignment); @@ -293,14 +268,15 @@ protected Map readMyExecutors(String topoId, String as } if (hasShared) { localAssignment.set_total_node_shared(amountShared); - } + } if (assignment.is_set_owner()) { localAssignment.set_owner(assignment.get_owner()); } portTasks.put(port.intValue(), localAssignment); } List executorInfoList = localAssignment.get_executors(); - executorInfoList.add(new ExecutorInfo(entry.getKey().get(0).intValue(), entry.getKey().get(entry.getKey().size() - 1).intValue())); + executorInfoList.add(new ExecutorInfo(entry.getKey().get(0).intValue(), + entry.getKey().get(entry.getKey().size() - 1).intValue())); } } } diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainer.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainer.java index 44008b23692..2559d7a2a2b 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainer.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainer.java @@ -34,17 +34,20 @@ public class RunAsUserContainer extends BasicContainer { private static final Logger LOG = LoggerFactory.getLogger(RunAsUserContainer.class); - public RunAsUserContainer(Container.ContainerType type, Map conf, String supervisorId, int port, - LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, LocalState localState, + public RunAsUserContainer(Container.ContainerType type, Map conf, String supervisorId, + int supervisorPort, int port, LocalAssignment assignment, + ResourceIsolationInterface resourceIsolationManager, LocalState localState, String workerId) throws IOException { - this(type, conf, supervisorId, port, assignment, resourceIsolationManager, localState, workerId, null, null, null); + this(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, localState, workerId, + null, null, null); } - RunAsUserContainer(Container.ContainerType type, Map conf, String supervisorId, int port, - LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, LocalState localState, - String workerId, Map topoConf, AdvancedFSOps ops, String profileCmd) throws IOException { - super(type, conf, supervisorId, port, assignment, resourceIsolationManager, localState, workerId, topoConf, ops, - profileCmd); + RunAsUserContainer(Container.ContainerType type, Map conf, String supervisorId, int supervisorPort, + int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, + LocalState localState, String workerId, Map topoConf, AdvancedFSOps ops, + String profileCmd) throws IOException { + super(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, localState, + workerId, topoConf, ops, profileCmd); if (Utils.isOnWindows()) { throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet"); } diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java index c8bee270085..e6439db0af3 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java @@ -28,18 +28,21 @@ public class RunAsUserContainerLauncher extends ContainerLauncher { private final Map _conf; private final String _supervisorId; + private final int _supervisorPort; protected final ResourceIsolationInterface _resourceIsolationManager; - public RunAsUserContainerLauncher(Map conf, String supervisorId, ResourceIsolationInterface resourceIsolationManager) throws IOException { + public RunAsUserContainerLauncher(Map conf, String supervisorId, int supervisorPort, + ResourceIsolationInterface resourceIsolationManager) throws IOException { _conf = conf; _supervisorId = supervisorId; + _supervisorPort = supervisorPort; _resourceIsolationManager = resourceIsolationManager; } @Override public Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException { - Container container = new RunAsUserContainer(ContainerType.LAUNCH, _conf, _supervisorId, port, assignment, - _resourceIsolationManager, state, null, null, null, null); + Container container = new RunAsUserContainer(ContainerType.LAUNCH, _conf, _supervisorId, _supervisorPort, port, + assignment, _resourceIsolationManager, state, null, null, null, null); container.setup(); container.launch(); return container; @@ -47,13 +50,13 @@ public Container launchContainer(int port, LocalAssignment assignment, LocalStat @Override public Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException { - return new RunAsUserContainer(ContainerType.RECOVER_FULL, _conf, _supervisorId, port, assignment, - _resourceIsolationManager, state, null, null, null, null); + return new RunAsUserContainer(ContainerType.RECOVER_FULL, _conf, _supervisorId, _supervisorPort, port, + assignment, _resourceIsolationManager, state, null, null, null, null); } @Override public Killable recoverContainer(String workerId, LocalState localState) throws IOException { - return new RunAsUserContainer(ContainerType.RECOVER_PARTIAL, _conf, _supervisorId, -1, null, + return new RunAsUserContainer(ContainerType.RECOVER_PARTIAL, _conf, _supervisorId, _supervisorPort, -1, null, _resourceIsolationManager, localState, workerId, null, null, null); } diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java index 6c90c0ecf83..4e5b86197f1 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java @@ -20,6 +20,8 @@ import java.io.File; import java.io.IOException; +import java.net.BindException; +import java.net.ServerSocket; import java.net.UnknownHostException; import java.util.Collection; import java.util.HashMap; @@ -29,7 +31,10 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.io.FileUtils; +import org.apache.storm.Config; import org.apache.storm.DaemonConfig; import org.apache.storm.StormTimer; import org.apache.storm.cluster.ClusterStateContext; @@ -37,22 +42,41 @@ import org.apache.storm.cluster.DaemonType; import org.apache.storm.cluster.IStormClusterState; import org.apache.storm.daemon.DaemonCommon; +import org.apache.storm.daemon.StormCommon; +import org.apache.storm.daemon.supervisor.timer.ReportWorkerHeartbeats; import org.apache.storm.daemon.supervisor.timer.SupervisorHealthCheck; import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat; +import org.apache.storm.daemon.supervisor.timer.SynchronizeAssignments; import org.apache.storm.event.EventManager; import org.apache.storm.event.EventManagerImp; +import org.apache.storm.generated.Assignment; +import org.apache.storm.generated.AuthorizationException; import org.apache.storm.generated.LocalAssignment; +import org.apache.storm.generated.Nimbus; +import org.apache.storm.generated.NotAliveException; +import org.apache.storm.generated.SupervisorAssignments; +import org.apache.storm.generated.SupervisorWorkerHeartbeat; import org.apache.storm.localizer.AsyncLocalizer; +import org.apache.storm.logging.ThriftAccessLogger; import org.apache.storm.messaging.IContext; import org.apache.storm.metric.StormMetricsRegistry; import org.apache.storm.scheduler.ISupervisor; +import org.apache.storm.security.auth.IAuthorizer; +import org.apache.storm.security.auth.ReqContext; +import org.apache.storm.security.auth.ThriftConnectionType; +import org.apache.storm.security.auth.ThriftServer; import org.apache.storm.utils.ConfigUtils; +import org.apache.storm.utils.ServerConfigUtils; +import org.apache.storm.utils.ObjectReader; import org.apache.storm.utils.LocalState; import org.apache.storm.utils.ObjectReader; import org.apache.storm.utils.ServerConfigUtils; import org.apache.storm.utils.Time; import org.apache.storm.utils.Utils; import org.apache.storm.utils.VersionInfo; +import org.apache.thrift.TException; +import org.apache.thrift.TProcessor; +import org.apache.zookeeper.data.ACL; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,6 +84,7 @@ public class Supervisor implements DaemonCommon, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(Supervisor.class); private final Map conf; private final IContext sharedContext; + private final IAuthorizer authorizationHandler; private volatile boolean active; private final ISupervisor iSupervisor; private final Utils.UptimeComputer upTime; @@ -68,10 +93,12 @@ public class Supervisor implements DaemonCommon, AutoCloseable { private final LocalState localState; private final String supervisorId; private final String assignmentId; + private final int supervisorPort; private final String hostName; // used for reporting used ports when heartbeating private final AtomicReference> currAssignment; private final StormTimer heartbeatTimer; + private final StormTimer workerHeartbeatTimer; private final StormTimer eventTimer; //Right now this is only used for sending metrics to nimbus, // but we may want to combine it with the heartbeatTimer at some point @@ -80,12 +107,24 @@ public class Supervisor implements DaemonCommon, AutoCloseable { private final AsyncLocalizer asyncLocalizer; private EventManager eventManager; private ReadClusterState readState; + private ThriftServer thriftServer; + //used for local cluster heartbeating + private Nimbus.Iface localNimbus; - private Supervisor(ISupervisor iSupervisor) throws IOException { + private Supervisor(ISupervisor iSupervisor) + throws IOException, IllegalAccessException, InstantiationException, ClassNotFoundException { this(Utils.readStormConfig(), null, iSupervisor); } - - public Supervisor(Map conf, IContext sharedContext, ISupervisor iSupervisor) throws IOException { + + /** + * Constructor for supervisor daemon. + * @param conf config + * @param sharedContext {@link IContext} + * @param iSupervisor {@link ISupervisor} + * @throws IOException + */ + public Supervisor(Map conf, IContext sharedContext, ISupervisor iSupervisor) + throws IOException, IllegalAccessException, ClassNotFoundException, InstantiationException{ this.conf = conf; this.iSupervisor = iSupervisor; this.active = true; @@ -93,11 +132,18 @@ public Supervisor(Map conf, IContext sharedContext, ISupervisor this.stormVersion = VersionInfo.getVersion(); this.sharedContext = sharedContext; this.heartbeatExecutor = Executors.newFixedThreadPool(1); + this.authorizationHandler = StormCommon.mkAuthorizationHandler( + (String) conf.get(DaemonConfig.SUPERVISOR_AUTHORIZER), conf); + if (authorizationHandler == null && conf.get(DaemonConfig.NIMBUS_AUTHORIZER) != null) { + throw new IllegalStateException("It looks like authorization is turned on for nimbus but not for the " + + "supervisor...."); + } iSupervisor.prepare(conf, ServerConfigUtils.supervisorIsupervisorDir(conf)); try { - this.stormClusterState = ClusterUtils.mkStormClusterState(conf, new ClusterStateContext(DaemonType.SUPERVISOR, conf)); + this.stormClusterState = ClusterUtils.mkStormClusterState(conf, + new ClusterStateContext(DaemonType.SUPERVISOR, conf)); } catch (Exception e) { LOG.error("supervisor can't create stormClusterState"); throw Utils.wrapInRuntime(e); @@ -113,6 +159,7 @@ public Supervisor(Map conf, IContext sharedContext, ISupervisor } this.supervisorId = iSupervisor.getSupervisorId(); this.assignmentId = iSupervisor.getAssignmentId(); + this.supervisorPort = ObjectReader.getInt(conf.get(Config.SUPERVISOR_THRIFT_PORT)); try { this.hostName = Utils.hostname(); @@ -122,6 +169,8 @@ public Supervisor(Map conf, IContext sharedContext, ISupervisor this.heartbeatTimer = new StormTimer(null, new DefaultUncaughtExceptionHandler()); + this.workerHeartbeatTimer = new StormTimer(null, new DefaultUncaughtExceptionHandler()); + this.eventTimer = new StormTimer(null, new DefaultUncaughtExceptionHandler()); } @@ -160,6 +209,10 @@ public IStormClusterState getStormClusterState() { return stormClusterState; } + public ReadClusterState getReadClusterState() { + return readState; + } + LocalState getLocalState() { return localState; } @@ -168,6 +221,10 @@ public String getAssignmentId() { return assignmentId; } + public int getThriftServerPort() { + return supervisorPort; + } + public String getHostName() { return hostName; } @@ -183,6 +240,18 @@ AsyncLocalizer getAsyncLocalizer() { EventManager getEventManger() { return eventManager; } + + Supervisor getSupervisor() { + return this; + } + + public void setLocalNimbus(Nimbus.Iface nimbus) { + this.localNimbus = nimbus; + } + + public Nimbus.Iface getLocalNimbus() { + return this.localNimbus; + } /** * Launch the supervisor. @@ -206,11 +275,16 @@ public void launch() throws Exception { if ((Boolean) conf.get(DaemonConfig.SUPERVISOR_ENABLE)) { // This isn't strictly necessary, but it doesn't hurt and ensures that the machine stays up // to date even if callbacks don't all work exactly right - eventTimer.scheduleRecurring(0, 10, new EventManagerPushCallback(readState, eventManager)); + eventTimer.scheduleRecurring(0, 10, + new EventManagerPushCallback(new SynchronizeAssignments(this, null, readState), eventManager)); // supervisor health check eventTimer.scheduleRecurring(30, 30, new SupervisorHealthCheck(this)); } + + ReportWorkerHeartbeats reportWorkerHeartbeats = new ReportWorkerHeartbeats(conf, this); + Integer workerHeartbeatFrequency = ObjectReader.getInt(conf.get(Config.WORKER_HEARTBEAT_FREQUENCY_SECS)); + workerHeartbeatTimer.scheduleRecurring(0, workerHeartbeatFrequency, reportWorkerHeartbeats); LOG.info("Starting supervisor with id {} at host {}.", getId(), getHostName()); } @@ -225,6 +299,8 @@ public void launchDaemon() { throw new IllegalArgumentException("Cannot start server in local mode!"); } launch(); + //must invoke after launch cause some services must be initialized + launchSupervisorThriftServer(conf); Utils.addShutdownHookWithForceKillIn1Sec(this::close); registerWorkerNumGauge("supervisor:num-slots-used-gauge", conf); StormMetricsRegistry.startMetricsReporters(conf); @@ -234,6 +310,125 @@ public void launchDaemon() { } } + @VisibleForTesting + public void checkAuthorization(String operation) throws AuthorizationException { + checkAuthorization(null, null, operation, null); + } + + @VisibleForTesting + public void checkAuthorization(String topoName, Map topoConf, String operation) + throws AuthorizationException { + checkAuthorization(topoName, topoConf, operation, null); + } + + @VisibleForTesting + public void checkAuthorization(String topoName, Map topoConf, String operation, ReqContext context) + throws AuthorizationException { + IAuthorizer aclHandler = authorizationHandler; + if (context == null) { + context = ReqContext.context(); + } + Map checkConf = new HashMap<>(); + if (topoConf != null) { + checkConf.putAll(topoConf); + } else if (topoName != null) { + checkConf.put(Config.TOPOLOGY_NAME, topoName); + } + + if (context.isImpersonating()) { + LOG.warn("principal: {} is trying to impersonate principal: {}", context.realPrincipal(), + context.principal()); + throw new AuthorizationException("Supervisor does not support impersonation"); + } + + if (aclHandler != null) { + if (!aclHandler.permit(context, operation, checkConf)) { + ThriftAccessLogger.logAccess(context.requestID(), context.remoteAddress(), context.principal(), + operation, topoName, "access-denied"); + throw new AuthorizationException( operation + (topoName != null ? " on topology " + topoName : "") + + " is not authorized"); + } else { + ThriftAccessLogger.logAccess(context.requestID(), context.remoteAddress(), context.principal(), + operation, topoName, "access-granted"); + } + } + } + + private void launchSupervisorThriftServer(Map conf) throws IOException { + // validate port + int port = getThriftServerPort(); + try { + ServerSocket socket = new ServerSocket(port); + socket.close(); + } catch (BindException e) { + LOG.error("{} is not available. Check if another process is already listening on {}", port, port); + throw new RuntimeException(e); + } + + TProcessor processor = new org.apache.storm.generated.Supervisor.Processor( + new org.apache.storm.generated.Supervisor.Iface() { + @Override + public void sendSupervisorAssignments(SupervisorAssignments assignments) + throws AuthorizationException, TException { + checkAuthorization("sendSupervisorAssignments"); + LOG.info("Got an assignments from master, will start to sync with assignments: {}", assignments); + SynchronizeAssignments syn = new SynchronizeAssignments(getSupervisor(), assignments, + getReadClusterState()); + getEventManger().add(syn); + } + + @Override + public Assignment getLocalAssignmentForStorm(String id) + throws NotAliveException, AuthorizationException, TException { + Map topoConf = null; + try { + topoConf = ConfigUtils.readSupervisorStormConf(conf, id); + } catch (IOException e) { + LOG.warn("Topology config is not localized yet..."); + } + checkAuthorization(id, topoConf, "getLocalAssignmentForStorm"); + Assignment assignment = getStormClusterState().assignmentInfo(id, null); + if (null == assignment) { + throw new NotAliveException("No local assignment assigned for storm: " + + id + + " for node: " + + getHostName()); + } + return assignment; + } + + @Override + public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heartbeat) + throws AuthorizationException, NotAliveException, TException { + // do nothing except validate heartbeat for now. + String id = heartbeat.get_storm_id(); + Map topoConf = null; + try { + topoConf = ConfigUtils.readSupervisorStormConf(conf, id); + } catch (IOException e) { + LOG.warn("Topology config is not localized yet..."); + throw new NotAliveException(id + " does not appear to be alive, you should probably exit"); + } + checkAuthorization(id, topoConf, "sendSupervisorWorkerHeartbeat"); + } + }); + this.thriftServer = new ThriftServer(conf, processor, ThriftConnectionType.SUPERVISOR); + this.thriftServer.serve(); + } + + /** + * Used for local cluster assignments distribution. + * @param assignments {@link SupervisorAssignments} + */ + public void sendSupervisorAssignments(SupervisorAssignments assignments) { + //for local test + if (Time.isSimulating() && !(Boolean) conf.get(DaemonConfig.SUPERVISOR_ENABLE)) { + return; + } + SynchronizeAssignments syn = new SynchronizeAssignments(this, assignments, readState); + eventManager.add(syn); + } + private void registerWorkerNumGauge(String name, final Map conf) { StormMetricsRegistry.registerGauge(name, new Callable() { @Override @@ -250,6 +445,7 @@ public void close() { LOG.info("Shutting down supervisor {}", getId()); this.active = false; heartbeatTimer.close(); + workerHeartbeatTimer.close(); eventTimer.close(); if (eventManager != null) { eventManager.close(); @@ -259,6 +455,9 @@ public void close() { } asyncLocalizer.close(); getStormClusterState().disconnect(); + if(thriftServer != null) { + this.thriftServer.stop(); + } } catch (Exception e) { LOG.error("Error Shutting down", e); } @@ -307,7 +506,8 @@ public void shutdownAllWorkers(UniFunc onWarnTimeout, UniFunc onErro readState.shutdownAllWorkers(onWarnTimeout, onErrorTimeout); } else { try { - ContainerLauncher launcher = ContainerLauncher.make(getConf(), getId(), getSharedContext()); + ContainerLauncher launcher = ContainerLauncher.make(getConf(), getId(), getThriftServerPort(), + getSharedContext()); killWorkers(SupervisorUtils.supervisorWorkerIds(conf), launcher); } catch (Exception e) { throw Utils.wrapInRuntime(e); @@ -321,18 +521,18 @@ public boolean isWaiting() { return true; } - if (heartbeatTimer.isTimerWaiting() && eventTimer.isTimerWaiting() && eventManager.waiting()) { - return true; - } - return false; + return heartbeatTimer.isTimerWaiting() + && workerHeartbeatTimer.isTimerWaiting() + && eventTimer.isTimerWaiting() + && eventManager.waiting(); } /** - * supervisor daemon enter entrance + * supervisor daemon enter entrance. * * @param args */ - public static void main(String[] args) throws IOException { + public static void main(String[] args) throws Exception { Utils.setupDefaultUncaughtExceptionHandler(); @SuppressWarnings("resource") Supervisor instance = new Supervisor(new StandaloneSupervisor()); diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java index aba94591678..90e54516096 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SupervisorUtils.java @@ -15,8 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.daemon.supervisor; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.storm.Config; import org.apache.storm.generated.LSWorkerHeartbeat; import org.apache.storm.localizer.LocalResource; @@ -30,13 +38,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - public class SupervisorUtils { private static final Logger LOG = LoggerFactory.getLogger(SupervisorUtils.class); @@ -75,7 +76,7 @@ public static boolean shouldUncompressBlob(Map blobInfo) { /** * Given the blob information returns the value of the workerRestart field, handling it either being a string or a boolean value, or - * if it's not specified then returns false + * if it's not specified then returns false. * * @param blobInfo the info for the blob. * @return true if the blob needs a worker restart by way of the callback else false. diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/ReportWorkerHeartbeats.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/ReportWorkerHeartbeats.java new file mode 100644 index 00000000000..c01819b0431 --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/ReportWorkerHeartbeats.java @@ -0,0 +1,112 @@ +/** + * 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.daemon.supervisor.timer; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.storm.daemon.supervisor.Supervisor; +import org.apache.storm.daemon.supervisor.SupervisorUtils; +import org.apache.storm.generated.LSWorkerHeartbeat; +import org.apache.storm.generated.SupervisorWorkerHeartbeat; +import org.apache.storm.generated.SupervisorWorkerHeartbeats; +import org.apache.storm.utils.ConfigUtils; +import org.apache.storm.utils.NimbusClient; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Runnable reporting local worker reported heartbeats to master, supervisor should take care the of the heartbeats + * integrity for the master heartbeats recovery, a non-null node id means that the heartbeats are full, + * and master can go on to check and wait others nodes when doing a heartbeats recovery. + */ +public class ReportWorkerHeartbeats implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(ReportWorkerHeartbeats.class); + + private Supervisor supervisor; + private Map conf; + + public ReportWorkerHeartbeats(Map conf, Supervisor supervisor) { + this.conf = conf; + this.supervisor = supervisor; + } + + @Override + public void run() { + SupervisorWorkerHeartbeats supervisorWorkerHeartbeats = getAndResetWorkerHeartbeats(); + reportWorkerHeartbeats(supervisorWorkerHeartbeats); + } + + private SupervisorWorkerHeartbeats getAndResetWorkerHeartbeats() { + Map localHeartbeats; + try { + localHeartbeats = SupervisorUtils.readWorkerHeartbeats(this.conf); + return getSupervisorWorkerHeartbeatsFromLocal(localHeartbeats); + } catch (Exception e) { + LOG.error("Read local worker heartbeats error, skipping heartbeats for this round, msg:{}", e.getMessage()); + return null; + } + } + + private SupervisorWorkerHeartbeats getSupervisorWorkerHeartbeatsFromLocal(Map localHeartbeats) { + SupervisorWorkerHeartbeats supervisorWorkerHeartbeats = new SupervisorWorkerHeartbeats(); + + List heartbeatList = new ArrayList<>(); + + for (LSWorkerHeartbeat lsWorkerHeartbeat : localHeartbeats.values()) { + // local worker heartbeat can be null cause some error/exception + if (null == lsWorkerHeartbeat) { + continue; + } + + SupervisorWorkerHeartbeat supervisorWorkerHeartbeat = new SupervisorWorkerHeartbeat(); + supervisorWorkerHeartbeat.set_storm_id(lsWorkerHeartbeat.get_topology_id()); + supervisorWorkerHeartbeat.set_executors(lsWorkerHeartbeat.get_executors()); + supervisorWorkerHeartbeat.set_time_secs(lsWorkerHeartbeat.get_time_secs()); + + heartbeatList.add(supervisorWorkerHeartbeat); + } + supervisorWorkerHeartbeats.set_supervisor_id(this.supervisor.getId()); + supervisorWorkerHeartbeats.set_worker_heartbeats(heartbeatList); + return supervisorWorkerHeartbeats; + } + + private void reportWorkerHeartbeats(SupervisorWorkerHeartbeats supervisorWorkerHeartbeats) { + if (supervisorWorkerHeartbeats == null) { + // error/exception thrown, just skip + return; + } + // if it is local mode, just get the local nimbus instance and set the heartbeats + if (ConfigUtils.isLocalMode(conf)) { + try { + this.supervisor.getLocalNimbus().sendSupervisorWorkerHeartbeats(supervisorWorkerHeartbeats); + } catch (TException tex) { + LOG.error("Send local supervisor heartbeats error", tex); + } + } else { + try (NimbusClient master = NimbusClient.getConfiguredClient(conf)) { + master.getClient().sendSupervisorWorkerHeartbeats(supervisorWorkerHeartbeats); + } catch (Exception t) { + LOG.error("Send worker heartbeats to master exception", t); + } + } + } +} diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SupervisorHeartbeat.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SupervisorHeartbeat.java index 2be241a48c9..14ecf942bc4 100644 --- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SupervisorHeartbeat.java +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SupervisorHeartbeat.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.storm.daemon.supervisor.timer; import java.util.ArrayList; @@ -33,10 +34,10 @@ public class SupervisorHeartbeat implements Runnable { - private final IStormClusterState stormClusterState; - private final String supervisorId; - private final Map conf; - private final Supervisor supervisor; + private final IStormClusterState stormClusterState; + private final String supervisorId; + private final Map conf; + private final Supervisor supervisor; public SupervisorHeartbeat(Map conf, Supervisor supervisor) { this.stormClusterState = supervisor.getStormClusterState(); @@ -50,17 +51,19 @@ private SupervisorInfo buildSupervisorInfo(Map conf, Supervisor supervisorInfo.set_time_secs(Time.currentTimeSecs()); supervisorInfo.set_hostname(supervisor.getHostName()); supervisorInfo.set_assignment_id(supervisor.getAssignmentId()); + supervisorInfo.set_server_port(supervisor.getThriftServerPort()); List usedPorts = new ArrayList<>(); usedPorts.addAll(supervisor.getCurrAssignment().get().keySet()); supervisorInfo.set_used_ports(usedPorts); List metaDatas = (List)supervisor.getiSupervisor().getMetadata(); List portList = new ArrayList<>(); - if (metaDatas != null){ - for (Object data : metaDatas){ + if (metaDatas != null) { + for (Object data : metaDatas) { Integer port = ObjectReader.getInt(data); - if (port != null) + if (port != null) { portList.add(port.longValue()); + } } } diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SynchronizeAssignments.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SynchronizeAssignments.java new file mode 100644 index 00000000000..ba8d133917a --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SynchronizeAssignments.java @@ -0,0 +1,133 @@ +/** + * 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.daemon.supervisor.timer; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.storm.cluster.IStormClusterState; +import org.apache.storm.daemon.supervisor.ReadClusterState; +import org.apache.storm.daemon.supervisor.Supervisor; +import org.apache.storm.generated.Assignment; +import org.apache.storm.generated.SupervisorAssignments; +import org.apache.storm.utils.ConfigUtils; +import org.apache.storm.utils.NimbusClient; +import org.apache.storm.utils.Time; +import org.apache.storm.utils.Utils; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A runnable which will synchronize assignments to node local and then worker processes. + */ +public class SynchronizeAssignments implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(SynchronizeAssignments.class); + + private Supervisor supervisor; + private SupervisorAssignments assignments; + private ReadClusterState readClusterState; + + /** + * Constructor. + * @param supervisor {@link Supervisor} + * @param assignments {@link SupervisorAssignments} + * @param readClusterState {@link ReadClusterState} + */ + public SynchronizeAssignments(Supervisor supervisor, SupervisorAssignments assignments, ReadClusterState readClusterState) { + this.supervisor = supervisor; + this.assignments = assignments; + this.readClusterState = readClusterState; + } + + @Override + public void run() { + // first sync assignments to local, then sync processes. + if (null == assignments) { + getAssignmentsFromMaster(this.supervisor.getConf(), this.supervisor.getStormClusterState(), this.supervisor.getAssignmentId()); + } else { + assignedAssignmentsToLocal(this.supervisor.getStormClusterState(), assignments); + } + this.readClusterState.run(); + } + + /** + * Used by {@link Supervisor} to fetch assignments when start up. + * @param supervisor {@link Supervisor} + */ + public void getAssignmentsFromMasterUntilSuccess(Supervisor supervisor) { + boolean success = false; + while (!success) { + try (NimbusClient master = NimbusClient.getConfiguredClient(supervisor.getConf())) { + SupervisorAssignments assignments = master.getClient().getSupervisorAssignments(supervisor.getAssignmentId()); + assignedAssignmentsToLocal(supervisor.getStormClusterState(), assignments); + success = true; + } catch (Exception t) { + // just ignore the exception + } + if (!success) { + LOG.info("Waiting for a success sync of assignments from master..."); + try { + Time.sleep(5000L); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + } + + } + + /** + * Used by {@link Supervisor} to fetch assignments when start up. + * @param conf config + * @param clusterState {@link IStormClusterState} + * @param node id of node + */ + public void getAssignmentsFromMaster(Map conf, IStormClusterState clusterState, String node) { + if (ConfigUtils.isLocalMode(conf)) { + try { + SupervisorAssignments assignments = this.supervisor.getLocalNimbus().getSupervisorAssignments(node); + assignedAssignmentsToLocal(clusterState, assignments); + } catch (TException e) { + LOG.error("Get assignments from local master exception", e); + } + } else { + try (NimbusClient master = NimbusClient.getConfiguredClient(conf)) { + SupervisorAssignments assignments = master.getClient().getSupervisorAssignments(node); + LOG.debug("Sync an assignments from master, will start to sync with assignments: {}", assignments); + assignedAssignmentsToLocal(clusterState, assignments); + } catch (Exception t) { + LOG.error("Get assignments from master exception", t); + } + } + } + + private static void assignedAssignmentsToLocal(IStormClusterState clusterState, SupervisorAssignments assignments) { + if (null == assignments) { + //unknown error, just skip + return; + } + Map serAssignments = new HashMap<>(); + for (Map.Entry entry : assignments.get_storm_assignment().entrySet()) { + serAssignments.put(entry.getKey(), Utils.serialize(entry.getValue())); + } + clusterState.syncRemoteAssignments(serAssignments); + } +} diff --git a/storm-server/src/main/java/org/apache/storm/nimbus/AssignmentDistributionService.java b/storm-server/src/main/java/org/apache/storm/nimbus/AssignmentDistributionService.java new file mode 100644 index 00000000000..558e570bcfb --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/nimbus/AssignmentDistributionService.java @@ -0,0 +1,312 @@ +/** + * 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.nimbus; + +import java.io.Closeable; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.apache.storm.DaemonConfig; +import org.apache.storm.daemon.supervisor.Supervisor; +import org.apache.storm.generated.SupervisorAssignments; +import org.apache.storm.scheduler.SupervisorDetails; +import org.apache.storm.utils.ConfigUtils; +import org.apache.storm.utils.ObjectReader; +import org.apache.storm.utils.SupervisorClient; +import org.apache.storm.utils.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A service for distributing master assignments to supervisors, this service makes the assignments notification + * asynchronous. + * + *

We support multiple working threads to distribute assignment, every thread has a queue buffer. + * + *

Master will shuffle its node request to the queues, if the target queue is full, we just discard the request, + * let the supervisors sync instead. + * + *

Caution: this class is not thread safe. + * + *

{@code
+ * Working mode
+ *                      +--------+         +-----------------+
+ *                      | queue1 |   ==>   | Working thread1 |
+ * +--------+ shuffle   +--------+         +-----------------+
+ * | Master |   ==>
+ * +--------+           +--------+         +-----------------+
+ *                      | queue2 |   ==>   | Working thread2 |
+ *                      +--------+         +-----------------+
+ * }
+ * 
+ */ +public class AssignmentDistributionService implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(AssignmentDistributionService.class); + private ExecutorService service; + /** + * Flag to indicate if the service is active. + */ + private volatile boolean active = false; + + private Random random; + /** + * Working threads num. + */ + private int threadsNum = 0; + /** + * Working thread queue size. + */ + private int queueSize = 0; + + /** + * Assignments request queue. + */ + private volatile Map> assignmentsQueue; + + /** + * local supervisors for local cluster assignments distribution. + */ + private Map localSupervisors; + + private Map conf; + + private boolean isLocalMode = false; // boolean cache for local mode decision + + /** + * Function for initialization. + * + * @param conf config + */ + public void prepare(Map conf) { + this.conf = conf; + this.random = new Random(47); + + this.threadsNum = ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_ASSIGNMENTS_SERVICE_THREADS), 10); + this.queueSize = ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_ASSIGNMENTS_SERVICE_THREAD_QUEUE_SIZE), 100); + + this.assignmentsQueue = new HashMap<>(); + for (int i = 0; i < threadsNum; i++) { + this.assignmentsQueue.put(i, new LinkedBlockingQueue(queueSize)); + } + //start the thread pool + this.service = Executors.newFixedThreadPool(threadsNum); + this.active = true; + //start the threads + for (int i = 0; i < threadsNum; i++) { + this.service.submit(new DistributeTask(this, i)); + } + // for local cluster + localSupervisors = new HashMap<>(); + if (ConfigUtils.isLocalMode(conf)) { + isLocalMode = true; + } + } + + @Override + public void close() throws IOException { + this.active = false; + this.service.shutdownNow(); + try { + this.service.awaitTermination(1L, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.error("Failed to close assignments distribute service"); + } + this.assignmentsQueue = null; + } + + /** + * Add an assignments for a node/supervisor for distribution. + * @param node node id of supervisor. + * @param host host name for the node. + * @param serverPort node thrift server port. + * @param assignments the {@link org.apache.storm.generated.SupervisorAssignments} + */ + public void addAssignmentsForNode(String node, String host, Integer serverPort, SupervisorAssignments assignments) { + try { + //For some reasons, we can not get supervisor port info, eg: supervisor shutdown, + //Just skip for this scheduling round. + if (serverPort == null) { + LOG.warn("Discard an assignment distribution for node {} because server port info is missing.", node); + return; + } + + boolean success = nextQueue().offer(NodeAssignments.getInstance(node, host, serverPort, assignments), 5L, TimeUnit.SECONDS); + if (!success) { + LOG.warn("Discard an assignment distribution for node {} because the target sub queue is full.", node); + } + + } catch (InterruptedException e) { + LOG.error("Add node assignments interrupted: {}", e.getMessage()); + throw new RuntimeException(e); + } + } + + static class NodeAssignments { + private String node; + private String host; + private Integer serverPort; + private SupervisorAssignments assignments; + + private NodeAssignments(String node, String host, Integer serverPort, SupervisorAssignments assignments) { + this.node = node; + this.host = host; + this.serverPort = serverPort; + this.assignments = assignments; + } + + public static NodeAssignments getInstance(String node, String host, Integer serverPort, + SupervisorAssignments assignments) { + return new NodeAssignments(node, host, serverPort, assignments); + } + + //supervisor assignment id/supervisor id + public String getNode() { + return this.node; + } + + public String getHost() { + return host; + } + + public Integer getServerPort() { + return serverPort; + } + + public SupervisorAssignments getAssignments() { + return this.assignments; + } + + } + + /** + * Task to distribute assignments. + */ + static class DistributeTask implements Runnable { + private AssignmentDistributionService service; + private Integer queueIndex; + + DistributeTask(AssignmentDistributionService service, Integer index) { + this.service = service; + this.queueIndex = index; + } + + @Override + public void run() { + while (service.isActive()) { + try { + NodeAssignments nodeAssignments = this.service.nextAssignments(queueIndex); + sendAssignmentsToNode(nodeAssignments); + } catch (InterruptedException e) { + if (service.isActive()) { + LOG.error("Get an unexpected interrupt when distributing assignments to node, {}", e.getCause()); + } else { + // service is off now just interrupt it. + Thread.currentThread().interrupt(); + } + } + } + } + + private void sendAssignmentsToNode(NodeAssignments assignments) { + if (this.service.isLocalMode) { + //local node + Supervisor supervisor = this.service.localSupervisors.get(assignments.getNode()); + if (supervisor != null) { + supervisor.sendSupervisorAssignments(assignments.getAssignments()); + } else { + LOG.error("Can not find node {} for assignments distribution", assignments.getNode()); + throw new RuntimeException("null for node " + assignments.getNode() + " supervisor instance."); + } + } else { + // distributed mode + try (SupervisorClient client = SupervisorClient.getConfiguredClient(service.getConf(), + assignments.getHost(), assignments.getServerPort())){ + try { + client.getClient().sendSupervisorAssignments(assignments.getAssignments()); + } catch (Exception e) { + //just ignore the exception. + LOG.error("Exception when trying to send assignments to node {}: {}", assignments.getNode(), e.getMessage()); + } + } catch (Throwable e) { + //just ignore any error/exception. + LOG.error("Exception to create supervisor client for node {}: {}", assignments.getNode(), e.getMessage()); + } + + } + } + } + + public void addLocalSupervisor(Supervisor supervisor) { + this.localSupervisors.put(supervisor.getId(), supervisor); + } + + private Integer nextQueueId() { + return this.random.nextInt(threadsNum); + } + + private LinkedBlockingQueue nextQueue() { + return this.assignmentsQueue.get(nextQueueId()); + } + + private LinkedBlockingQueue getQueueById(Integer queueIndex) { + return this.assignmentsQueue.get(queueIndex); + } + + /** + * Get an assignments from the target queue with the specific index. + * @param queueIndex index of the queue + * @return an {@link NodeAssignments} + * @throws InterruptedException + */ + public NodeAssignments nextAssignments(Integer queueIndex) throws InterruptedException { + NodeAssignments target = null; + while (true) { + target = getQueueById(queueIndex).poll(); + if (target != null) { + return target; + } + Time.sleep(100L); + } + } + + public boolean isActive() { + return this.active; + } + + public Map getConf() { + return this.conf; + } + + /** + * Factory method for initialize a instance. + * @param conf config. + * @return an instance of {@link AssignmentDistributionService} + */ + public static AssignmentDistributionService getInstance(Map conf) { + AssignmentDistributionService service = new AssignmentDistributionService(); + service.prepare(conf); + return service; + } +} diff --git a/storm-server/src/main/java/org/apache/storm/nimbus/ILeaderElector.java b/storm-server/src/main/java/org/apache/storm/nimbus/ILeaderElector.java index c4a0f64cd5c..e362fbc8664 100644 --- a/storm-server/src/main/java/org/apache/storm/nimbus/ILeaderElector.java +++ b/storm-server/src/main/java/org/apache/storm/nimbus/ILeaderElector.java @@ -47,19 +47,19 @@ public interface ILeaderElector extends Closeable { void removeFromLeaderLockQueue() throws Exception; /** - * + * Decide if the caller currently has the leader lock. * @return true if the caller currently has the leader lock. */ boolean isLeader() throws Exception; /** - * + * Get the current leader's address. * @return the current leader's address , may return null if no one has the lock. */ NimbusInfo getLeader(); /** - * + * Get list of current nimbus addresses. * @return list of current nimbus addresses, includes leader. */ List getAllNimbuses()throws Exception; diff --git a/storm-server/src/main/java/org/apache/storm/nimbus/IWorkerHeartbeatsRecoveryStrategy.java b/storm-server/src/main/java/org/apache/storm/nimbus/IWorkerHeartbeatsRecoveryStrategy.java new file mode 100644 index 00000000000..bde58dc49a9 --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/nimbus/IWorkerHeartbeatsRecoveryStrategy.java @@ -0,0 +1,48 @@ +/** + * 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.nimbus; + +import java.util.Map; +import java.util.Set; + +/** + * Interface for strategy to recover heartbeats when master gains leadership. + */ +public interface IWorkerHeartbeatsRecoveryStrategy { + + /** + * Function to prepare the strategy. + * @param conf config + */ + void prepare(Map conf); + + /** + * Function to decide if the heartbeats is ready. + * @param nodeIds all the node ids from current physical plan[assignments], read from {@code ClusterState} + * @return true if all node worker heartbeats reported + */ + boolean isReady(Set nodeIds); + + /** + * report the node id to this strategy to help to decide {@code isReady}. + * @param nodeId the node id from reported SupervisorWorkerHeartbeats + */ + void reportNodeId(String nodeId); +} + diff --git a/storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java b/storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java new file mode 100644 index 00000000000..59cd4621208 --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java @@ -0,0 +1,231 @@ +/** + * 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.nimbus; + +import com.google.common.base.Joiner; +import com.google.common.collect.Sets; + +import javax.security.auth.Subject; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.commons.io.IOUtils; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.recipes.leader.LeaderLatch; +import org.apache.storm.Config; +import org.apache.storm.blobstore.BlobStore; +import org.apache.storm.blobstore.InputStreamWithMeta; +import org.apache.storm.cluster.ClusterUtils; +import org.apache.storm.cluster.IStormClusterState; +import org.apache.storm.daemon.nimbus.TopoCache; +import org.apache.storm.generated.AuthorizationException; +import org.apache.storm.generated.KeyNotFoundException; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.security.auth.ReqContext; +import org.apache.storm.utils.Utils; +import org.apache.storm.zookeeper.ClientZookeeper; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.data.ACL; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A callback function when nimbus gains leadership. + */ +public class LeaderListenerCallback { + private static final Logger LOG = LoggerFactory.getLogger(LeaderListenerCallback.class); + + private final BlobStore blobStore; + private final TopoCache tc; + private final IStormClusterState clusterState; + + private final CuratorFramework zk; + private final LeaderLatch leaderLatch; + + private final Map conf; + private final List acls; + + private static final String STORM_JAR_SUFFIX = "-stormjar.jar"; + private static final String STORM_CODE_SUFFIX = "-stormcode.ser"; + private static final String STORM_CONF_SUFFIX = "-stormconf.ser"; + + /** + * Constructor for {@LeaderListenerCallback}. + * @param conf config + * @param zk zookeeper CuratorFramework client + * @param leaderLatch LeaderLatch + * @param blobStore BlobStore + * @param tc TopoCache + * @param clusterState IStormClusterState + * @param acls zookeeper acls + */ + public LeaderListenerCallback(Map conf, CuratorFramework zk, LeaderLatch leaderLatch, BlobStore blobStore, + TopoCache tc, IStormClusterState clusterState, List acls) { + this.blobStore = blobStore; + this.tc = tc; + this.clusterState = clusterState; + this.zk = zk; + this.leaderLatch = leaderLatch; + this.conf = conf; + this.acls = acls; + } + + /** + * Invoke when gains leadership. + */ + public void leaderCallBack() { + //set up nimbus-info to zk + setUpNimbusInfo(acls); + //sync zk assignments/id-info to local + LOG.info("Sync remote assignments and id-info to local"); + clusterState.syncRemoteAssignments(null); + clusterState.syncRemoteIds(null); + clusterState.setAssignmentsBackendSynchronized(); + + Set activeTopologyIds = new TreeSet<>(ClientZookeeper.getChildren(zk, + ClusterUtils.STORMS_SUBTREE, false)); + + Set activeTopologyBlobKeys = populateTopologyBlobKeys(activeTopologyIds); + Set activeTopologyCodeKeys = filterTopologyCodeKeys(activeTopologyBlobKeys); + Set allLocalBlobKeys = Sets.newHashSet(blobStore.listKeys()); + Set allLocalTopologyBlobKeys = filterTopologyBlobKeys(allLocalBlobKeys); + + // this finds all active topologies blob keys from all local topology blob keys + Sets.SetView diffTopology = Sets.difference(activeTopologyBlobKeys, allLocalTopologyBlobKeys); + LOG.info("active-topology-blobs [{}] local-topology-blobs [{}] diff-topology-blobs [{}]", + generateJoinedString(activeTopologyIds), generateJoinedString(allLocalTopologyBlobKeys), + generateJoinedString(diffTopology)); + + if (diffTopology.isEmpty()) { + Set activeTopologyDependencies = getTopologyDependencyKeys(activeTopologyCodeKeys); + + // this finds all dependency blob keys from active topologies from all local blob keys + Sets.SetView diffDependencies = Sets.difference(activeTopologyDependencies, allLocalBlobKeys); + LOG.info("active-topology-dependencies [{}] local-blobs [{}] diff-topology-dependencies [{}]", + generateJoinedString(activeTopologyDependencies), generateJoinedString(allLocalBlobKeys), + generateJoinedString(diffDependencies)); + + if (diffDependencies.isEmpty()) { + LOG.info("Accepting leadership, all active topologies and corresponding dependencies found locally."); + tc.clear(); + } else { + LOG.info("Code for all active topologies is available locally, but some dependencies are not found locally, " + + "giving up leadership."); + closeLatch(); + } + } else { + LOG.info("code for all active topologies not available locally, giving up leadership."); + closeLatch(); + } + } + + /** + * Invoke when lost leadership. + */ + public void notLeaderCallback() { + tc.clear(); + } + + private void setUpNimbusInfo(List acls) { + String leaderInfoPath = conf.get(Config.STORM_ZOOKEEPER_ROOT) + ClusterUtils.LEADERINFO_SUBTREE; + NimbusInfo nimbusInfo = NimbusInfo.fromConf(conf); + if (ClientZookeeper.existsNode(zk, leaderInfoPath, false)) { + ClientZookeeper.setData(zk, leaderInfoPath, Utils.javaSerialize(nimbusInfo)); + } else { + ClientZookeeper.createNode(zk, leaderInfoPath, Utils.javaSerialize(nimbusInfo), CreateMode.PERSISTENT, acls); + } + } + + private String generateJoinedString(Set activeTopologyIds) { + return Joiner.on(",").join(activeTopologyIds); + } + + private Set populateTopologyBlobKeys(Set activeTopologyIds) { + Set activeTopologyBlobKeys = new TreeSet<>(); + for (String activeTopologyId : activeTopologyIds) { + activeTopologyBlobKeys.add(activeTopologyId + STORM_JAR_SUFFIX); + activeTopologyBlobKeys.add(activeTopologyId + STORM_CODE_SUFFIX); + activeTopologyBlobKeys.add(activeTopologyId + STORM_CONF_SUFFIX); + } + return activeTopologyBlobKeys; + } + + private Set filterTopologyBlobKeys(Set blobKeys) { + Set topologyBlobKeys = new HashSet<>(); + for (String blobKey : blobKeys) { + if (blobKey.endsWith(STORM_JAR_SUFFIX) + || blobKey.endsWith(STORM_CODE_SUFFIX) + || blobKey.endsWith(STORM_CONF_SUFFIX)) { + topologyBlobKeys.add(blobKey); + } + } + return topologyBlobKeys; + } + + private Set filterTopologyCodeKeys(Set blobKeys) { + Set topologyCodeKeys = new HashSet<>(); + for (String blobKey : blobKeys) { + if (blobKey.endsWith(STORM_CODE_SUFFIX)) { + topologyCodeKeys.add(blobKey); + } + } + return topologyCodeKeys; + } + + private Set getTopologyDependencyKeys(Set activeTopologyCodeKeys) { + Set activeTopologyDependencies = new TreeSet<>(); + Subject subject = ReqContext.context().subject(); + + for (String activeTopologyCodeKey : activeTopologyCodeKeys) { + try { + InputStreamWithMeta blob = blobStore.getBlob(activeTopologyCodeKey, subject); + byte[] blobContent = IOUtils.readFully(blob, new Long(blob.getFileLength()).intValue()); + StormTopology stormCode = Utils.deserialize(blobContent, StormTopology.class); + if (stormCode.is_set_dependency_jars()) { + activeTopologyDependencies.addAll(stormCode.get_dependency_jars()); + } + if (stormCode.is_set_dependency_artifacts()) { + activeTopologyDependencies.addAll(stormCode.get_dependency_artifacts()); + } + } catch (AuthorizationException | KeyNotFoundException | IOException e) { + LOG.error("Exception occurs while reading blob for key: " + + activeTopologyCodeKey + + ", exception: " + + e, e); + throw new RuntimeException("Exception occurs while reading blob for key: " + + activeTopologyCodeKey + + ", exception: " + e, e); + } + } + return activeTopologyDependencies; + } + + private void closeLatch() { + try { + leaderLatch.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + +} diff --git a/storm-server/src/main/java/org/apache/storm/nimbus/NimbusHeartbeatsPressureTest.java b/storm-server/src/main/java/org/apache/storm/nimbus/NimbusHeartbeatsPressureTest.java new file mode 100644 index 00000000000..f725f5d8eb6 --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/nimbus/NimbusHeartbeatsPressureTest.java @@ -0,0 +1,241 @@ +/** + * 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.nimbus; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Random; + +import org.apache.storm.Config; +import org.apache.storm.generated.ExecutorInfo; +import org.apache.storm.generated.SupervisorWorkerHeartbeat; +import org.apache.storm.generated.SupervisorWorkerHeartbeats; +import org.apache.storm.utils.NimbusClient; +import org.apache.storm.utils.Utils; +import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransportException; + +/** + * Test for nimbus heartbeats max throughput, This is a client to collect the statistics. + */ +public class NimbusHeartbeatsPressureTest { + /** + * the args below can be configured. + */ + private static String NIMBUS_HOST = "localhost"; + private static int NIMBUS_PORT = 6627; + + private static int THREADS_NUM = 50; + private static int THREAD_SUBMIT_NUM = 1; + private static int MOCKED_STORM_NUM = 5000; + private static volatile boolean[] readyFlags = new boolean[THREADS_NUM]; + + static { + for (int i = 0; i < THREADS_NUM; i++) { + readyFlags[i] = false; + } + } + + private static Random rand = new Random(47); + private static List totalCostTimesBook = new ArrayList<>(); + + /** + * Initialize a fake config. + * @return conf + */ + private static Config initializedConfig() { + Config conf = new Config(); + conf.putAll(Utils.readDefaultConfig()); + ArrayList nimbusSeeds = new ArrayList<>(); + nimbusSeeds.add(NIMBUS_HOST); + + conf.put(Config.NIMBUS_SEEDS, nimbusSeeds); + conf.put(Config.NIMBUS_THRIFT_PORT, NIMBUS_PORT); + return conf; + } + + /** + * Test max throughput with the specific config args. + */ + public static void testMaxThroughput() { + ExecutorService service = Executors.newFixedThreadPool(THREADS_NUM); + + long submitStart = System.currentTimeMillis(); + for (int i = 0; i < THREADS_NUM; i++) { + service.submit(new HeartbeatSendTask(i, THREAD_SUBMIT_NUM)); + } + long submitEnd = System.currentTimeMillis(); + println(THREADS_NUM + " tasks, " + THREAD_SUBMIT_NUM * THREADS_NUM + " submit cost " + + (submitEnd - submitStart) / 1000D + "seconds"); + long totalStart = System.currentTimeMillis(); + while (!allTasksReady()) { + try { + + Thread.sleep(10L); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + long totalEnd = System.currentTimeMillis(); + println(THREADS_NUM + " tasks, " + THREAD_SUBMIT_NUM * THREADS_NUM + + " requests cost " + (totalEnd - totalStart) / 1000D + "seconds"); + printStatistics(totalCostTimesBook); + try { + service.shutdownNow(); + service.awaitTermination(3, TimeUnit.SECONDS); + } catch (InterruptedException e) { + e.printStackTrace(); + } + + } + + private static boolean allTasksReady() { + for (boolean ready : readyFlags) { + if (!ready) { + return false; + } + } + return true; + } + + private static void println(Object msg) { + if (msg instanceof Collection) { + Iterator itr = ((Collection) msg).iterator(); + while (itr.hasNext()) { + System.out.println(itr.next()); + } + } else { + System.out.println(msg); + } + } + + static class HeartbeatSendTask implements Runnable { + private double[] runtimesBook; + private int taskId; + private int tryTimes; + private NimbusClient client; + + public HeartbeatSendTask(int taskId, int tryTimes) { + this.taskId = taskId; + this.tryTimes = tryTimes; + this.runtimesBook = new double[tryTimes]; + try { + client = new NimbusClient(initializedConfig(), NIMBUS_HOST, NIMBUS_PORT, null, null); + } catch (TTransportException e) { + e.printStackTrace(); + } + } + + private static SupervisorWorkerHeartbeat nextMockedWorkerbeat() { + List executorInfos = new ArrayList<>(); + executorInfos.add(new ExecutorInfo(1, 1)); + executorInfos.add(new ExecutorInfo(2, 2)); + executorInfos.add(new ExecutorInfo(3, 3)); + executorInfos.add(new ExecutorInfo(4, 4)); + SupervisorWorkerHeartbeat heartbeat = new SupervisorWorkerHeartbeat(); + heartbeat.set_executors(executorInfos); + // generate a random storm id + heartbeat.set_storm_id("storm_name_example_" + rand.nextInt(MOCKED_STORM_NUM)); + heartbeat.set_time_secs(1221212121); + return heartbeat; + } + + private static SupervisorWorkerHeartbeats mockedHeartbeats() { + SupervisorWorkerHeartbeats heartbeats = new SupervisorWorkerHeartbeats(); + heartbeats.set_supervisor_id("123124134123413412341351234143"); + List workers = new ArrayList<>(); + for (int i = 0; i < 25; i++) { + workers.add(nextMockedWorkerbeat()); + } + heartbeats.set_worker_heartbeats(workers); + return heartbeats; + } + + @Override + public void run() { + try { + for (int i = 0; i < tryTimes; i++) { + long thisBegin = System.currentTimeMillis(); + client.getClient().sendSupervisorWorkerHeartbeats(mockedHeartbeats()); + long thisEnd = System.currentTimeMillis(); + this.runtimesBook[i] = (thisEnd - thisBegin) / 1000D; + } + totalCostTimesBook.add(this.runtimesBook); + readyFlags[taskId] = true; + Thread.currentThread().interrupt(); + } catch (TException e) { + e.printStackTrace(); + } + } + } + + private static void printTimeCostArray(Double[] array) { + StringBuilder builder = new StringBuilder(); + builder.append("["); + for (int i = 0; i < array.length; i++) { + if (i != array.length - 1) { + builder.append(array[i] + ","); + } else { + builder.append(array[i] + ""); + } + } + builder.append("]"); + System.out.println(builder.toString()); + } + + private static void printStatistics(List data) { + + List totalPoints = new ArrayList<>(); + double total = 0D; + for (double[] item : data) { + for (Double point : item) { + if (point != null) { + totalPoints.add(point); + total += point; + } + } + } + Double[] totalPointsArray = new Double[totalPoints.size()]; + + totalPoints.toArray(totalPointsArray); + Arrays.sort(totalPointsArray); + // printTimeCostArray(totalPointsArray); + println("===== statistics ================"); + println("===== min time cost: " + totalPointsArray[0] + " ====="); + println("===== max time cost: " + totalPointsArray[totalPointsArray.length - 2] + " ====="); + + double meanVal = total / totalPointsArray.length; + println("===== mean time cost: " + meanVal + " ====="); + int middleIndex = (int) (totalPointsArray.length * 0.5); + println("===== median time cost: " + totalPointsArray[middleIndex] + " ====="); + int top90Index = (int) (totalPointsArray.length * 0.9); + println("===== top90 time cost: " + totalPointsArray[top90Index] + " ====="); + } + + public static void main(String[] args) { + testMaxThroughput(); + } + +} diff --git a/storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java b/storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java new file mode 100644 index 00000000000..78f880ab22c --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java @@ -0,0 +1,80 @@ +/** + * 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.nimbus; + +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.storm.Config; +import org.apache.storm.utils.ObjectReader; +import org.apache.storm.utils.Time; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.util.stream.Collectors.toSet; + +/** + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies: + * + *

1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids, + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers + * heartbeats on the node are reported. + * + *

2: If several supervisor also crush and will never recover[or all crush for some unknown reason], + * workers will report their heartbeats directly to master, so it has not any effect. + */ +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy { + private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class); + + private static int NODE_MAX_TIMEOUT_SECS = 600; + + private long startTimeSecs; + + private Set reportedIds; + + @Override + public void prepare(Map conf) { + NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600); + this.startTimeSecs = Time.currentTimeMillis() / 1000L; + this.reportedIds = new HashSet<>(); + } + + @Override + public boolean isReady(Set nodeIds) { + if (exceedsMaxTimeOut()) { + Set tmp = nodeIds.stream().filter(id -> !this.reportedIds.contains(id)).collect(toSet()); + LOG.warn("Failed to recover heartbeats for nodes: {} with timeout {}s", tmp, NODE_MAX_TIMEOUT_SECS); + return true; + } + + return nodeIds.stream().allMatch(id -> this.reportedIds.contains(id)); + } + + @Override + public void reportNodeId(String nodeId) { + this.reportedIds.add(nodeId); + } + + private boolean exceedsMaxTimeOut() { + return (Time.currentTimeMillis() / 1000L - this.startTimeSecs) > NODE_MAX_TIMEOUT_SECS; + } + +} diff --git a/storm-server/src/main/java/org/apache/storm/nimbus/WorkerHeartbeatsRecoveryStrategyFactory.java b/storm-server/src/main/java/org/apache/storm/nimbus/WorkerHeartbeatsRecoveryStrategyFactory.java new file mode 100644 index 00000000000..4befaf8a05a --- /dev/null +++ b/storm-server/src/main/java/org/apache/storm/nimbus/WorkerHeartbeatsRecoveryStrategyFactory.java @@ -0,0 +1,53 @@ +/** + * 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.nimbus; + +import com.google.common.base.Preconditions; +import java.util.Map; +import org.apache.storm.DaemonConfig; +import org.apache.storm.utils.ReflectionUtils; + +/** + * Factory class for recovery strategy. + */ +public class WorkerHeartbeatsRecoveryStrategyFactory { + + /** + * Get instance of {@link IWorkerHeartbeatsRecoveryStrategy} with conf. + * @param conf strategy config + * @return an instance of {@link IWorkerHeartbeatsRecoveryStrategy} + */ + public static IWorkerHeartbeatsRecoveryStrategy getStrategy(Map conf) { + IWorkerHeartbeatsRecoveryStrategy strategy; + if (conf.get(DaemonConfig.NIMBUS_WORKER_HEARTBEATS_RECOVERY_STRATEGY_CLASS) != null) { + Object targetObj = ReflectionUtils.newInstance((String) + conf.get(DaemonConfig.NIMBUS_WORKER_HEARTBEATS_RECOVERY_STRATEGY_CLASS)); + Preconditions.checkState(targetObj instanceof IWorkerHeartbeatsRecoveryStrategy, + "{} must implements IWorkerHeartbeatsRecoveryStrategy", + DaemonConfig.NIMBUS_WORKER_HEARTBEATS_RECOVERY_STRATEGY_CLASS); + strategy = ((IWorkerHeartbeatsRecoveryStrategy) targetObj); + } else { + strategy = new TimeOutWorkerHeartbeatsRecoveryStrategy(); + } + + strategy.prepare(conf); + return strategy; + } + +} diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/SupervisorDetails.java b/storm-server/src/main/java/org/apache/storm/scheduler/SupervisorDetails.java index 242b54c4311..00b8ea0e1f3 100644 --- a/storm-server/src/main/java/org/apache/storm/scheduler/SupervisorDetails.java +++ b/storm-server/src/main/java/org/apache/storm/scheduler/SupervisorDetails.java @@ -31,6 +31,10 @@ public class SupervisorDetails { private static final Logger LOG = LoggerFactory.getLogger(SupervisorDetails.class); private final String id; + /** + * thrift server of this supervisor. + */ + private final Integer serverPort; /** * hostname of this supervisor. */ @@ -52,16 +56,18 @@ public class SupervisorDetails { /** * Create the details of a new supervisor. * @param id the ID as reported by the supervisor. + * @param serverPort the thrift server for the supervisor. * @param host the host the supervisor is on. * @param meta meta data reported by the supervisor (should be a collection of the ports on the supervisor). * @param schedulerMeta Not used and can probably be removed. * @param allPorts all of the ports for the supervisor (a better version of meta) * @param totalResources all of the resources for this supervisor. */ - public SupervisorDetails(String id, String host, Object meta, Object schedulerMeta, + public SupervisorDetails(String id, Integer serverPort, String host, Object meta, Object schedulerMeta, Collection allPorts, Map totalResources) { this.id = id; + this.serverPort = serverPort; this.host = host; this.meta = meta; this.schedulerMeta = schedulerMeta; @@ -75,24 +81,29 @@ public SupervisorDetails(String id, String host, Object meta, Object schedulerMe } public SupervisorDetails(String id, Object meta) { - this(id, null, meta, null, null, null); + this(id, null, null, meta, null, null, null); } public SupervisorDetails(String id, Object meta, Map totalResources) { - this(id, null, meta, null, null, totalResources); + this(id, null, null, meta, null, null, totalResources); } public SupervisorDetails(String id, Object meta, Collection allPorts) { - this(id, null, meta, null, allPorts, null); + this(id, null, null, meta, null, allPorts, null); } public SupervisorDetails(String id, String host, Object schedulerMeta, Collection allPorts) { - this(id, host, null, schedulerMeta, allPorts, null); + this(id, null, host, null, schedulerMeta, allPorts, null); } public SupervisorDetails(String id, String host, Object schedulerMeta, Collection allPorts, Map totalResources) { - this(id, host, null, schedulerMeta, allPorts, totalResources); + this(id, null, host, null, schedulerMeta, allPorts, totalResources); + } + + public SupervisorDetails(String id, int serverPort, String host, Object schedulerMeta, + Collection allPorts, Map totalResources) { + this(id, serverPort, host, null, schedulerMeta, allPorts, totalResources); } @Override @@ -114,6 +125,10 @@ public String getId() { return id; } + public int getServerPort() { + return serverPort; + } + public String getHost() { return host; } diff --git a/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java b/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java index 5dfdf0b1a0a..59313a098ce 100644 --- a/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java +++ b/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java @@ -22,10 +22,13 @@ import org.apache.curator.framework.recipes.leader.LeaderLatchListener; import org.apache.curator.framework.recipes.leader.Participant; import org.apache.storm.blobstore.BlobStore; +import org.apache.storm.cluster.IStormClusterState; import org.apache.storm.daemon.nimbus.TopoCache; import org.apache.storm.nimbus.ILeaderElector; +import org.apache.storm.nimbus.LeaderListenerCallback; import org.apache.storm.nimbus.NimbusInfo; import org.apache.storm.utils.Utils; +import org.apache.zookeeper.data.ACL; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,10 +49,12 @@ public class LeaderElectorImp implements ILeaderElector { private final AtomicReference leaderLatchListener; private final BlobStore blobStore; private final TopoCache tc; + private final IStormClusterState clusterState; + private final List acls; public LeaderElectorImp(Map conf, List servers, CuratorFramework zk, String leaderlockPath, String id, AtomicReference leaderLatch, AtomicReference leaderLatchListener, - BlobStore blobStore, final TopoCache tc) { + BlobStore blobStore, final TopoCache tc, IStormClusterState clusterState, List acls) { this.conf = conf; this.servers = servers; this.zk = zk; @@ -59,6 +64,8 @@ public LeaderElectorImp(Map conf, List servers, CuratorF this.leaderLatchListener = leaderLatchListener; this.blobStore = blobStore; this.tc = tc; + this.clusterState = clusterState; + this.acls = acls; } @Override @@ -71,7 +78,8 @@ public void addToLeaderLockQueue() throws Exception { // if this latch is already closed, we need to create new instance. if (LeaderLatch.State.CLOSED.equals(leaderLatch.get().getState())) { leaderLatch.set(new LeaderLatch(zk, leaderlockPath)); - leaderLatchListener.set(Zookeeper.leaderLatchListenerImpl(conf, zk, blobStore, leaderLatch.get(), tc)); + LeaderListenerCallback callback = new LeaderListenerCallback(conf, zk, leaderLatch.get(), blobStore, tc, clusterState, acls); + leaderLatchListener.set(Zookeeper.leaderLatchListenerImpl(callback)); LOG.info("LeaderLatch was in closed state. Resetted the leaderLatch and listeners."); } // Only if the latch is not already started we invoke start diff --git a/storm-server/src/main/java/org/apache/storm/zookeeper/Zookeeper.java b/storm-server/src/main/java/org/apache/storm/zookeeper/Zookeeper.java index 8f46dbec899..0b401a5c12e 100644 --- a/storm-server/src/main/java/org/apache/storm/zookeeper/Zookeeper.java +++ b/storm-server/src/main/java/org/apache/storm/zookeeper/Zookeeper.java @@ -17,10 +17,7 @@ */ package org.apache.storm.zookeeper; -import com.google.common.base.Joiner; -import com.google.common.collect.Sets; -import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderLatch; @@ -28,25 +25,18 @@ import org.apache.curator.framework.recipes.leader.Participant; import org.apache.storm.Config; import org.apache.storm.blobstore.BlobStore; -import org.apache.storm.blobstore.InputStreamWithMeta; -import org.apache.storm.callback.DefaultWatcherCallBack; -import org.apache.storm.cluster.ClusterUtils; +import org.apache.storm.cluster.IStormClusterState; import org.apache.storm.daemon.nimbus.TopoCache; -import org.apache.storm.generated.AuthorizationException; -import org.apache.storm.generated.KeyNotFoundException; -import org.apache.storm.generated.StormTopology; import org.apache.storm.nimbus.ILeaderElector; +import org.apache.storm.nimbus.LeaderListenerCallback; import org.apache.storm.nimbus.NimbusInfo; -import org.apache.storm.security.auth.ReqContext; -import org.apache.storm.utils.Utils; +import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.server.NIOServerCnxnFactory; import org.apache.zookeeper.server.ZooKeeperServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.security.auth.Subject; import java.io.File; -import java.io.IOException; import java.net.BindException; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -125,127 +115,22 @@ public static NimbusInfo toNimbusInfo(Participant participant) { } // Leader latch listener that will be invoked when we either gain or lose leadership - public static LeaderLatchListener leaderLatchListenerImpl(final Map conf, final CuratorFramework zk, - final BlobStore blobStore, final LeaderLatch leaderLatch, - final TopoCache tc) + public static LeaderLatchListener leaderLatchListenerImpl(final LeaderListenerCallback callback) throws UnknownHostException { final String hostName = InetAddress.getLocalHost().getCanonicalHostName(); return new LeaderLatchListener() { - final String STORM_JAR_SUFFIX = "-stormjar.jar"; - final String STORM_CODE_SUFFIX = "-stormcode.ser"; - final String STORM_CONF_SUFFIX = "-stormconf.ser"; @Override public void isLeader() { - Set activeTopologyIds = new TreeSet<>(ClientZookeeper.getChildren(zk, - ClusterUtils.STORMS_SUBTREE, false)); - - Set activeTopologyBlobKeys = populateTopologyBlobKeys(activeTopologyIds); - Set activeTopologyCodeKeys = filterTopologyCodeKeys(activeTopologyBlobKeys); - Set allLocalBlobKeys = Sets.newHashSet(blobStore.listKeys()); - Set allLocalTopologyBlobKeys = filterTopologyBlobKeys(allLocalBlobKeys); - - // this finds all active topologies blob keys from all local topology blob keys - Sets.SetView diffTopology = Sets.difference(activeTopologyBlobKeys, allLocalTopologyBlobKeys); - LOG.info("active-topology-blobs [{}] local-topology-blobs [{}] diff-topology-blobs [{}]", - generateJoinedString(activeTopologyIds), generateJoinedString(allLocalTopologyBlobKeys), - generateJoinedString(diffTopology)); - - if (diffTopology.isEmpty()) { - Set activeTopologyDependencies = getTopologyDependencyKeys(activeTopologyCodeKeys); - - // this finds all dependency blob keys from active topologies from all local blob keys - Sets.SetView diffDependencies = Sets.difference(activeTopologyDependencies, allLocalBlobKeys); - LOG.info("active-topology-dependencies [{}] local-blobs [{}] diff-topology-dependencies [{}]", - generateJoinedString(activeTopologyDependencies), generateJoinedString(allLocalBlobKeys), - generateJoinedString(diffDependencies)); - - if (diffDependencies.isEmpty()) { - LOG.info("Accepting leadership, all active topologies and corresponding dependencies found locally."); - tc.clear(); - } else { - LOG.info("Code for all active topologies is available locally, but some dependencies are not found locally, " - + "giving up leadership."); - closeLatch(); - } - } else { - LOG.info("code for all active topologies not available locally, giving up leadership."); - closeLatch(); - } + callback.leaderCallBack(); + LOG.info("{} gained leadership.", hostName); } @Override public void notLeader() { LOG.info("{} lost leadership.", hostName); //Just to be sure - tc.clear(); - } - - private String generateJoinedString(Set activeTopologyIds) { - return Joiner.on(",").join(activeTopologyIds); - } - - private Set populateTopologyBlobKeys(Set activeTopologyIds) { - Set activeTopologyBlobKeys = new TreeSet<>(); - for (String activeTopologyId : activeTopologyIds) { - activeTopologyBlobKeys.add(activeTopologyId + STORM_JAR_SUFFIX); - activeTopologyBlobKeys.add(activeTopologyId + STORM_CODE_SUFFIX); - activeTopologyBlobKeys.add(activeTopologyId + STORM_CONF_SUFFIX); - } - return activeTopologyBlobKeys; - } - - private Set filterTopologyBlobKeys(Set blobKeys) { - Set topologyBlobKeys = new HashSet<>(); - for (String blobKey : blobKeys) { - if (blobKey.endsWith(STORM_JAR_SUFFIX) || blobKey.endsWith(STORM_CODE_SUFFIX) || - blobKey.endsWith(STORM_CONF_SUFFIX)) { - topologyBlobKeys.add(blobKey); - } - } - return topologyBlobKeys; - } - - private Set filterTopologyCodeKeys(Set blobKeys) { - Set topologyCodeKeys = new HashSet<>(); - for (String blobKey : blobKeys) { - if (blobKey.endsWith(STORM_CODE_SUFFIX)) { - topologyCodeKeys.add(blobKey); - } - } - return topologyCodeKeys; - } - - private Set getTopologyDependencyKeys(Set activeTopologyCodeKeys) { - Set activeTopologyDependencies = new TreeSet<>(); - Subject subject = ReqContext.context().subject(); - - for (String activeTopologyCodeKey : activeTopologyCodeKeys) { - try { - InputStreamWithMeta blob = blobStore.getBlob(activeTopologyCodeKey, subject); - byte[] blobContent = IOUtils.readFully(blob, new Long(blob.getFileLength()).intValue()); - StormTopology stormCode = Utils.deserialize(blobContent, StormTopology.class); - if (stormCode.is_set_dependency_jars()) { - activeTopologyDependencies.addAll(stormCode.get_dependency_jars()); - } - if (stormCode.is_set_dependency_artifacts()) { - activeTopologyDependencies.addAll(stormCode.get_dependency_artifacts()); - } - } catch (AuthorizationException | KeyNotFoundException | IOException e) { - LOG.error("Exception occurs while reading blob for key: " + activeTopologyCodeKey + ", exception: " + e, e); - throw new RuntimeException("Exception occurs while reading blob for key: " + activeTopologyCodeKey + - ", exception: " + e, e); - } - } - return activeTopologyDependencies; - } - - private void closeLatch() { - try { - leaderLatch.close(); - } catch (IOException e) { - throw new RuntimeException(e); - } + callback.notLeaderCallback(); } }; } @@ -256,24 +141,26 @@ private void closeLatch() { * @param zkClient ZkClient, the client must have a default Config.STORM_ZOOKEEPER_ROOT as root path. * @param blobStore {@link BlobStore} * @param tc {@link TopoCache} + * @param clusterState {@link IStormClusterState} + * @param acls ACLs * @return Instance of {@link ILeaderElector} * @throws UnknownHostException */ - public static ILeaderElector zkLeaderElector(Map conf, CuratorFramework zkClient, - BlobStore blobStore, final TopoCache tc) throws UnknownHostException { - return _instance.zkLeaderElectorImpl(conf, zkClient, blobStore, tc); + public static ILeaderElector zkLeaderElector(Map conf, CuratorFramework zkClient, BlobStore blobStore, + final TopoCache tc, IStormClusterState clusterState, List acls) throws UnknownHostException { + return _instance.zkLeaderElectorImpl(conf, zkClient, blobStore, tc, clusterState, acls); } - protected ILeaderElector zkLeaderElectorImpl(Map conf, CuratorFramework zk, BlobStore blobStore, final TopoCache tc) - throws UnknownHostException { + protected ILeaderElector zkLeaderElectorImpl(Map conf, CuratorFramework zk, BlobStore blobStore, + final TopoCache tc, IStormClusterState clusterState, List acls) throws UnknownHostException { List servers = (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS); String leaderLockPath = "/leader-lock"; String id = NimbusInfo.fromConf(conf).toHostPortString(); AtomicReference leaderLatchAtomicReference = new AtomicReference<>(new LeaderLatch(zk, leaderLockPath, id)); AtomicReference leaderLatchListenerAtomicReference = - new AtomicReference<>(leaderLatchListenerImpl(conf, zk, blobStore, leaderLatchAtomicReference.get(), tc)); + new AtomicReference<>(leaderLatchListenerImpl(new LeaderListenerCallback(conf, zk, leaderLatchAtomicReference.get(), blobStore, tc, clusterState, acls))); return new LeaderElectorImp(conf, servers, zk, leaderLockPath, id, leaderLatchAtomicReference, - leaderLatchListenerAtomicReference, blobStore, tc); + leaderLatchListenerAtomicReference, blobStore, tc, clusterState, acls); } } diff --git a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/BasicContainerTest.java b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/BasicContainerTest.java index fe0a993430b..b70e844a84a 100644 --- a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/BasicContainerTest.java +++ b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/BasicContainerTest.java @@ -56,12 +56,12 @@ public CommandRun(List cmd, Map env, File pwd) { } public static class MockBasicContainer extends BasicContainer { - public MockBasicContainer(ContainerType type, Map conf, String supervisorId, int port, - LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, LocalState localState, - String workerId, Map topoConf, AdvancedFSOps ops, String profileCmd) - throws IOException { - super(type, conf, supervisorId, port, assignment, resourceIsolationManager, localState, workerId, topoConf, ops, - profileCmd); + public MockBasicContainer(ContainerType type, Map conf, String supervisorId, int supervisorPort, + int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, + LocalState localState, String workerId, Map topoConf, AdvancedFSOps ops, + String profileCmd) throws IOException { + super(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, localState, + workerId, topoConf, ops, profileCmd); } public final List profileCmds = new ArrayList<>(); @@ -118,6 +118,7 @@ protected String javaLibraryPath(String stormRoot, Map conf) { @Test public void testCreateNewWorkerId() throws Exception { final String topoId = "test_topology"; + final int supervisorPort = 6628; final int port = 8080; LocalAssignment la = new LocalAssignment(); la.set_topology_id(topoId); @@ -129,7 +130,7 @@ public void testCreateNewWorkerId() throws Exception { LocalState ls = mock(LocalState.class); MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf, - "SUPERVISOR", port, la, null, ls, null, new HashMap<>(), ops, "profile"); + "SUPERVISOR", supervisorPort, port, la, null, ls, null, new HashMap<>(), ops, "profile"); //null worker id means generate one... assertNotNull(mc._workerId); @@ -143,6 +144,7 @@ public void testCreateNewWorkerId() throws Exception { public void testRecovery() throws Exception { final String topoId = "test_topology"; final String workerId = "myWorker"; + final int supervisorPort = 6628; final int port = 8080; LocalAssignment la = new LocalAssignment(); la.set_topology_id(topoId); @@ -158,7 +160,7 @@ public void testRecovery() throws Exception { when(ops.doRequiredTopoFilesExist(superConf, topoId)).thenReturn(true); MockBasicContainer mc = new MockBasicContainer(ContainerType.RECOVER_FULL, superConf, - "SUPERVISOR", port, la, null, ls, null, new HashMap<>(), ops, "profile"); + "SUPERVISOR", supervisorPort, port, la, null, ls, null, new HashMap<>(), ops, "profile"); assertEquals(workerId, mc._workerId); } @@ -166,6 +168,7 @@ public void testRecovery() throws Exception { @Test public void testRecoveryMiss() throws Exception { final String topoId = "test_topology"; + final int supervisorPort = 6628; final int port = 8080; LocalAssignment la = new LocalAssignment(); la.set_topology_id(topoId); @@ -178,7 +181,7 @@ public void testRecoveryMiss() throws Exception { try { new MockBasicContainer(ContainerType.RECOVER_FULL, new HashMap(), - "SUPERVISOR", port, la, null, ls, null, new HashMap<>(), null, "profile"); + "SUPERVISOR", supervisorPort, port, la, null, ls, null, new HashMap<>(), null, "profile"); fail("Container recovered worker incorrectly"); } catch (ContainerRecoveryException e) { //Expected @@ -188,6 +191,7 @@ public void testRecoveryMiss() throws Exception { @Test public void testCleanUp() throws Exception { final String topoId = "test_topology"; + final int supervisorPort = 6628; final int port = 8080; final String workerId = "worker-id"; LocalAssignment la = new LocalAssignment(); @@ -204,7 +208,7 @@ public void testCleanUp() throws Exception { when(ls.getApprovedWorkers()).thenReturn(new HashMap<>(workerState)); MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf, - "SUPERVISOR", port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); + "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); mc.cleanUp(); @@ -218,6 +222,7 @@ public void testCleanUp() throws Exception { public void testRunProfiling() throws Exception { final long pid = 100; final String topoId = "test_topology"; + final int supervisorPort = 6628; final int port = 8080; final String workerId = "worker-id"; final String stormLocal = ContainerTest.asAbsPath("tmp", "testing"); @@ -238,7 +243,7 @@ public void testRunProfiling() throws Exception { LocalState ls = mock(LocalState.class); MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf, - "SUPERVISOR", port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); + "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); //HEAP DUMP ProfileRequest req = new ProfileRequest(); @@ -357,7 +362,8 @@ private static void assertListEquals(List a, List b) { @Test public void testLaunch() throws Exception { - final String topoId = "test_topology"; + final String topoId = "test_topology_current"; + final int supervisorPort = 6628; final int port = 8080; final String stormHome = ContainerTest.asAbsPath("tmp", "storm-home"); final String stormLogDir = ContainerTest.asFile(".", "target").getCanonicalPath(); @@ -393,8 +399,8 @@ public void testLaunch() throws Exception { LocalState ls = mock(LocalState.class); checkpoint(() -> { - MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf, - "SUPERVISOR", port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); + MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf, + "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); mc.launch(); @@ -440,21 +446,223 @@ public void testLaunch() throws Exception { "-cp", "FRAMEWORK_CP:" + stormjar.getAbsolutePath(), "org.apache.storm.daemon.worker.Worker", - topoId, + topoId, "SUPERVISOR", + String.valueOf(supervisorPort), String.valueOf(port), workerId ), cmd.cmd); assertEquals(new File(workerRoot), cmd.pwd); - }, + }, "storm.home", stormHome, "storm.log.dir", stormLogDir); } - + + @Test + public void testLaunchStorm1version() throws Exception { + final String topoId = "test_topology_storm_1.x"; + final int supervisorPort = 6628; + final int port = 8080; + final String stormHome = ContainerTest.asAbsPath("tmp", "storm-home"); + final String stormLogDir = ContainerTest.asFile(".", "target").getCanonicalPath(); + final String workerId = "worker-id"; + final String stormLocal = ContainerTest.asAbsPath("tmp", "storm-local"); + final String distRoot = ContainerTest.asAbsPath(stormLocal, "supervisor", "stormdist", topoId); + final File stormcode = new File(distRoot, "stormcode.ser"); + final File stormjar = new File(distRoot, "stormjar.jar"); + final String log4jdir = ContainerTest.asAbsPath(stormHome, "conf"); + final String workerConf = ContainerTest.asAbsPath(log4jdir, "worker.xml"); + final String workerRoot = ContainerTest.asAbsPath(stormLocal, "workers", workerId); + final String workerTmpDir = ContainerTest.asAbsPath(workerRoot, "tmp"); + + final StormTopology st = new StormTopology(); + st.set_spouts(new HashMap<>()); + st.set_bolts(new HashMap<>()); + st.set_state_spouts(new HashMap<>()); + + // minimum 1.x version of supporting STORM-2448 would be 1.0.4 + st.set_storm_version("1.0.4"); + byte [] serializedState = Utils.gzip(Utils.thriftSerialize(st)); + + final Map superConf = new HashMap<>(); + superConf.put(Config.STORM_LOCAL_DIR, stormLocal); + superConf.put(Config.STORM_WORKERS_ARTIFACTS_DIR, stormLocal); + superConf.put(DaemonConfig.STORM_LOG4J2_CONF_DIR, log4jdir); + superConf.put(Config.WORKER_CHILDOPTS, " -Dtesting=true"); + + LocalAssignment la = new LocalAssignment(); + la.set_topology_id(topoId); + + AdvancedFSOps ops = mock(AdvancedFSOps.class); + when(ops.doRequiredTopoFilesExist(superConf, topoId)).thenReturn(true); + when(ops.slurp(stormcode)).thenReturn(serializedState); + + LocalState ls = mock(LocalState.class); + + checkpoint(() -> { + MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf, + "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); + + mc.launch(); + + assertEquals(1, mc.workerCmds.size()); + CommandRun cmd = mc.workerCmds.get(0); + mc.workerCmds.clear(); + assertListEquals(Arrays.asList( + "java", + "-cp", + "FRAMEWORK_CP:" + stormjar.getAbsolutePath(), + "-Dlogging.sensitivity=S3", + "-Dlogfile.name=worker.log", + "-Dstorm.home=" + stormHome, + "-Dworkers.artifacts=" + stormLocal, + "-Dstorm.id=" + topoId, + "-Dworker.id=" + workerId, + "-Dworker.port=" + port, + "-Dstorm.log.dir=" + stormLogDir, + "-Dlog4j.configurationFile=" + workerConf, + "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector", + "-Dstorm.local.dir=" + stormLocal, + "-Dworker.memory_limit_mb=768", + "org.apache.storm.LogWriter", + "java", + "-server", + "-Dlogging.sensitivity=S3", + "-Dlogfile.name=worker.log", + "-Dstorm.home=" + stormHome, + "-Dworkers.artifacts=" + stormLocal, + "-Dstorm.id=" + topoId, + "-Dworker.id=" + workerId, + "-Dworker.port=" + port, + "-Dstorm.log.dir=" + stormLogDir, + "-Dlog4j.configurationFile=" + workerConf, + "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector", + "-Dstorm.local.dir=" + stormLocal, + "-Dworker.memory_limit_mb=768", + "-Dtesting=true", + "-Djava.library.path=JLP", + "-Dstorm.conf.file=", + "-Dstorm.options=", + "-Djava.io.tmpdir="+workerTmpDir, + "-cp", + "FRAMEWORK_CP:" + stormjar.getAbsolutePath(), + "org.apache.storm.daemon.worker", + topoId, + "SUPERVISOR", + String.valueOf(port), + workerId + ), cmd.cmd); + assertEquals(new File(workerRoot), cmd.pwd); + }, + "storm.home", stormHome, + "storm.log.dir", stormLogDir); + } + + @Test + public void testLaunchStorm0version() throws Exception { + final String topoId = "test_topology_storm_0.x"; + final int supervisorPort = 6628; + final int port = 8080; + final String stormHome = ContainerTest.asAbsPath("tmp", "storm-home"); + final String stormLogDir = ContainerTest.asFile(".", "target").getCanonicalPath(); + final String workerId = "worker-id"; + final String stormLocal = ContainerTest.asAbsPath("tmp", "storm-local"); + final String distRoot = ContainerTest.asAbsPath(stormLocal, "supervisor", "stormdist", topoId); + final File stormcode = new File(distRoot, "stormcode.ser"); + final File stormjar = new File(distRoot, "stormjar.jar"); + final String log4jdir = ContainerTest.asAbsPath(stormHome, "conf"); + final String workerConf = ContainerTest.asAbsPath(log4jdir, "worker.xml"); + final String workerRoot = ContainerTest.asAbsPath(stormLocal, "workers", workerId); + final String workerTmpDir = ContainerTest.asAbsPath(workerRoot, "tmp"); + + final StormTopology st = new StormTopology(); + st.set_spouts(new HashMap<>()); + st.set_bolts(new HashMap<>()); + st.set_state_spouts(new HashMap<>()); + + // minimum 0.x version of supporting STORM-2448 would be 0.10.3 + st.set_storm_version("0.10.3"); + byte [] serializedState = Utils.gzip(Utils.thriftSerialize(st)); + + final Map superConf = new HashMap<>(); + superConf.put(Config.STORM_LOCAL_DIR, stormLocal); + superConf.put(Config.STORM_WORKERS_ARTIFACTS_DIR, stormLocal); + superConf.put(DaemonConfig.STORM_LOG4J2_CONF_DIR, log4jdir); + superConf.put(Config.WORKER_CHILDOPTS, " -Dtesting=true"); + + LocalAssignment la = new LocalAssignment(); + la.set_topology_id(topoId); + + AdvancedFSOps ops = mock(AdvancedFSOps.class); + when(ops.doRequiredTopoFilesExist(superConf, topoId)).thenReturn(true); + when(ops.slurp(stormcode)).thenReturn(serializedState); + + LocalState ls = mock(LocalState.class); + + checkpoint(() -> { + MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf, + "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); + + mc.launch(); + + assertEquals(1, mc.workerCmds.size()); + CommandRun cmd = mc.workerCmds.get(0); + mc.workerCmds.clear(); + assertListEquals(Arrays.asList( + "java", + "-cp", + "FRAMEWORK_CP:" + stormjar.getAbsolutePath(), + "-Dlogging.sensitivity=S3", + "-Dlogfile.name=worker.log", + "-Dstorm.home=" + stormHome, + "-Dworkers.artifacts=" + stormLocal, + "-Dstorm.id=" + topoId, + "-Dworker.id=" + workerId, + "-Dworker.port=" + port, + "-Dstorm.log.dir=" + stormLogDir, + "-Dlog4j.configurationFile=" + workerConf, + "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector", + "-Dstorm.local.dir=" + stormLocal, + "-Dworker.memory_limit_mb=768", + "backtype.storm.LogWriter", + "java", + "-server", + "-Dlogging.sensitivity=S3", + "-Dlogfile.name=worker.log", + "-Dstorm.home=" + stormHome, + "-Dworkers.artifacts=" + stormLocal, + "-Dstorm.id=" + topoId, + "-Dworker.id=" + workerId, + "-Dworker.port=" + port, + "-Dstorm.log.dir=" + stormLogDir, + "-Dlog4j.configurationFile=" + workerConf, + "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector", + "-Dstorm.local.dir=" + stormLocal, + "-Dworker.memory_limit_mb=768", + "-Dtesting=true", + "-Djava.library.path=JLP", + "-Dstorm.conf.file=", + "-Dstorm.options=", + "-Djava.io.tmpdir="+workerTmpDir, + "-cp", + "FRAMEWORK_CP:" + stormjar.getAbsolutePath(), + "backtype.storm.daemon.worker", + topoId, + "SUPERVISOR", + String.valueOf(port), + workerId + ), cmd.cmd); + assertEquals(new File(workerRoot), cmd.pwd); + }, + "storm.home", stormHome, + "storm.log.dir", stormLogDir); + } + @Test public void testSubstChildOpts() throws Exception { String workerId = "w-01"; String topoId = "s-01"; + int supervisorPort = 6628; int port = 9999; int memOnheap = 512; @@ -469,7 +677,7 @@ public void testSubstChildOpts() throws Exception { LocalState ls = mock(LocalState.class); MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf, - "SUPERVISOR", port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); + "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new HashMap<>(), ops, "profile"); assertListEquals(Arrays.asList( "-Xloggc:/tmp/storm/logs/gc.worker-9999-s-01-w-01-9999.log", diff --git a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java index 35c579039f8..c0202441653 100644 --- a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java +++ b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java @@ -46,10 +46,11 @@ public class ContainerTest { public static class MockContainer extends Container { - protected MockContainer(ContainerType type, Map conf, String supervisorId, int port, - LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, String workerId, - Map topoConf, AdvancedFSOps ops) throws IOException { - super(type, conf, supervisorId, port, assignment, resourceIsolationManager, workerId, topoConf, ops); + protected MockContainer(ContainerType type, Map conf, String supervisorId, int supervisorPort, + int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, + String workerId, Map topoConf, AdvancedFSOps ops) throws IOException { + super(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, workerId, + topoConf, ops); } public final List killedPids = new ArrayList<>(); @@ -103,7 +104,7 @@ public void testKill() throws Exception { LocalAssignment la = new LocalAssignment(); la.set_topology_id(topoId); MockContainer mc = new MockContainer(ContainerType.LAUNCH, superConf, - "SUPERVISOR", 8080, la, null, "worker", new HashMap<>(), ops); + "SUPERVISOR", 6628, 8080, la, null, "worker", new HashMap<>(), ops); mc.kill(); assertEquals(Collections.EMPTY_LIST, mc.killedPids); assertEquals(Collections.EMPTY_LIST, mc.forceKilledPids); @@ -184,7 +185,7 @@ public void testSetup() throws Exception { la.set_topology_id(topoId); la.set_owner(user); MockContainer mc = new MockContainer(ContainerType.LAUNCH, superConf, - "SUPERVISOR", 8080, la, null, workerId, topoConf, ops); + "SUPERVISOR", 6628, 8080, la, null, workerId, topoConf, ops); mc.setup(); @@ -222,6 +223,7 @@ public void testSetup() throws Exception { @Test public void testCleanup() throws Exception { + final int supervisorPort = 6628; final int port = 8080; final long pid = 100; final String topoId = "test_topology"; @@ -254,7 +256,7 @@ public void testCleanup() throws Exception { la.set_owner(user); la.set_topology_id(topoId); MockContainer mc = new MockContainer(ContainerType.LAUNCH, superConf, - "SUPERVISOR", port, la, iso, workerId, topoConf, ops); + "SUPERVISOR", supervisorPort, port, la, iso, workerId, topoConf, ops); mc.allPids.add(pid); mc.cleanUp();