Merge tag 'v0.9.2-incubating' into security

[maven-release-plugin]  copy for tag v0.9.2-incubating

Conflicts:
	storm-core/pom.xml
diff --git a/SECURITY.md b/SECURITY.md
index 93036b2..882954a 100644
--- a/SECURITY.md
+++ b/SECURITY.md
@@ -1,16 +1,19 @@
 # Running Apache Storm Securely
 
-The current release of Apache Storm offers no authentication or authorization.
-It does not encrypt any data being sent across the network, and does not 
-attempt to restrict access to data stored on the local file system or in
-Apache Zookeeper.  As such there are a number of different precautions you may
-want to enact outside of storm itself to be sure storm is running securely.
+Apache Storm offers a range of configuration options when trying to secure
+your cluster.  By default all authentication and authorization is disabled but 
+can be turned on as needed.
+
+## Firewall/OS level Security
+
+You can still have a secure storm cluster without turning on formal
+Authentication and Authorization. But to do so usually requires 
+configuring your Operating System to ristrict the operations that can be done.
+This is generally a good idea even if you plan on running your cluster with Auth.
 
 The exact detail of how to setup these precautions varies a lot and is beyond
 the scope of this document.
 
-## Network Security
-
 It is generally a good idea to enable a firewall and restrict incoming network
 connections to only those originating from the cluster itself and from trusted
 hosts and services, a complete list of ports storm uses are below. 
@@ -34,41 +37,305 @@
 
 The UI and logviewer processes provide a way to not only see what a cluster is
 doing, but also manipulate running topologies.  In general these processes should
-not be exposed except to users of the cluster.  It is often simplest to restrict
-these ports to only accept connections from local hosts, and then front them with another web server,
-like Apache httpd, that can authenticate/authorize incoming connections and
+not be exposed except to users of the cluster.
+
+Some form of Authentication is typically required, with using java servlet filters 
+
+```yaml
+ui.filter: "filter.class"
+ui.filter.params: "param1":"value1"
+```
+or by restricting the UI/log viewers ports to only accept connections from local
+hosts, and then front them with another web server, like Apache httpd, that can
+authenticate/authorize incoming connections and
 proxy the connection to the storm process.  To make this work the ui process must have
 logviewer.port set to the port of the proxy in its storm.yaml, while the logviewers
 must have it set to the actual port that they are going to bind to.
 
-### Nimbus
+The servlet filters are prefered because it allows indavidual topologies to
+specificy who is and who is not allowed to access the pages associated with
+them.
 
-Nimbus's Thrift port should be locked down as it can be used to control the entire
-cluster including running arbitrary user code on different nodes in the cluster.
-Ideally access to it is restricted to nodes within the cluster and possibly some gateway
-nodes that allow authorized users to log into them and run storm client commands.
+## Authentication (Kerberos)
+
+Storm offers pluggable authentication support through thrift and SASL.  This
+example only goes off of Kerberos as it is a common setup for most big data
+projects.
+
+Setting up a KDC and configuring kerberos on each node is beyond the scope of
+this document and it is assumed that you have done that already.
+
+### Create Headless Principals and keytabs
+
+Each Zookeeper Server, Nimbus, and DRPC server will need a service principal, which, by convention, includes the FQDN of the host it will run on.  Be aware that the zookeeper user *MUST* be zookeeper.  
+The supervisors and UI also need a principal to run as, but because they are outgoing connections they do not need to be service principals. 
+The following is an example of how to setup kerberos principals, but the
+details may varry depending on your KDC and OS.
+
+
+```bash
+# Zookeeper (Will need one of these for each box in teh Zk ensamble)
+sudo kadmin.local -q 'addprinc zookeeper/zk1.example.com@STORM.EXAMPLE.COM'
+sudo kadmin.local -q "ktadd -k /tmp/zk.keytab  zookeeper/zk1.example.com@STORM.EXAMPLE.COM"
+# Nimbus and DRPC
+sudo kadmin.local -q 'addprinc storm/storm.example.com@STORM.EXAMPLE.COM'
+sudo kadmin.local -q "ktadd -k /tmp/storm.keytab storm/storm.example.com@STORM.EXAMPLE.COM"
+# All UI logviewer and Supervisors
+sudo kadmin.local -q 'addprinc storm@STORM.EXAMPLE.COM'
+sudo kadmin.local -q "ktadd -k /tmp/storm.keytab storm@STORM.EXAMPLE.COM"
+```
+
+be sure to distribute the keytab(s) to the appropriate boxes and set the FS permissions so that only the headless user running ZK, or storm has access to them.
+
+#### Storm Kerberos Configuration
+
+Both storm and Zookeeper use jaas configuration files to log the user in.
+Each jaas file may have multiple sections for different interfaces being used.
+
+To enable Kerberos authentication in storm you need to set the following storm.yaml configs
+```yaml
+storm.thrift.transport: "backtype.storm.security.auth.kerberos.KerberosSaslTransportPlugin"
+java.security.auth.login.config: "/path/to/jaas.conf"
+```
+
+Nimbus and the supervisor processes will also connect to ZooKeeper(ZK) and we want to configure them to use Kerberos for authentication with ZK. To do this append 
+```
+-Djava.security.auth.login.config=/path/to/jaas.conf
+```
+
+to the childopts of nimbus, ui, and supervisor.  Here is an example given the default childopts settings at the time of writing:
+
+```yaml
+nimbus.childopts: "-Xmx1024m -Djava.security.auth.login.config=/path/to/jaas.conf"
+ui.childopts: "-Xmx768m -Djava.security.auth.login.config=/path/to/jaas.conf"
+supervisor.childopts: "-Xmx256m -Djava.security.auth.login.config=/path/to/jaas.conf"
+```
+
+The jaas.conf file should look something like the following for the storm nodes.
+The StormServer section is used by nimbus and the DRPC Nodes.  It does not need to be included on supervisor nodes.
+The StormClient section is used by all storm clients that want to talk to nimbus, including the ui, logviewer, and supervisor.  We will use this section on the gateways as well but the structure of that will be a bit different.
+The Client section is used by processes wanting to talk to zookeeper and really only needs to be included with nimbus and the supervisors.
+The Server section is used by the zookeeper servers.
+Having unused sections in the jaas is not a problem.
+
+```
+StormServer {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="$keytab"
+   storeKey=true
+   useTicketCache=false
+   principal="$principal";
+};
+StormClient {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="$keytab"
+   storeKey=true
+   useTicketCache=false
+   serviceName="$nimbus_user"
+   principal="$principal";
+};
+Client {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="$keytab"
+   storeKey=true
+   useTicketCache=false
+   serviceName="zookeeper"
+   principal="$principal";
+};
+Server {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="$keytab"
+   storeKey=true
+   useTicketCache=false
+   principal="$principal";
+};
+```
+
+The following is an example based off of the keytabs generated
+```
+StormServer {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="/keytabs/storm.keytab"
+   storeKey=true
+   useTicketCache=false
+   principal="storm/storm.example.com@STORM.EXAMPLE.COM";
+};
+StormClient {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="/keytabs/storm.keytab"
+   storeKey=true
+   useTicketCache=false
+   serviceName="storm"
+   principal="storm@STORM.EXAMPLE.COM";
+};
+Client {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="/keytabs/storm.keytab"
+   storeKey=true
+   useTicketCache=false
+   serviceName="zookeeper"
+   principal="storm@STORM.EXAMPLE.COM";
+};
+Server {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="/keytabs/zk.keytab"
+   storeKey=true
+   useTicketCache=false
+   serviceName="zookeeper"
+   principal="zookeeper/zk1.example.com@STORM.EXAMPLE.COM";
+};
+```
+
+Nimbus also will translate the principal into a local user name, so that other services can use this name.  To configure this for Kerberos authentication set
+
+```
+storm.principal.tolocal: "backtype.storm.security.auth.KerberosPrincipalToLocal"
+```
+
+This only needs to be done on nimbus, but it will not hurt on any node.
+We also need to inform the topology who the supervisor daemon and the nimbus daemon are running as from a ZooKeeper perspective.
+
+```
+storm.zookeeper.superACL: "sasl:${nimbus-user}"
+```
+
+Here *nimbus-user* is the Kerberos user that nimbus uses to authenticate with ZooKeeper.  If ZooKeeeper is stripping host and realm then this needs to have host and realm stripped too.
+
+#### ZooKeeper Ensemble
+
+Complete details of how to setup a secure ZK are beyond the scope of this document.  But in general you want to enable SASL authentication on each server, and optionally strip off host and realm
+
+```
+authProvider.1 = org.apache.zookeeper.server.auth.SASLAuthenticationProvider
+kerberos.removeHostFromPrincipal = true
+kerberos.removeRealmFromPrincipal = true
+```
+
+And you want to include the jaas.conf on the command line when launching the server so it can use it can find the keytab.
+```
+-Djava.security.auth.login.config=/jaas/zk_jaas.conf
+```
+
+#### Gateways
+
+Ideally the end user will only need to run kinit before interacting with storm.  To make this happen seamlessly we need the default jaas.conf on the gateways to be something like
+
+```
+StormClient {
+   com.sun.security.auth.module.Krb5LoginModule required
+   doNotPrompt=false
+   useTicketCache=true
+   serviceName="$nimbus_user";
+};
+```
+
+The end user can override this if they have a headless user that has a keytab.
+
+### Authorization Setup
+
+*Authentication* does the job of verifying who the user is, but we also need *authorization* to do the job of enforcing what each user can do.
+
+The preferred authorization plug-in for nimbus is The *SimpleACLAuthorizer*.  To use the *SimpleACLAuthorizer*, set the following:
+
+```yaml
+nimbus.authorizer: "backtype.storm.security.auth.authorizer.SimpleACLAuthorizer"
+```
+
+DRPC has a separate authorizer configuration for it.  Do not use SimpleACLAuthorizer for DRPC.
+
+The *SimpleACLAuthorizer* plug-in needs to know who the supervisor users are, and it needs to know about all of the administrator users, including the user running the ui daemon. 
+
+These are set through *nimbus.supervisor.users* and *nimbus.admins* respectively.  Each can either be a full Kerberos principal name, or the name of the user with host and realm stripped off.
+
+The UI and Log servers have their own authorization configurations.  These are set through *logs.users* and *ui.users*.  These should be set to the admin users for all of the nodes in the cluster.  
+
+When a topology is sumbitted, the sumbitting user can specify users in this list as well.  The users specified-in addition to the users in the cluster-wide setting-will be granted access to the submitted topology's details in the ui and/or to the topology's worker logs in the logviewers.  
+
+### Supervisors headless User and group Setup
+
+To ensure isolation of users in multi-tenancy, there is need to run supervisors and headless user and group unique to execution on the supervisor nodes.  To enable this follow below steps.
+1. Add headlessuser to all supervisor hosts.
+2. Create unique group and make it the primary group for the headless user on the supervisor nodes.
+3. The set following properties on storm for these supervisor nodes.
+
+### Multi-tenant Scheduler
+
+To support multi-tenancy better we have written a new scheduler.  To enable this scheduler set.
+```yaml
+storm.scheduler: "backtype.storm.scheduler.multitenant.MultitenantScheduler"
+```
+Be aware that many of the features of this scheduler rely on storm authentication.  Without them the scheduler will not know what the user is and will not isolate topologies properly.
+
+The goal of the multi-tenant scheduler is to provide a way to isolate topologies from one another, but to also limit the resources that an individual user can have in the cluster.
+
+The scheduler currently has one config that can be set either through =storm.yaml= or through a separate config file called =multitenant-scheduler.yaml= that should be placed in the same directory as =storm.yaml=.  It is preferable to use =multitenant-scheduler.yaml= because it can be updated without needing to restart nimbus.
+
+There is currently only one config in =multitenant-scheduler.yaml=, =multitenant.scheduler.user.pools= is a map from the user name, to the maximum number of nodes that user is guaranteed to be able to use for their topologies.
+
+For example:
+
+```yaml
+multitenant.scheduler.user.pools: 
+    "evans": 10
+    "derek": 10
+```
+
+### Run as User
+By default storm runs workers as the user that is running the supervisor.  This is not ideal for security.  To make storm run the topologies as the user that launched them set.
+
+```yaml
+supervisor.run.worker.as.user: true
+```
+
+There are several files that go along with this that are needed to be configured properly to make storm secure.
+
+The worker-launcher executable is a special program that allows the supervisor to launch workers as different users.  For this to work it needs to be owned by root, but with the group set to be a group that only teh supervisor headless user is a part of.
+It also needs to have 6550 permissions.
+There is also a worker-launcher.cfg file, usually located under /etc/ that should look somethign like the following
+
+```
+storm.worker-launcher.group=$(worker_launcher_group)
+min.user.id=$(min_user_id)
+```
+where worker_launcher_group is the same group the supervisor is a part of, and min.user.id is set to the first real user id on the system.
+This config file also needs to be owned by root and not have world or group write permissions.
+
+### Automatic Credentials Push and Renewal
+Individual topologies have the ability to push credentials (tickets and tokens) to workers so that they can access secure services.  Exposing this to all of the users can be a pain for them.
+To hide this from them in the common case plugins can be used to populate the credentials, unpack them on the other side into a java Subject, and also allow Nimbus to renew the credentials if needed.
+These are controlled by the following configs. topology.auto-credentials is a list of java plugins that populate the credentials and unpack them on the worker side.
+On a kerberos secure cluster they should be set by default to point to backtype.storm.security.auth.kerberos.AutoTGT.  nimbus.credential.renewers.classes should also be set to this value so that nimbus can periodically renew the TGT on behalf of the user.
+
+nimbus.credential.renewers.freq.secs controls how often the renewer will poll to see if anything needs to be renewed, but the default should be fine.
+
+### Limits
+By default storm allows any sized topology to be submitted. But ZK and others have limitations on how big a topology can actually be.  The following configs allow you to limit the maximum size a topology can be.
+
+| YAML Setting | Description |
+|------------|----------------------|
+| nimbus.slots.perTopology | The maximum number of slots/workers a topology can use. |
+| nimbus.executors.perTopology | The maximum number of executors/threads a topology can use. |
+
+### Log Cleanup
+The Logviewer deamon now is also responsible for cleaning up old log files for dead topologies.
+
+| YAML Setting | Description |
+|--------------|-------------------------------------|
+| logviewer.cleanup.age.mins | How old (by last modification time) must a worker's log be before that log is considered for clean-up. (Living workers' logs are never cleaned up by the logviewer: Their logs are rolled via logback.) |
+| logviewer.cleanup.interval.secs | Interval of time in seconds that the logviewer cleans up worker logs. |
+
 
 ### DRPC
+Hopefully more on this soon
 
-Each DRPC server has two different ports.  The invocations port is accessed by worker
-processes within the cluster.  The other port is accessed by external clients that
-want to query the topology.  The external port should be restricted to hosts that you
-want to be able to do queries.
 
-### Supervisors
-
-Supervisors are only clients they are not servers, and as such don't need special restrictions.
-
-### Workers
-
-Worker processes receive data from each other.  There is the option to encrypt this data using
-Blowfish by setting `topology.tuple.serializer` to `backtype.storm.security.serialization.BlowfishTupleSerializer`
-and setting `topology.tuple.serializer.blowfish.key` to a secret key you want your topology to use.
-
-### Zookeeper
-
-Zookeeper uses other ports for communications within the ensemble the details of which
-are beyond the scope of this document.  You should look at restricting Zookeeper access
-as well, because storm does not set up any ACLs for the data it write to Zookeeper.
 
   
diff --git a/bin/storm b/bin/storm
index c9885a2..d657aa7 100755
--- a/bin/storm
+++ b/bin/storm
@@ -190,6 +190,18 @@
         jvmtype="-client", 
         extrajars=[USER_CONF_DIR, STORM_DIR + "/bin"])
 
+
+def upload_credentials(*args):
+    """Syntax: [storm upload_credentials topology-name [credkey credvalue]*]
+
+    Uploads a new set of credentials to a running topology
+    """
+    exec_storm_class(
+        "backtype.storm.command.upload_credentials", 
+        args=args, 
+        jvmtype="-client", 
+        extrajars=[USER_CONF_DIR, STORM_DIR + "/bin"])
+
 def activate(*args):
     """Syntax: [storm activate topology-name]
 
@@ -431,7 +443,8 @@
             "drpc": drpc, "supervisor": supervisor, "localconfvalue": print_localconfvalue,
             "remoteconfvalue": print_remoteconfvalue, "repl": repl, "classpath": print_classpath,
             "activate": activate, "deactivate": deactivate, "rebalance": rebalance, "help": print_usage,
-            "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version}
+            "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version, 
+            "upload-credentials": upload_credentials}
 
 def parse_config(config_list):
     global CONFIG_OPTS
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 39c4b92..7f17054 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -32,14 +32,25 @@
 storm.zookeeper.retry.times: 5
 storm.zookeeper.retry.interval: 1000
 storm.zookeeper.retry.intervalceiling.millis: 30000
+storm.zookeeper.auth.user: null
+storm.zookeeper.auth.password: null
 storm.cluster.mode: "distributed" # can be distributed or local
 storm.local.mode.zmq: false
 storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin"
+storm.principal.tolocal: "backtype.storm.security.auth.DefaultPrincipalToLocal"
 storm.messaging.transport: "backtype.storm.messaging.netty.Context"
+storm.nimbus.retry.times: 5
+storm.nimbus.retry.interval.millis: 2000
+storm.nimbus.retry.intervalceiling.millis: 60000
+storm.auth.simple-white-list.users: []
+storm.auth.simple-acl.users: []
+storm.auth.simple-acl.users.commands: []
+storm.auth.simple-acl.admins: []
 
 ### nimbus.* configs are for the master
 nimbus.host: "localhost"
 nimbus.thrift.port: 6627
+nimbus.thrift.threads: 64
 nimbus.thrift.max_buffer_size: 1048576
 nimbus.childopts: "-Xmx1024m"
 nimbus.task.timeout.secs: 30
@@ -51,22 +62,40 @@
 nimbus.reassign: true
 nimbus.file.copy.expiration.secs: 600
 nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator"
+nimbus.credential.renewers.freq.secs: 600
 
 ### ui.* configs are for the master
 ui.port: 8080
 ui.childopts: "-Xmx768m"
+ui.actions.enabled: true
+ui.filter: null
+ui.filter.params: null
+ui.users: null
+ui.header.buffer.bytes: 4096
+ui.http.creds.plugin: backtype.storm.security.auth.DefaultHttpCredentialsPlugin
 
 logviewer.port: 8000
 logviewer.childopts: "-Xmx128m"
+logviewer.cleanup.age.mins: 10080
 logviewer.appender.name: "A1"
 
+logs.users: null
 
 drpc.port: 3772
 drpc.worker.threads: 64
+drpc.max_buffer_size: 1048576
 drpc.queue.size: 128
 drpc.invocations.port: 3773
+drpc.invocations.threads: 64
 drpc.request.timeout.secs: 600
 drpc.childopts: "-Xmx768m"
+drpc.http.port: 3774
+drpc.https.port: -1
+drpc.https.keystore.password: ""
+drpc.https.keystore.type: "JKS"
+drpc.http.creds.plugin: backtype.storm.security.auth.DefaultHttpCredentialsPlugin
+drpc.authorizer.acl.filename: "drpc-auth-acl.yaml"
+drpc.authorizer.acl.strict: false
 
 transactional.zookeeper.root: "/transactional"
 transactional.zookeeper.servers: null
@@ -80,6 +109,7 @@
     - 6702
     - 6703
 supervisor.childopts: "-Xmx256m"
+supervisor.run.worker.as.user: false
 #how long supervisor will wait to ensure that a worker process is started
 supervisor.worker.start.timeout.secs: 120
 #how long between heartbeats until supervisor considers that worker dead and tries to restart it
@@ -89,9 +119,13 @@
 #how frequently the supervisor heartbeats to the cluster state (for nimbus)
 supervisor.heartbeat.frequency.secs: 5
 supervisor.enable: true
+supervisor.supervisors: []
+supervisor.supervisors.commands: []
+
 
 ### worker.* configs are for task workers
 worker.childopts: "-Xmx768m"
+worker.gc.childopts: ""
 worker.heartbeat.frequency.secs: 1
 
 # control how many worker receiver threads we need per worker 
@@ -99,6 +133,7 @@
 
 task.heartbeat.frequency.secs: 3
 task.refresh.poll.secs: 10
+task.credentials.poll.secs: 30
 
 zmq.threads: 1
 zmq.linger.millis: 5000
@@ -149,5 +184,6 @@
 topology.kryo.factory: "backtype.storm.serialization.DefaultKryoFactory"
 topology.tuple.serializer: "backtype.storm.serialization.types.ListDelegateSerializer"
 topology.trident.batch.emit.interval.millis: 500
+topology.testing.always.try.serialize: false
 
 dev.zookeeper.path: "/tmp/dev-storm-zookeeper"
diff --git a/conf/jaas_digest.conf b/conf/jaas_digest.conf
index 06dd7aa..301f3e0 100644
--- a/conf/jaas_digest.conf
+++ b/conf/jaas_digest.conf
@@ -17,21 +17,17 @@
  */
 
 /* This is example of JAAS Login configuration for digest authentication
-*/
 
-/* 
 StormServer section should contain a list of authorized users and their passwords. 
+StormClient section contains one user name and his/her password.
 */
 StormServer {
        org.apache.zookeeper.server.auth.DigestLoginModule required
        user_super="adminsecret"
-       user_bob="bobsecret";
+       user_bob="bobsecret"
        user_john="johnsecret";
 };
 
-/*
-StormClient section contains one user name and his/her password.
-*/
 StormClient {
        org.apache.zookeeper.server.auth.DigestLoginModule required
        username="bob"
diff --git a/conf/jaas_kerberos.conf b/conf/jaas_kerberos.conf
new file mode 100644
index 0000000..5861df2
--- /dev/null
+++ b/conf/jaas_kerberos.conf
@@ -0,0 +1,15 @@
+StormServer {
+       com.sun.security.auth.module.Krb5LoginModule required
+       useKeyTab=true
+       keyTab="/etc/nimbus_server.keytab"
+       storeKey=true
+       useTicketCache=false
+       principal="nimbus_server/carcloth.corp.yahoo.com@STORM.CORP.YAHOO.COM";
+};
+StormClient {
+       com.sun.security.auth.module.Krb5LoginModule required
+       doNotPrompt=true
+       useTicketCache=true
+       serviceName="nimbus_server";
+};
+
diff --git a/pom.xml b/pom.xml
index cba38e3..19bfa8c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -206,6 +206,9 @@
         <reply.version>0.3.0</reply.version>
         <conjure.version>2.1.3</conjure.version>
         <zookeeper.version>3.4.5</zookeeper.version>
+        <conjure.version>2.1.3</conjure.version>
+        <clojure-data-codec.version>0.1.0</clojure-data-codec.version>
+        <clojure-contrib.version>1.2.0</clojure-contrib.version>
 
     </properties>
 
@@ -442,6 +445,23 @@
                 <artifactId>netty</artifactId>
                 <version>${netty.version}</version>
             </dependency>
+           <dependency>
+               <groupId>org.clojars.runa</groupId>
+               <artifactId>conjure</artifactId>
+               <version>${conjure.version}</version>
+               <scope>test</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.clojure</groupId>
+                <artifactId>clojure-contrib</artifactId>
+                <version>${clojure-contrib.version}</version>
+                <scope>test</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.clojure</groupId>
+                <artifactId>data.codec</artifactId>
+                <version>${clojure-data-codec.version}</version>
+            </dependency>
             <dependency>
                 <groupId>org.clojure</groupId>
                 <artifactId>tools.nrepl</artifactId>
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 34118fe..4272c60 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -28,6 +28,11 @@
     <name>Storm Core</name>
     <description>Storm Core Java API and Clojure implementation.</description>
 
+    <properties>
+        <worker-launcher.conf.dir>/etc/storm</worker-launcher.conf.dir>
+        <worker-launcher.additional_cflags></worker-launcher.additional_cflags>
+    </properties>
+
     <dependencies>
         <!--clojure-->
         <dependency>
@@ -76,7 +81,22 @@
             <artifactId>clojure-complete</artifactId>
             <scope>test</scope>
         </dependency>
-
+        <dependency>
+            <groupId>org.clojars.runa</groupId>
+            <artifactId>conjure</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.clojure</groupId>
+            <artifactId>clojure-contrib</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.clojure</groupId>
+            <artifactId>data.codec</artifactId>
+            <scope>test</scope>
+        </dependency>
+ 
         <!--java-->
         <dependency>
             <groupId>commons-io</groupId>
@@ -245,7 +265,8 @@
                             <goal>test-with-junit</goal>
                         </goals>
                         <configuration>
-                            <vmargs>${test.extra.args}</vmargs>
+                            <!-- argLine is set by JaCoCo for code coverage -->
+                            <vmargs>${argLine} ${test.extra.args}</vmargs>
                         </configuration>
                     </execution>
                 </executions>
@@ -310,7 +331,204 @@
                     </dependency>
                 </dependencies>
             </plugin>
-
+            <plugin>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <version>2.6</version>
+                <executions>
+                    <execution>
+                        <id>copy-dependencies</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>copy-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <overWriteReleases>false</overWriteReleases>
+                            <overWriteSnapshots>false</overWriteSnapshots>
+                            <overWriteIfNewer>true</overWriteIfNewer>
+                            <includeScope>runtime</includeScope>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-source-plugin</artifactId>
+                <version>2.2.1</version>
+                <executions>
+                    <execution>
+                        <id>attach-sources</id>
+                        <goals>
+                            <!-- avoid warning about recursion -->
+                            <goal>jar-no-fork</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
+
+    <profiles>
+        <profile>
+            <id>coverage</id>
+            <activation><activeByDefault>true</activeByDefault></activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.7</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources><!-- Allows JaCoCo to find the clojure source code -->
+                                        <source>src/clj</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <version>1.7</version>
+                        <executions>
+                            <execution>
+                                <id>pre-test-jacoco-clean</id>
+                                <phase>process-test-classes</phase>
+                                <configuration>
+                                    <tasks>
+                                        <delete file="target\jacoco.exec" />
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.jacoco</groupId>
+                        <artifactId>jacoco-maven-plugin</artifactId>
+                        <version>0.6.1.201212231917</version>
+                        <executions>
+                            <execution>
+                                <id>prepare-agent</id>
+                                <goals>
+                                    <goal>prepare-agent</goal>
+                                </goals>
+                                <configuration>
+                                    <append>true</append>
+                                    <excludes>
+                                        <exclude>backtype/storm/metric/api/IMetricsConsumer$DataPointFieldAccess</exclude>
+                                        <exclude>backtype/storm/metric/api/IMetricsConsumer$TaskInfoFieldAccess</exclude>
+                                        <exclude>backtype/storm/testing/TestSerObjectFieldAccess</exclude>
+                                    </excludes>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>report</id>
+                                <phase>prepare-package</phase>
+                                <goals>
+                                    <goal>report</goal>
+                                </goals>
+                                <configuration>
+                                    <excludes>
+                                        <exclude>backtype/storm/generated/*</exclude> <!--Thrift generated code-->
+                                    </excludes>
+                                    <includes>
+                                        <include>backtype/*/*/*/*</include>
+                                        <include>backtype/*/*/*</include>
+                                        <include>backtype/*/*</include>
+                                        <include>backtype/*</include>
+                                        <include>zilch/*</include>
+                                        <include>storm/*/*/*/*</include>
+                                        <include>storm/*/*/*</include>
+                                        <include>storm/*/*</include>
+                                        <include>storm/*</include>
+                                    </includes>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+        <profile>
+            <id>native</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <version>1.2.1</version>
+                        <executions>
+                            <execution>
+                                <phase>generate-sources</phase>
+                                <goals><goal>exec</goal></goals>
+                            </execution>
+                        </executions>
+                        <configuration>
+                            <executable>sh</executable>
+                            <arguments>
+                                <argument>-c</argument>
+                                <argument>mkdir -p ${project.build.directory}/; cp -rufv ${basedir}/src/native/ ${project.build.directory}/</argument>
+                            </arguments>
+                        </configuration>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>make-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>compile</id>
+                                <phase>compile</phase>
+                                <goals>
+                                    <goal>autoreconf</goal>
+                                    <goal>configure</goal>
+                                    <goal>make-install</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>test</id>
+                                <phase>test</phase>
+                                <goals>
+                                    <goal>test</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                        <configuration>
+                            <!-- autoreconf settings -->
+                            <workDir>${project.build.directory}/native/worker-launcher</workDir>
+                            <arguments>
+                                <argument>-i</argument>
+                            </arguments>
+
+                            <!-- configure settings -->
+                            <configureEnvironment>
+                                <property>
+                                    <name>CFLAGS</name>
+                                    <value>-DEXEC_CONF_DIR=${worker-launcher.conf.dir} ${worker-launcher.additional_cflags}</value>
+                                </property>
+                            </configureEnvironment>
+                            <configureWorkDir>${project.build.directory}/native/worker-launcher</configureWorkDir>
+                            <prefix>/usr/local</prefix>
+
+                            <!-- configure & make settings -->
+                            <destDir>${project.build.directory}/native/target</destDir>
+
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+            <activation>
+                <activeByDefault>false</activeByDefault>
+            </activation>
+        </profile>
+    </profiles>
+
 </project>
diff --git a/storm-core/src/clj/backtype/storm/LocalCluster.clj b/storm-core/src/clj/backtype/storm/LocalCluster.clj
index dc8214d..c9b5a3b 100644
--- a/storm-core/src/clj/backtype/storm/LocalCluster.clj
+++ b/storm-core/src/clj/backtype/storm/LocalCluster.clj
@@ -42,6 +42,10 @@
   (submit-local-topology-with-opts
     (:nimbus (. this state)) name conf topology submit-opts))
 
+(defn -uploadNewCredentials
+  [this name creds]
+  (.uploadNewCredentials (:nimbus (. this state)) name creds))
+
 (defn -shutdown
   [this]
   (kill-local-storm-cluster (. this state)))
diff --git a/storm-core/src/clj/backtype/storm/LocalDRPC.clj b/storm-core/src/clj/backtype/storm/LocalDRPC.clj
index daead2e..9773821 100644
--- a/storm-core/src/clj/backtype/storm/LocalDRPC.clj
+++ b/storm-core/src/clj/backtype/storm/LocalDRPC.clj
@@ -16,7 +16,7 @@
 
 (ns backtype.storm.LocalDRPC
   (:require [backtype.storm.daemon [drpc :as drpc]])
-  (:use [backtype.storm util])
+  (:use [backtype.storm config util])
   (:import [backtype.storm.utils InprocMessaging ServiceRegistry])
   (:gen-class
    :init init
@@ -25,7 +25,7 @@
    :state state ))
 
 (defn -init []
-  (let [handler (drpc/service-handler)
+  (let [handler (drpc/service-handler (read-storm-config))
         id (ServiceRegistry/registerService handler)
         ]
     [[] {:service-id id :handler handler}]
diff --git a/storm-core/src/clj/backtype/storm/bootstrap.clj b/storm-core/src/clj/backtype/storm/bootstrap.clj
index 3ee6f76..c1063cf 100644
--- a/storm-core/src/clj/backtype/storm/bootstrap.clj
+++ b/storm-core/src/clj/backtype/storm/bootstrap.clj
@@ -25,10 +25,11 @@
      (import (quote [backtype.storm.utils Utils LocalState Time TimeCacheMap
                      TimeCacheMap$ExpiredCallback
                      RotatingMap RotatingMap$ExpiredCallback
-                     BufferFileInputStream
+                     BufferFileInputStream ZookeeperServerCnxnFactory
                      RegisteredGlobalState ThriftTopologyUtils DisruptorQueue
                      MutableObject MutableLong]))
      (import (quote [backtype.storm.serialization KryoTupleSerializer KryoTupleDeserializer]))
+     (import (quote [backtype.storm.security.auth ThriftServer ThriftClient ReqContext]))
      (import (quote [backtype.storm.spout ISpout SpoutOutputCollector ISpoutOutputCollector ShellSpout]))
      (import (quote [backtype.storm.tuple Tuple TupleImpl Fields MessageId]))
      (import (quote [backtype.storm.task IBolt IOutputCollector
@@ -54,7 +55,7 @@
                      TopologySummary ExecutorSummary ExecutorStats ExecutorSpecificStats
                      SpoutStats BoltStats ErrorInfo SupervisorSummary ExecutorInfo
                      KillOptions SubmitOptions RebalanceOptions JavaObject JavaObjectArg
-                     TopologyInitialStatus]))
+                     TopologyInitialStatus AuthorizationException]))
      (import (quote [backtype.storm.daemon.common StormBase Assignment
                      SupervisorInfo WorkerHeartbeat]))
      (import (quote [backtype.storm.grouping CustomStreamGrouping]))
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index e370b97..baa4f69 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -15,37 +15,47 @@
 ;; limitations under the License.
 
 (ns backtype.storm.cluster
-  (:import [org.apache.zookeeper.data Stat])
-  (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException])
+  (:import [org.apache.zookeeper.data Stat ACL Id])
+  (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
   (:import [backtype.storm.utils Utils])
+  (:import [java.security MessageDigest])
+  (:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
   (:use [backtype.storm util log config])
   (:require [backtype.storm [zookeeper :as zk]])
   (:require [backtype.storm.daemon [common :as common]]))
 
 (defprotocol ClusterState
-  (set-ephemeral-node [this path data])
+  (set-ephemeral-node [this path data acls])
   (delete-node [this path])
-  (create-sequential [this path data])
+  (create-sequential [this path data acls])
   ;; if node does not exist, create persistent with this data
-  (set-data [this path data])
+  (set-data [this path data acls])
   (get-data [this path watch?])
   (get-children [this path watch?])
-  (mkdirs [this path])
+  (mkdirs [this path acls])
+  (exists-node? [this path watch?])
   (close [this])
   (register [this callback])
   (unregister [this id]))
 
-(defn mk-distributed-cluster-state
-  [conf]
-  (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf conf)]
-    (zk/mkdirs zk (conf STORM-ZOOKEEPER-ROOT))
+(defn mk-topo-only-acls
+  [topo-conf]
+  (let [payload (.get topo-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)]
+    (when (Utils/isZkAuthenticationConfiguredTopology topo-conf)
+      [(first ZooDefs$Ids/CREATOR_ALL_ACL)
+       (ACL. ZooDefs$Perms/READ (Id. "digest" (DigestAuthenticationProvider/generateDigest payload)))])))
+
+(defnk mk-distributed-cluster-state
+  [conf :auth-conf nil :acls nil]
+  (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf auth-conf)]
+    (zk/mkdirs zk (conf STORM-ZOOKEEPER-ROOT) acls)
     (.close zk))
   (let [callbacks (atom {})
         active (atom true)
         zk (zk/mk-client conf
                          (conf STORM-ZOOKEEPER-SERVERS)
                          (conf STORM-ZOOKEEPER-PORT)
-                         :auth-conf conf
+                         :auth-conf auth-conf
                          :root (conf STORM-ZOOKEEPER-ROOT)
                          :watcher (fn [state type path]
                                     (when @active
@@ -68,29 +78,28 @@
        (swap! callbacks dissoc id))
 
      (set-ephemeral-node
-       [this path data]
-       (zk/mkdirs zk (parent-path path))
+       [this path data acls]
+       (zk/mkdirs zk (parent-path path) acls)
        (if (zk/exists zk path false)
          (try-cause
            (zk/set-data zk path data) ; should verify that it's ephemeral
            (catch KeeperException$NoNodeException e
              (log-warn-error e "Ephemeral node disappeared between checking for existing and setting data")
-             (zk/create-node zk path data :ephemeral)
-             ))
-         (zk/create-node zk path data :ephemeral)))
+             (zk/create-node zk path data :ephemeral acls)))
+         (zk/create-node zk path data :ephemeral acls)))
 
      (create-sequential
-       [this path data]
-       (zk/create-node zk path data :sequential))
+       [this path data acls]
+       (zk/create-node zk path data :sequential acls))
 
      (set-data
-       [this path data]
+       [this path data acls]
        ;; note: this does not turn off any existing watches
        (if (zk/exists zk path false)
          (zk/set-data zk path data)
          (do
-           (zk/mkdirs zk (parent-path path))
-           (zk/create-node zk path data :persistent))))
+           (zk/mkdirs zk (parent-path path) acls)
+           (zk/create-node zk path data :persistent acls))))
 
      (delete-node
        [this path]
@@ -105,8 +114,12 @@
        (zk/get-children zk path watch?))
 
      (mkdirs
-       [this path]
-       (zk/mkdirs zk path))
+       [this path acls]
+       (zk/mkdirs zk path acls))
+
+     (exists-node?
+       [this path watch?]
+       (zk/exists-node? zk path watch?))
 
      (close
        [this]
@@ -137,6 +150,8 @@
   (remove-storm! [this storm-id])
   (report-error [this storm-id task-id error])
   (errors [this storm-id task-id])
+  (set-credentials! [this storm-id creds topo-conf])
+  (credentials [this storm-id callback])
   (disconnect [this]))
 
 (def ASSIGNMENTS-ROOT "assignments")
@@ -145,12 +160,14 @@
 (def SUPERVISORS-ROOT "supervisors")
 (def WORKERBEATS-ROOT "workerbeats")
 (def ERRORS-ROOT "errors")
+(def CREDENTIALS-ROOT "credentials")
 
 (def ASSIGNMENTS-SUBTREE (str "/" ASSIGNMENTS-ROOT))
 (def STORMS-SUBTREE (str "/" STORMS-ROOT))
 (def SUPERVISORS-SUBTREE (str "/" SUPERVISORS-ROOT))
 (def WORKERBEATS-SUBTREE (str "/" WORKERBEATS-ROOT))
 (def ERRORS-SUBTREE (str "/" ERRORS-ROOT))
+(def CREDENTIALS-SUBTREE (str "/" CREDENTIALS-ROOT))
 
 (defn supervisor-path
   [id]
@@ -180,6 +197,10 @@
   [storm-id component-id]
   (str (error-storm-root storm-id) "/" (url-encode component-id)))
 
+(defn credentials-path
+  [storm-id]
+  (str CREDENTIALS-SUBTREE "/" storm-id))
+
 (defn- issue-callback!
   [cb-atom]
   (let [cb @cb-atom]
@@ -219,29 +240,31 @@
          (into {}))))
 
 ;; Watches should be used for optimization. When ZK is reconnecting, they're not guaranteed to be called.
-(defn mk-storm-cluster-state
-  [cluster-state-spec]
+(defnk mk-storm-cluster-state
+  [cluster-state-spec :acls nil]
   (let [[solo? cluster-state] (if (satisfies? ClusterState cluster-state-spec)
                                 [false cluster-state-spec]
-                                [true (mk-distributed-cluster-state cluster-state-spec)])
+                                [true (mk-distributed-cluster-state cluster-state-spec :auth-conf cluster-state-spec :acls acls)])
         assignment-info-callback (atom {})
         supervisors-callback (atom nil)
         assignments-callback (atom nil)
         storm-base-callback (atom {})
+        credentials-callback (atom {})
         state-id (register
-                   cluster-state
-                   (fn [type path]
-                     (let [[subtree & args] (tokenize-path path)]
-                       (condp = subtree
+                  cluster-state
+                  (fn [type path]
+                    (let [[subtree & args] (tokenize-path path)]
+                      (condp = subtree
                          ASSIGNMENTS-ROOT (if (empty? args)
-                                            (issue-callback! assignments-callback)
-                                            (issue-map-callback! assignment-info-callback (first args)))
+                                             (issue-callback! assignments-callback)
+                                             (issue-map-callback! assignment-info-callback (first args)))
                          SUPERVISORS-ROOT (issue-callback! supervisors-callback)
                          STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
+                         CREDENTIALS-ROOT (issue-map-callback! credentials-callback (first args))
                          ;; this should never happen
                          (halt-process! 30 "Unknown callback for subtree " subtree args)))))]
     (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE]]
-      (mkdirs cluster-state p))
+      (mkdirs cluster-state p acls))
     (reify
       StormClusterState
 
@@ -300,7 +323,7 @@
 
       (worker-heartbeat!
         [this storm-id node port info]
-        (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize info)))
+        (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize info) acls))
 
       (remove-worker-heartbeat!
         [this storm-id node port]
@@ -308,7 +331,7 @@
 
       (setup-heartbeats!
         [this storm-id]
-        (mkdirs cluster-state (workerbeat-storm-root storm-id)))
+        (mkdirs cluster-state (workerbeat-storm-root storm-id) acls))
 
       (teardown-heartbeats!
         [this storm-id]
@@ -326,11 +349,11 @@
 
       (supervisor-heartbeat!
         [this supervisor-id info]
-        (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize info)))
+        (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize info) acls))
 
       (activate-storm!
         [this storm-id storm-base]
-        (set-data cluster-state (storm-path storm-id) (Utils/serialize storm-base)))
+        (set-data cluster-state (storm-path storm-id) (Utils/serialize storm-base) acls))
 
       (update-storm!
         [this storm-id new-elems]
@@ -340,7 +363,8 @@
           (set-data cluster-state (storm-path storm-id)
                     (-> base
                         (merge new-elems)
-                        Utils/serialize))))
+                        Utils/serialize)
+                    acls)))
 
       (storm-base
         [this storm-id callback]
@@ -354,43 +378,56 @@
 
       (set-assignment!
         [this storm-id info]
-        (set-data cluster-state (assignment-path storm-id) (Utils/serialize info)))
+        (set-data cluster-state (assignment-path storm-id) (Utils/serialize info) acls))
 
       (remove-storm!
         [this storm-id]
         (delete-node cluster-state (assignment-path storm-id))
+        (delete-node cluster-state (credentials-path storm-id))
         (remove-storm-base! this storm-id))
 
+      (set-credentials!
+         [this storm-id creds topo-conf]
+         (let [topo-acls (mk-topo-only-acls topo-conf)
+               path (credentials-path storm-id)]
+           (set-data cluster-state path (Utils/serialize creds) topo-acls)))
+
+      (credentials
+        [this storm-id callback]
+        (when callback
+          (swap! credentials-callback assoc storm-id callback))
+        (maybe-deserialize (get-data cluster-state (credentials-path storm-id) (not-nil? callback))))
+
       (report-error
-        [this storm-id component-id error]
-        (let [path (error-path storm-id component-id)
-              data {:time-secs (current-time-secs) :error (stringify-error error)}
-              _ (mkdirs cluster-state path)
-              _ (create-sequential cluster-state (str path "/e") (Utils/serialize data))
-              to-kill (->> (get-children cluster-state path false)
-                           (sort-by parse-error-path)
-                           reverse
-                           (drop 10))]
-          (doseq [k to-kill]
-            (delete-node cluster-state (str path "/" k)))))
+         [this storm-id component-id error]                
+         (let [path (error-path storm-id component-id)
+               data {:time-secs (current-time-secs) :error (stringify-error error)}
+               _ (mkdirs cluster-state path acls)
+               _ (create-sequential cluster-state (str path "/e") (Utils/serialize data) acls)
+               to-kill (->> (get-children cluster-state path false)
+                            (sort-by parse-error-path)
+                            reverse
+                            (drop 10))]
+           (doseq [k to-kill]
+             (delete-node cluster-state (str path "/" k)))))
 
       (errors
-        [this storm-id component-id]
-        (let [path (error-path storm-id component-id)
-              _ (mkdirs cluster-state path)
-              children (get-children cluster-state path false)
-              errors (dofor [c children]
-                            (let [data (-> (get-data cluster-state (str path "/" c) false)
-                                           maybe-deserialize)]
-                              (when data
-                                (struct TaskError (:error data) (:time-secs data))
-                                )))
-              ]
-          (->> (filter not-nil? errors)
-               (sort-by (comp - :time-secs)))))
-
+         [this storm-id component-id]
+         (let [path (error-path storm-id component-id)
+               errors (if (exists-node? cluster-state path false)
+                        (dofor [c (get-children cluster-state path false)]
+                          (let [data (-> (get-data cluster-state (str path "/" c) false)
+                                         maybe-deserialize)]
+                            (when data
+                              (struct TaskError (:error data) (:time-secs data))
+                              )))
+                        ())
+               ]
+           (->> (filter not-nil? errors)
+                (sort-by (comp - :time-secs)))))
+      
       (disconnect
-        [this]
+         [this]
         (unregister cluster-state state-id)
         (when solo?
           (close cluster-state))))))
diff --git a/storm-core/src/clj/backtype/storm/command/upload_credentials.clj b/storm-core/src/clj/backtype/storm/command/upload_credentials.clj
new file mode 100644
index 0000000..05a82cb
--- /dev/null
+++ b/storm-core/src/clj/backtype/storm/command/upload_credentials.clj
@@ -0,0 +1,35 @@
+;; 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.
+(ns backtype.storm.command.upload-credentials
+  (:use [clojure.tools.cli :only [cli]])
+  (:use [backtype.storm log util])
+  (:import [backtype.storm StormSubmitter])
+  (:import [java.util Properties])
+  (:import [java.io FileReader])
+  (:gen-class))
+
+(defn read-map [file-name]
+  (let [props (Properties. )
+        _ (.load props (FileReader. file-name))]
+    (clojurify-structure props)))
+
+(defn -main [& args]
+  (let [[{cred-file :file} [name & rawCreds]] (cli args ["-f" "--file" :default nil])
+        _ (when (and rawCreds (not (even? (.size rawCreds)))) (throw (RuntimeException.  "Need an even number of arguments to make a map")))
+        mapping (if rawCreds (apply assoc {} rawCreds) {})
+        file-mapping (if (nil? cred-file) {} (read-map cred-file))]
+      (StormSubmitter/pushCredentials name {} (merge file-mapping mapping))
+      (log-message "Uploaded new creds to topology: " name)))
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index 14beb21..98b1da2 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -15,12 +15,12 @@
 ;; limitations under the License.
 
 (ns backtype.storm.config
-  (:import [java.io FileReader File])
+  (:import [java.io FileReader File IOException])
   (:import [backtype.storm Config ConfigValidation$FieldValidator])
   (:import [backtype.storm.utils Utils LocalState])
   (:import [org.apache.commons.io FileUtils])
   (:require [clojure [string :as str]])
-  (:use [backtype.storm util]))
+  (:use [backtype.storm log util]))
 
 (def RESOURCES-SUBDIR "resources")
 
@@ -123,10 +123,12 @@
     conf))
 
 (defn read-yaml-config
-  [name]
-  (let [conf (clojurify-structure (Utils/findAndReadConfigFile name true))]
-    (validate-configs-with-schemas conf)
-    conf))
+  ([name must-exist]
+     (let [conf (clojurify-structure (Utils/findAndReadConfigFile name must-exist))]
+       (validate-configs-with-schemas conf)
+       conf))
+  ([name]
+     (read-yaml-config true)))
 
 (defn master-local-dir
   [conf]
@@ -219,6 +221,32 @@
     (Utils/deserialize (FileUtils/readFileToByteArray (File. topology-path)))
     ))
 
+(defn worker-user-root [conf]
+  (str (conf STORM-LOCAL-DIR) "/workers-users"))
+
+(defn worker-user-file [conf worker-id]
+  (str (worker-user-root conf) "/" worker-id))
+
+(defn get-worker-user [conf worker-id]
+  (log-message "GET worker-user " worker-id)
+  (try
+    (str/trim (slurp (worker-user-file conf worker-id)))
+  (catch IOException e
+    (log-warn-error e "Failed to get worker user for " worker-id ".")
+    nil
+    )))
+
+  
+(defn set-worker-user! [conf worker-id user]
+  (log-message "SET worker-user " worker-id " " user)
+  (let [file (worker-user-file conf worker-id)]
+    (.mkdirs (.getParentFile (File. file)))
+    (spit (worker-user-file conf worker-id) user)))
+
+(defn remove-worker-user! [conf worker-id]
+  (log-message "REMOVE worker-user " worker-id)
+  (.delete (File. (worker-user-file conf worker-id))))
+
 (defn worker-root
   ([conf]
    (str (conf STORM-LOCAL-DIR) file-path-separator "workers"))
diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj
index e6bf81a..6a99602 100644
--- a/storm-core/src/clj/backtype/storm/daemon/common.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/common.clj
@@ -21,6 +21,7 @@
   (:import [backtype.storm.task WorkerTopologyContext])
   (:import [backtype.storm Constants])
   (:import [backtype.storm.metric SystemBolt])
+  (:import [backtype.storm.security.auth IAuthorizer]) 
   (:require [clojure.set :as set])  
   (:require [backtype.storm.daemon.acker :as acker])
   (:require [backtype.storm.thrift :as thrift])
@@ -40,6 +41,7 @@
 (def SYSTEM-TICK-STREAM-ID Constants/SYSTEM_TICK_STREAM_ID)
 (def METRICS-STREAM-ID Constants/METRICS_STREAM_ID)
 (def METRICS-TICK-STREAM-ID Constants/METRICS_TICK_STREAM_ID)
+(def CREDENTIALS-CHANGED-STREAM-ID Constants/CREDENTIALS_CHANGED_STREAM_ID)
 
 ;; the task id is the virtual port
 ;; node->host is here so that tasks know who to talk to just from assignment
@@ -48,7 +50,7 @@
 
 
 ;; component->executors is a map from spout/bolt id to number of executors for that component
-(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors])
+(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors owner])
 
 (defrecord SupervisorInfo [time-secs hostname assignment-id used-ports meta scheduler-meta uptime-secs])
 
@@ -288,7 +290,8 @@
                           {}
                           (SystemBolt.)
                           {SYSTEM-TICK-STREAM-ID (thrift/output-fields ["rate_secs"])
-                           METRICS-TICK-STREAM-ID (thrift/output-fields ["interval"])}                          
+                           METRICS-TICK-STREAM-ID (thrift/output-fields ["interval"])
+                           CREDENTIALS-CHANGED-STREAM-ID (thrift/output-fields ["creds"])}
                           :p 0
                           :conf {TOPOLOGY-TASKS 0})]
     (.put_to_bolts topology SYSTEM-COMPONENT-ID system-bolt-spec)))
@@ -349,3 +352,14 @@
   (->> executor->node+port
        (mapcat (fn [[e node+port]] (for [t (executor-id->tasks e)] [t node+port])))
        (into {})))
+
+(defn mk-authorization-handler [klassname conf]
+  (let [aznClass (if klassname (Class/forName klassname))
+        aznHandler (if aznClass (.newInstance aznClass))] 
+    (if aznHandler (.prepare ^IAuthorizer aznHandler conf))
+    (log-debug "authorization class name:" klassname
+                 " class:" aznClass
+                 " handler:" aznHandler)
+    aznHandler
+  )) 
+
diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
index eb4d6d3..baba96b 100644
--- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
@@ -15,21 +15,23 @@
 ;; limitations under the License.
 
 (ns backtype.storm.daemon.drpc
-  (:import [org.apache.thrift.server THsHaServer THsHaServer$Args])
-  (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
-  (:import [org.apache.thrift.exception])
-  (:import [org.apache.thrift.transport
-            TNonblockingServerTransport TNonblockingServerSocket])
-  (:import [backtype.storm.generated
-            DistributedRPC DistributedRPC$Iface DistributedRPC$Processor
-            DRPCRequest DRPCExecutionException
-            DistributedRPCInvocations DistributedRPCInvocations$Iface
+  (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftConnectionType])
+  (:import [backtype.storm.security.auth.authorizer DRPCAuthorizerBase])
+  (:import [backtype.storm.generated DistributedRPC DistributedRPC$Iface DistributedRPC$Processor
+            DRPCRequest DRPCExecutionException DistributedRPCInvocations DistributedRPCInvocations$Iface
             DistributedRPCInvocations$Processor])
   (:import [java.util.concurrent Semaphore ConcurrentLinkedQueue
             ThreadPoolExecutor ArrayBlockingQueue TimeUnit])
   (:import [backtype.storm.daemon Shutdownable])
   (:import [java.net InetAddress])
+  (:import [backtype.storm.generated AuthorizationException])
   (:use [backtype.storm bootstrap config log])
+  (:use [backtype.storm.daemon common])
+  (:use [backtype.storm.ui helpers])
+  (:use compojure.core)
+  (:use ring.middleware.reload)
+  (:use [ring.adapter.jetty :only [run-jetty]])
+  (:require [compojure.handler :as handler])
   (:gen-class))
 
 (bootstrap)
@@ -44,9 +46,22 @@
         amap)))
   (@queues-atom function))
 
+(defn check-authorization
+  ([aclHandler mapping operation context]
+    (if aclHandler
+      (let [context (or context (ReqContext/context))]
+        (if-not (.permit aclHandler context operation mapping)
+          (let [principal (.principal context)
+                user (if principal (.getName principal) "unknown")]
+              (throw (AuthorizationException.
+                       (str "DRPC request '" operation "' for '"
+                            user "' user is not authorized"))))))))
+  ([aclHandler mapping operation]
+    (check-authorization aclHandler mapping operation (ReqContext/context))))
+
 ;; TODO: change this to use TimeCacheMap
-(defn service-handler []
-  (let [conf (read-storm-config)
+(defn service-handler [conf]
+  (let [drpc-acl-handler (mk-authorization-handler (conf DRPC-AUTHORIZER) conf)
         ctr (atom 0)
         id->sem (atom {})
         id->result (atom {})
@@ -71,10 +86,12 @@
                              (cleanup id)))
                          (timeout-check-secs)))]
     (reify DistributedRPC$Iface
-
       (^String execute
         [this ^String function ^String args]
-        (log-debug "Received DRPC request for " function " " args " at " (System/currentTimeMillis))
+        (log-debug "Received DRPC request for " function " (" args ") at " (System/currentTimeMillis))
+        (check-authorization drpc-acl-handler
+                             {DRPCAuthorizerBase/FUNCTION_NAME function}
+                             "execute")
         (let [id (str (swap! ctr (fn [v] (mod (inc v) 1000000000))))
               ^Semaphore sem (Semaphore. 0)
               req (DRPCRequest. args id)
@@ -100,21 +117,33 @@
 
       (^void result
         [this ^String id ^String result]
-        (let [^Semaphore sem (@id->sem id)]
-          (log-debug "Received result " result " for " id " at " (System/currentTimeMillis))
-          (when sem
-            (swap! id->result assoc id result)
-            (.release sem))))
+        (when-let [func (@id->function id)]
+          (check-authorization drpc-acl-handler
+                               {DRPCAuthorizerBase/FUNCTION_NAME func}
+                               "result")
+          (let [^Semaphore sem (@id->sem id)]
+            (log-debug "Received result " result " for " id " at " (System/currentTimeMillis))
+            (when sem
+              (swap! id->result assoc id result)
+              (.release sem)
+              ))))
 
       (^void failRequest
         [this ^String id]
-        (let [^Semaphore sem (@id->sem id)]
-          (when sem
-            (swap! id->result assoc id (DRPCExecutionException. "Request failed"))
-            (.release sem))))
+        (when-let [func (@id->function id)]
+          (check-authorization drpc-acl-handler
+                               {DRPCAuthorizerBase/FUNCTION_NAME func}
+                               "failRequest")
+          (let [^Semaphore sem (@id->sem id)]
+            (when sem
+              (swap! id->result assoc id (DRPCExecutionException. "Request failed"))
+              (.release sem)))))
 
       (^DRPCRequest fetchRequest
         [this ^String func]
+        (check-authorization drpc-acl-handler
+                             {DRPCAuthorizerBase/FUNCTION_NAME func}
+                             "fetchRequest")
         (let [^ConcurrentLinkedQueue queue (acquire-queue request-queues func)
               ret (.poll queue)]
           (if ret
@@ -128,43 +157,90 @@
         [this]
         (.interrupt clear-thread)))))
 
+(defn handle-request [handler]
+  (fn [request]
+    (handler request)))
+
+(defn webapp [handler http-creds-handler]
+  (->
+    (routes
+      (POST "/drpc/:func" [:as {:keys [body servlet-request]} func & m]
+        (let [args (slurp body)]
+          (if http-creds-handler
+            (.populateContext http-creds-handler (ReqContext/context)
+                              servlet-request))
+          (.execute handler func args)))
+      (POST "/drpc/:func/" [:as {:keys [body servlet-request]} func & m]
+        (let [args (slurp body)]
+          (if http-creds-handler
+            (.populateContext http-creds-handler (ReqContext/context)
+                              servlet-request))
+          (.execute handler func args)))
+      (GET "/drpc/:func/:args" [:as {:keys [servlet-request]} func args & m]
+          (if http-creds-handler
+            (.populateContext http-creds-handler (ReqContext/context)
+                              servlet-request))
+          (.execute handler func args))
+      (GET "/drpc/:func/" [:as {:keys [servlet-request]} func & m]
+          (if http-creds-handler
+            (.populateContext http-creds-handler (ReqContext/context)
+                              servlet-request))
+          (.execute handler func ""))
+      (GET "/drpc/:func" [:as {:keys [servlet-request]} func & m]
+          (if http-creds-handler
+            (.populateContext http-creds-handler (ReqContext/context)
+                              servlet-request))
+    (.execute handler func "")))
+                (wrap-reload '[backtype.storm.daemon.drpc])
+                handle-request))
+
 (defn launch-server!
   ([]
     (let [conf (read-storm-config)
           worker-threads (int (conf DRPC-WORKER-THREADS))
           queue-size (int (conf DRPC-QUEUE-SIZE))
-          service-handler (service-handler)
-
-          ;; Requests and returns need to be on separate thread pools, since
-          ;; calls to "execute" don't unblock until other thrift methods are
-          ;; called. So if 64 threads are calling execute, the server won't
-          ;; accept the result invocations that will unblock those threads.
-
-          handler-server
-          (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-PORT)))
-                            (THsHaServer$Args.)
-                            (.workerThreads 64)
-                            (.executorService
-                              (ThreadPoolExecutor.
-                                worker-threads worker-threads 60 TimeUnit/SECONDS
-                                (ArrayBlockingQueue. queue-size)))
-                            (.protocolFactory (TBinaryProtocol$Factory.))
-                            (.processor (DistributedRPC$Processor. service-handler))))
-
-          invoke-server
-          (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-INVOCATIONS-PORT)))
-                            (THsHaServer$Args.)
-                            (.workerThreads 64)
-                            (.protocolFactory (TBinaryProtocol$Factory.))
-                            (.processor
-                              (DistributedRPCInvocations$Processor. service-handler))))]
-
-      (.addShutdownHook
-        (Runtime/getRuntime)
-        (Thread. (fn [] (.stop handler-server) (.stop invoke-server))))
+          drpc-http-port (int (conf DRPC-HTTP-PORT))
+          drpc-port (int (conf DRPC-PORT))
+          drpc-service-handler (service-handler conf)
+          ;; requests and returns need to be on separate thread pools, since calls to
+          ;; "execute" don't unblock until other thrift methods are called. So if 
+          ;; 64 threads are calling execute, the server won't accept the result
+          ;; invocations that will unblock those threads
+          handler-server (when (> drpc-port 0)
+                           (ThriftServer. conf
+                             (DistributedRPC$Processor. service-handler)
+                             ThriftConnectionType/DRPC))
+          invoke-server (ThriftServer. conf
+                          (DistributedRPCInvocations$Processor. service-handler)
+                          ThriftConnectionType/DRPC_INVOCATIONS)
+          http-creds-handler (AuthUtils/GetDrpcHttpCredentialsPlugin conf)] 
+      (.addShutdownHook (Runtime/getRuntime) (Thread. (fn []
+                                                        (if handler-server (.stop handler-server))
+                                                        (.stop invoke-server))))
       (log-message "Starting Distributed RPC servers...")
       (future (.serve invoke-server))
-      (.serve handler-server))))
+      (when (> drpc-http-port 0)
+        (let [app (webapp drpc-service-handler http-creds-handler)
+              filter-class (conf DRPC-HTTP-FILTER)
+              filter-params (conf DRPC-HTTP-FILTER-PARAMS)
+              filters-confs [{:filter-class filter-class
+                              :filter-params filter-params}]
+              https-port (int (conf DRPC-HTTPS-PORT))
+              https-ks-path (conf DRPC-HTTPS-KEYSTORE-PATH)
+              https-ks-password (conf DRPC-HTTPS-KEYSTORE-PASSWORD)
+              https-ks-type (conf DRPC-HTTPS-KEYSTORE-TYPE)]
+
+          (run-jetty app
+            {:port drpc-http-port :join? false
+             :configurator (fn [server]
+                             (config-ssl server
+                                         https-port 
+                                         https-ks-path 
+                                         https-ks-password
+                                         https-ks-type)
+                             (config-filter server app filters-confs))})))
+      (when handler-server
+        (.serve handler-server)))))
 
 (defn -main []
   (launch-server!))
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index 1bbe53d..bc491d9 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -16,13 +16,15 @@
 (ns backtype.storm.daemon.executor
   (:use [backtype.storm.daemon common])
   (:use [backtype.storm bootstrap])
+  (:import [backtype.storm ICredentialsListener])
   (:import [backtype.storm.hooks ITaskHook])
   (:import [backtype.storm.tuple Tuple])
   (:import [backtype.storm.spout ISpoutWaitStrategy])
   (:import [backtype.storm.hooks.info SpoutAckInfo SpoutFailInfo
             EmitInfo BoltFailInfo BoltAckInfo BoltExecuteInfo])
-  (:import [backtype.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
+  (:import [backtype.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint])
   (:import [backtype.storm Config])
+  (:import [java.util.concurrent ConcurrentLinkedQueue])
   (:require [backtype.storm [tuple :as tuple]])
   (:require [backtype.storm.daemon [task :as task]])
   (:require [backtype.storm.daemon.builtin-metrics :as builtin-metrics]))
@@ -157,7 +159,8 @@
 
 (defprotocol RunningExecutor
   (render-stats [this])
-  (get-executor-id [this]))
+  (get-executor-id [this])
+  (credentials-changed [this creds]))
 
 (defn throttled-report-error-fn [executor]
   (let [storm-conf (:storm-conf executor)
@@ -179,9 +182,11 @@
         ))))
 
 ;; in its own function so that it can be mocked out by tracked topologies
-(defn mk-executor-transfer-fn [batch-transfer->worker]
+(defn mk-executor-transfer-fn [batch-transfer->worker storm-conf]
   (fn this
-    ([task tuple block? ^List overflow-buffer]
+    ([task tuple block? ^ConcurrentLinkedQueue overflow-buffer]
+      (when (= true (storm-conf TOPOLOGY-DEBUG))
+        (log-message "TRANSFERING tuple TASK: " task " TUPLE: " tuple))
       (if (and overflow-buffer (not (.isEmpty overflow-buffer)))
         (.add overflow-buffer [task tuple])
         (try-cause
@@ -223,9 +228,10 @@
      :shared-executor-data (HashMap.)
      :storm-active-atom (:storm-active-atom worker)
      :batch-transfer-queue batch-transfer->worker
-     :transfer-fn (mk-executor-transfer-fn batch-transfer->worker)
+     :transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
      :suicide-fn (:suicide-fn worker)
-     :storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker))
+     :storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker) 
+                                                          :acls (Utils/getWorkerACL storm-conf))
      :type executor-type
      ;; TODO: should refactor this to be part of the executor specific map (spout or bolt with :common field)
      :stats (mk-executor-stats <> (sampling-rate storm-conf))
@@ -313,7 +319,7 @@
               [[nil (TupleImpl. context [tick-time-secs] Constants/SYSTEM_TASK_ID Constants/SYSTEM_TICK_STREAM_ID)]]
               )))))))
 
-(defn mk-executor [worker executor-id]
+(defn mk-executor [worker executor-id initial-credentials]
   (let [executor-data (mk-executor-data worker executor-id)
         _ (log-message "Loading executor " (:component-id executor-data) ":" (pr-str executor-id))
         task-datas (->> executor-data
@@ -330,7 +336,7 @@
         ;; trick isn't thread-safe)
         system-threads [(start-batch-transfer->worker-handler! worker executor-data)]
         handlers (with-error-reaction report-error-and-die
-                   (mk-threads executor-data task-datas))
+                   (mk-threads executor-data task-datas initial-credentials))
         threads (concat handlers system-threads)]    
     (setup-ticks! worker executor-data)
 
@@ -342,6 +348,13 @@
         (stats/render-stats! (:stats executor-data)))
       (get-executor-id [this]
         executor-id )
+      (credentials-changed [this creds]
+        (let [receive-queue (:receive-queue executor-data)
+              context (:worker-context executor-data)]
+          (disruptor/publish
+            receive-queue
+            [[nil (TupleImpl. context [creds] Constants/SYSTEM_TASK_ID Constants/CREDENTIALS_CHANGED_STREAM_ID)]]
+              )))
       Shutdownable
       (shutdown
         [this]
@@ -362,23 +375,25 @@
         (log-message "Shut down executor " component-id ":" (pr-str executor-id)))
         )))
 
-(defn- fail-spout-msg [executor-data task-data msg-id tuple-info time-delta]
+(defn- fail-spout-msg [executor-data task-data msg-id tuple-info time-delta reason id]
   (let [^ISpout spout (:object task-data)
+        storm-conf (:storm-conf executor-data)
         task-id (:task-id task-data)]
     ;;TODO: need to throttle these when there's lots of failures
-    (log-debug "Failing message " msg-id ": " tuple-info)
+    (when (= true (storm-conf TOPOLOGY-DEBUG))
+      (log-message "SPOUT Failing " id ": " tuple-info " REASON: " reason " MSG-ID: " msg-id))
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
       (builtin-metrics/spout-failed-tuple! (:builtin-metrics task-data) (:stats executor-data) (:stream tuple-info))      
       (stats/spout-failed-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
 
-(defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta]
+(defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id]
   (let [storm-conf (:storm-conf executor-data)
         ^ISpout spout (:object task-data)
         task-id (:task-id task-data)]
     (when (= true (storm-conf TOPOLOGY-DEBUG))
-      (log-message "Acking message " msg-id))
+      (log-message "SPOUT Acking message " id " " msg-id))
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
@@ -394,7 +409,7 @@
       (fn [tuple-batch sequence-id end-of-batch?]
         (fast-list-iter [[task-id msg] tuple-batch]
           (let [^TupleImpl tuple (if (instance? Tuple msg) msg (.deserialize deserializer msg))]
-            (when debug? (log-message "Processing received message " tuple))
+            (when debug? (log-message "Processing received message FOR " task-id " TUPLE: " tuple))
             (if task-id
               (tuple-action-fn task-id tuple)
               ;; null task ids are broadcast tuples
@@ -413,7 +428,7 @@
     ret
     ))
 
-(defmethod mk-threads :spout [executor-data task-datas]
+(defmethod mk-threads :spout [executor-data task-datas initial-credentials]
   (let [{:keys [storm-conf component-id worker-context transfer-fn report-error sampler open-or-prepare-was-called?]} executor-data
         ^ISpoutWaitStrategy spout-wait-strategy (init-spout-wait-strategy storm-conf)
         max-spout-pending (executor-max-spout-pending storm-conf (count task-datas))
@@ -425,15 +440,20 @@
         pending (RotatingMap.
                  2 ;; microoptimize for performance of .size method
                  (reify RotatingMap$ExpiredCallback
-                   (expire [this msg-id [task-id spout-id tuple-info start-time-ms]]
+                   (expire [this id [task-id spout-id tuple-info start-time-ms]]
                      (let [time-delta (if start-time-ms (time-delta-ms start-time-ms))]
-                       (fail-spout-msg executor-data (get task-datas task-id) spout-id tuple-info time-delta)
+                       (fail-spout-msg executor-data (get task-datas task-id) spout-id tuple-info time-delta "TIMEOUT" id)
                        ))))
         tuple-action-fn (fn [task-id ^TupleImpl tuple]
                           (let [stream-id (.getSourceStreamId tuple)]
                             (condp = stream-id
                               Constants/SYSTEM_TICK_STREAM_ID (.rotate pending)
                               Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data (get task-datas task-id) tuple)
+                              Constants/CREDENTIALS_CHANGED_STREAM_ID 
+                                (let [task-data (get task-datas task-id)
+                                      spout-obj (:object task-data)]
+                                  (when (instance? ICredentialsListener spout-obj)
+                                    (.setCredentials spout-obj (.getValue tuple 0))))
                               (let [id (.getValue tuple 0)
                                     [stored-task-id spout-id tuple-finished-info start-time-ms] (.remove pending id)]
                                 (when spout-id
@@ -442,9 +462,9 @@
                                   (let [time-delta (if start-time-ms (time-delta-ms start-time-ms))]
                                     (condp = stream-id
                                       ACKER-ACK-STREAM-ID (ack-spout-msg executor-data (get task-datas task-id)
-                                                                         spout-id tuple-finished-info time-delta)
+                                                                         spout-id tuple-finished-info time-delta id)
                                       ACKER-FAIL-STREAM-ID (fail-spout-msg executor-data (get task-datas task-id)
-                                                                           spout-id tuple-finished-info time-delta)
+                                                                           spout-id tuple-finished-info time-delta "FAIL-STREAM" id)
                                       )))
                                 ;; TODO: on failure, emit tuple to failure stream
                                 ))))
@@ -461,7 +481,7 @@
         ;; when the overflow buffer is full, spouts stop calling nextTuple until it's able to clear the overflow buffer
         ;; this limits the size of the overflow buffer to however many tuples a spout emits in one call of nextTuple, 
         ;; preventing memory issues
-        overflow-buffer (LinkedList.)]
+        overflow-buffer (ConcurrentLinkedQueue.)]
    
     [(async-loop
       (fn []
@@ -472,8 +492,8 @@
         (log-message "Opening spout " component-id ":" (keys task-datas))
         (doseq [[task-id task-data] task-datas
                 :let [^ISpout spout-obj (:object task-data)
-                      tasks-fn (:tasks-fn task-data)
-                      send-spout-msg (fn [out-stream-id values message-id out-task-id]
+                     tasks-fn (:tasks-fn task-data)
+                     send-spout-msg (fn [out-stream-id values message-id out-task-id]
                                        (.increment emitted-count)
                                        (let [out-tasks (if out-task-id
                                                          (tasks-fn out-task-id out-stream-id values)
@@ -507,13 +527,14 @@
                                            (when message-id
                                              (ack-spout-msg executor-data task-data message-id
                                                             {:stream out-stream-id :values values}
-                                                            (if (sampler) 0))))
+                                                            (if (sampler) 0) "0:")))
                                          (or out-tasks [])
                                          ))]]
           (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf (:user-context task-data))
           (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
                                                    :receive receive-queue}
                                                   storm-conf (:user-context task-data))
+          (when (instance? ICredentialsListener spout-obj) (.setCredentials spout-obj initial-credentials))
 
           (.open spout-obj
                  storm-conf
@@ -544,7 +565,7 @@
             (while (not (.isEmpty overflow-buffer))
               (let [[out-task out-tuple] (.peek overflow-buffer)]
                 (transfer-fn out-task out-tuple false nil)
-                (.removeFirst overflow-buffer)))
+                (.poll overflow-buffer)))
           (catch InsufficientCapacityException e
             ))
           
@@ -592,8 +613,9 @@
   (let [curr (or (.get pending key) (long 0))]
     (.put pending key (bit-xor curr id))))
 
-(defmethod mk-threads :bolt [executor-data task-datas]
-  (let [execute-sampler (mk-stats-sampler (:storm-conf executor-data))
+(defmethod mk-threads :bolt [executor-data task-datas initial-credentials]
+  (let [storm-conf (:storm-conf executor-data)
+        execute-sampler (mk-stats-sampler storm-conf)
         executor-stats (:stats executor-data)
         {:keys [storm-conf component-id worker-context transfer-fn report-error sampler
                 open-or-prepare-was-called?]} executor-data
@@ -617,6 +639,11 @@
                           ;; need to do it this way to avoid reflection
                           (let [stream-id (.getSourceStreamId tuple)]
                             (condp = stream-id
+                              Constants/CREDENTIALS_CHANGED_STREAM_ID 
+                                (let [task-data (get task-datas task-id)
+                                      bolt-obj (:object task-data)]
+                                  (when (instance? ICredentialsListener bolt-obj)
+                                    (.setCredentials bolt-obj (.getValue tuple 0))))
                               Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data (get task-datas task-id) tuple)
                               (let [task-data (get task-datas task-id)
                                     ^IBolt bolt-obj (:object task-data)
@@ -630,6 +657,9 @@
                                   (.setExecuteSampleStartTime tuple now))
                                 (.execute bolt-obj tuple)
                                 (let [delta (tuple-execute-time-delta! tuple)]
+                                  (when (= true (storm-conf TOPOLOGY-DEBUG))
+                                    (log-message "Execute done TUPLE " tuple " TASK: " task-id " DELTA: " delta))
+ 
                                   (task/apply-hooks user-context .boltExecute (BoltExecuteInfo. tuple task-id delta))
                                   (when delta
                                     (builtin-metrics/bolt-execute-tuple! (:builtin-metrics task-data)
@@ -640,7 +670,16 @@
                                     (stats/bolt-execute-tuple! executor-stats
                                                                (.getSourceComponent tuple)
                                                                (.getSourceStreamId tuple)
-                                                               delta)))))))]
+                                                               delta)))))))
+
+        ;; the overflow buffer is used to ensure that bolts do not block when emitting
+        ;; this ensures that the bolt can always clear the incoming messages, which
+        ;; prevents deadlock from occurs across the topology
+        ;; (e.g. Spout -> BoltA -> Splitter -> BoltB -> BoltA, and all
+        ;; buffers filled up)
+        ;; the overflow buffer is might gradually fill degrading the performance gradually
+        ;; eventually running out of memory, but at least prevent live-locks/deadlocks.
+        overflow-buffer (ConcurrentLinkedQueue.)]
     
     ;; TODO: can get any SubscribedState objects out of the context now
 
@@ -674,9 +713,11 @@
                                                                                values
                                                                                task-id
                                                                                stream
-                                                                               (MessageId/makeId anchors-to-ids)))))
+                                                                               (MessageId/makeId anchors-to-ids))
+                                                                   overflow-buffer)))
                                     (or out-tasks [])))]]
           (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context)
+          (when (instance? ICredentialsListener bolt-obj) (.setCredentials bolt-obj initial-credentials)) 
           (if (= component-id Constants/SYSTEM_COMPONENT_ID)
             (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
                                                      :receive (:receive-queue executor-data)
@@ -702,9 +743,12 @@
                            (fast-map-iter [[root id] (.. tuple getMessageId getAnchorsToIds)]
                                           (task/send-unanchored task-data
                                                                 ACKER-ACK-STREAM-ID
-                                                                [root (bit-xor id ack-val)])
+                                                                [root (bit-xor id ack-val)] overflow-buffer)
                                           ))
-                         (let [delta (tuple-time-delta! tuple)]
+                         (let [delta (tuple-time-delta! tuple)
+                               debug? (= true (storm-conf TOPOLOGY-DEBUG))]
+                           (when debug? 
+                             (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when delta
                              (builtin-metrics/bolt-acked-tuple! (:builtin-metrics task-data)
@@ -720,8 +764,11 @@
                          (fast-list-iter [root (.. tuple getMessageId getAnchors)]
                                          (task/send-unanchored task-data
                                                                ACKER-FAIL-STREAM-ID
-                                                               [root]))
-                         (let [delta (tuple-time-delta! tuple)]
+                                                               [root] overflow-buffer))
+                         (let [delta (tuple-time-delta! tuple)
+                               debug? (= true (storm-conf TOPOLOGY-DEBUG))]
+                           (when debug? 
+                             (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when delta
                              (builtin-metrics/bolt-failed-tuple! (:builtin-metrics task-data)
@@ -744,6 +791,16 @@
           (disruptor/consumer-started! receive-queue)
           (fn []            
             (disruptor/consume-batch-when-available receive-queue event-handler)
+            ;; try to clear the overflow-buffer
+            (try-cause
+              (while (not (.isEmpty overflow-buffer))
+                (let [[out-task out-tuple] (.peek overflow-buffer)]
+                  (transfer-fn out-task out-tuple false nil)
+                  (.poll overflow-buffer)))
+              (catch InsufficientCapacityException e
+                (when (= true (storm-conf TOPOLOGY-DEBUG))
+                  (log-message "Insufficient Capacity on queue to emit by bolt " component-id ":" (keys task-datas) ))
+                ))
             0)))
       :kill-fn (:report-error-and-die executor-data)
       :factory? true
diff --git a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
index 4903b4d..36a737f 100644
--- a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
@@ -15,35 +15,180 @@
 ;; limitations under the License.
 (ns backtype.storm.daemon.logviewer
   (:use compojure.core)
+  (:use [clojure.set :only [difference]])
+  (:use [clojure.string :only [blank?]])
   (:use [hiccup core page-helpers])
-  (:use [backtype.storm config util log])
+  (:use [backtype.storm config util log timer])
+  (:use [backtype.storm.ui helpers])
   (:use [ring.adapter.jetty :only [run-jetty]])
   (:import [org.slf4j LoggerFactory])
   (:import [ch.qos.logback.classic Logger])
   (:import [ch.qos.logback.core FileAppender])
-  (:import [java.io File])
+  (:import [java.io File FileFilter FileInputStream])
+  (:import [org.yaml.snakeyaml Yaml]
+           [org.yaml.snakeyaml.constructor SafeConstructor])
+  (:import [backtype.storm.ui InvalidRequestException]
+           [backtype.storm.security.auth AuthUtils])
   (:require [compojure.route :as route]
             [compojure.handler :as handler]
+            [ring.middleware.keyword-params]
+            [ring.util.response :as resp])
+  (:require [backtype.storm.daemon common [supervisor :as supervisor]])
+  (:import [java.io File FileFilter])
+  (:require [compojure.route :as route]
+            [compojure.handler :as handler]
+            [ring.util.response :as resp]
             [clojure.string :as string])
   (:gen-class))
 
-(defn tail-file [path tail root-dir]
-  (let [flen (.length (clojure.java.io/file path))
-        skip (- flen tail)
-        log-dir (.getCanonicalFile (File. root-dir))
-        log-file (File. path)]
-    (if (= log-dir (.getParentFile log-file))
-      (with-open [input (clojure.java.io/input-stream path)
-                  output (java.io.ByteArrayOutputStream.)]
-        (if (> skip 0) (.skip input skip))
-          (let [buffer (make-array Byte/TYPE 1024)]
-            (loop []
-              (let [size (.read input buffer)]
-                (when (and (pos? size) (< (.size output) tail))
-                  (do (.write output buffer 0 size)
-                      (recur))))))
-        (.toString output)) "File not found")
-    ))
+(def ^:dynamic *STORM-CONF* (read-storm-config))
+
+(defn cleanup-cutoff-age-millis [conf now-millis]
+  (- now-millis (* (conf LOGVIEWER-CLEANUP-AGE-MINS) 60 1000)))
+
+(defn mk-FileFilter-for-log-cleanup [conf now-millis]
+  (let [cutoff-age-millis (cleanup-cutoff-age-millis conf now-millis)]
+    (reify FileFilter (^boolean accept [this ^File file]
+                        (boolean (and
+                          (.isFile file)
+                          (re-find worker-log-filename-pattern (.getName file))
+                          (<= (.lastModified file) cutoff-age-millis)))))))
+
+(defn select-files-for-cleanup [conf now-millis root-dir]
+  (let [file-filter (mk-FileFilter-for-log-cleanup conf now-millis)]
+    (.listFiles (File. root-dir) file-filter)))
+
+(defn get-metadata-file-for-log-root-name [root-name root-dir]
+  (let [metaFile (clojure.java.io/file root-dir "metadata"
+                                       (str root-name ".yaml"))]
+    (if (.exists metaFile)
+      metaFile
+      (do
+        (log-warn "Could not find " (.getCanonicalPath metaFile)
+                  " to clean up for " root-name)
+        nil))))
+
+(defn get-worker-id-from-metadata-file [metaFile]
+  (get (clojure-from-yaml-file metaFile) "worker-id"))
+
+(defn get-topo-owner-from-metadata-file [metaFile]
+  (get (clojure-from-yaml-file metaFile) TOPOLOGY-SUBMITTER-USER))
+
+(defn get-log-root->files-map [log-files]
+  "Returns a map of \"root name\" to a the set of files in log-files having the
+  root name.  The \"root name\" of a log file is the part of the name preceding
+  the extension."
+  (reduce #(assoc %1                                      ;; The accumulated map so far
+                  (first %2)                              ;; key: The root name of the log file
+                  (conj (%1 (first %2) #{}) (second %2))) ;; val: The set of log files with the root name
+          {}                                              ;; initial (empty) map
+          (map #(list
+                  (second (re-find worker-log-filename-pattern (.getName %))) ;; The root name of the log file
+                  %)                                                          ;; The log file
+               log-files)))
+
+(defn identify-worker-log-files [log-files root-dir]
+  (into {} (for [log-root-entry (get-log-root->files-map log-files)
+                 :let [metaFile (get-metadata-file-for-log-root-name
+                                  (key log-root-entry) root-dir)
+                       log-root (key log-root-entry)
+                       files (val log-root-entry)]
+                 :when metaFile]
+             {(get-worker-id-from-metadata-file metaFile)
+              {:owner (get-topo-owner-from-metadata-file metaFile)
+               :files
+                 ;; If each log for this root name is to be deleted, then
+                 ;; include the metadata file also.
+                 (if (empty? (difference
+                                  (set (filter #(re-find (re-pattern log-root) %)
+                                               (read-dir-contents root-dir)))
+                                  (set (map #(.getName %) files))))
+                  (conj files metaFile)
+                  ;; Otherwise, keep the list of files as it is.
+                  files)}})))
+
+(defn get-dead-worker-files-and-owners [conf now-secs log-files root-dir]
+  (if (empty? log-files)
+    {}
+    (let [id->heartbeat (supervisor/read-worker-heartbeats conf)
+          alive-ids (keys (remove
+                            #(or (not (val %))
+                                 (supervisor/is-worker-hb-timed-out? now-secs (val %) conf))
+                            id->heartbeat))
+          id->entries (identify-worker-log-files log-files root-dir)]
+      (for [[id {:keys [owner files]}] id->entries
+            :when (not (contains? (set alive-ids) id))]
+        {:owner owner
+         :files files}))))
+
+(defn cleanup-fn! [log-root-dir]
+  (let [now-secs (current-time-secs)
+        old-log-files (select-files-for-cleanup *STORM-CONF* (* now-secs 1000) log-root-dir)
+        dead-worker-files (get-dead-worker-files-and-owners *STORM-CONF* now-secs old-log-files log-root-dir)]
+    (log-debug "log cleanup: now(" now-secs
+               ") old log files (" (seq (map #(.getName %) old-log-files))
+               ") dead worker files (" (seq (map #(.getName %) dead-worker-files)) ")")
+    (dofor [{:keys [owner files]} dead-worker-files
+            file files]
+      (let [path (.getCanonicalPath file)]
+        (log-message "Cleaning up: Removing " path)
+        (try
+          (if (or (blank? owner) (re-matches #".*\.yaml$" path))
+            (rmr path)
+            ;; worker-launcher does not actually launch a worker process.  It
+            ;; merely executes one of a prescribed set of commands.  In this case, we ask it
+            ;; to delete a file as the owner of that file.
+            (supervisor/worker-launcher *STORM-CONF* owner (str "rmr " path)))
+          (catch Exception ex
+            (log-error ex)))))))
+
+(defn start-log-cleaner! [conf log-root-dir]
+  (let [interval-secs (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)]
+    (when interval-secs
+      (log-debug "starting log cleanup thread at interval: " interval-secs)
+      (schedule-recurring (mk-timer :thread-name "logviewer-cleanup")
+                          0 ;; Start immediately.
+                          interval-secs
+                          (fn [] (cleanup-fn! log-root-dir))))))
+
+(defn page-file
+  ([path tail]
+    (let [flen (.length (clojure.java.io/file path))
+          skip (- flen tail)]
+      (page-file path skip tail)))
+  ([path start length]
+    (with-open [input (FileInputStream. path)
+                output (java.io.ByteArrayOutputStream.)]
+      (if (>= start (.length (clojure.java.io/file path)))
+        (throw
+          (InvalidRequestException. "Cannot start past the end of the file")))
+      (if (> start 0)
+        ;; FileInputStream#skip may not work the first time.
+        (loop [skipped 0]
+          (let [skipped (+ skipped (.skip input (- start skipped)))]
+            (if (< skipped start) (recur skipped)))))
+      (let [buffer (make-array Byte/TYPE 1024)]
+        (loop []
+          (when (< (.size output) length)
+            (let [size (.read input buffer 0 (min 1024 (- length (.size output))))]
+              (when (pos? size)
+                (.write output buffer 0 size)
+                (recur)))))
+      (.toString output)))))
+
+(defn get-log-user-whitelist [fname]
+  (let [wl-file (get-log-metadata-file fname)
+        m (clojure-from-yaml-file wl-file)]
+    (if-let [whitelist (.get m LOGS-USERS)] whitelist [])))
+
+(defn authorized-log-user? [user fname conf]
+  (if (or (blank? user) (blank? fname))
+    nil
+    (let [whitelist (get-log-user-whitelist fname)
+          logs-users (concat (conf LOGS-USERS)
+                             (conf NIMBUS-ADMINS)
+                             whitelist)]
+       (some #(= % user) logs-users))))
 
 (defn log-root-dir
   "Given an appender name, as configured, get the parent directory of the appender's log file.
@@ -56,52 +201,168 @@
       (throw
        (RuntimeException. "Log viewer could not find configured appender, or the appender is not a FileAppender. Please check that the appender name configured in storm and logback agree.")))))
 
-(defn log-page [file tail grep root-dir]
-  (let [path (.getCanonicalPath (File. root-dir file))
-        tail (if tail
-               (min 10485760 (Integer/parseInt tail))
-               10240)
-        tail-string (tail-file path tail root-dir)]
-    (if grep
-       (clojure.string/join "\n<br>"
-         (filter #(.contains % grep) (.split tail-string "\n")))
-       (.replaceAll tail-string "\n" "\n<br>"))))
+(defn pager-links [fname start length file-size]
+  (let [prev-start (max 0 (- start length))
+        next-start (if (> file-size 0)
+                     (min (max 0 (- file-size length)) (+ start length))
+                     (+ start length))]
+    [[:div.pagination
+      [:ul
+        (concat
+          [[(if (< prev-start start) (keyword "li") (keyword "li.disabled"))
+            (link-to (url "/log"
+                          {:file fname
+                             :start (max 0 (- start length))
+                             :length length})
+                     "Prev")]]
+          [[:li (link-to
+                  (url "/log"
+                       {:file fname
+                        :start 0
+                        :length length})
+                  "First")]]
+          [[:li (link-to
+                  (url "/log"
+                       {:file fname
+                        :length length})
+                  "Last")]]
+          [[(if (> next-start start) (keyword "li.next") (keyword "li.next.disabled"))
+            (link-to (url "/log"
+                          {:file fname
+                           :start (min (max 0 (- file-size length))
+                                       (+ start length))
+                           :length length})
+                     "Next")]])]]]))
 
-(defn log-template [body]
-  (html4
-   [:head
-    [:title "Storm log viewer"]
-    (include-css "/css/bootstrap-1.4.0.css")
-    (include-css "/css/style.css")
-    (include-js "/js/jquery-1.6.2.min.js")
-    (include-js "/js/jquery.tablesorter.min.js")
-    (include-js "/js/jquery.cookies.2.2.0.min.js")
-    (include-js "/js/script.js")
-    ]
-   [:body
-    (seq body)
-    ]))
+(defn- download-link [fname]
+  [[:p (link-to (url-format "/download/%s" fname) "Download Full Log")]])
+
+(defn log-page [fname start length grep user root-dir]
+  (if (or (blank? (*STORM-CONF* UI-FILTER))
+          (authorized-log-user? user fname *STORM-CONF*))
+    (let [file (.getCanonicalFile (File. root-dir fname))
+          file-length (.length file)
+          path (.getCanonicalPath file)]
+      (if (= (File. root-dir)
+             (.getParentFile file))
+        (let [default-length 51200
+              length (if length
+                       (min 10485760 length)
+                     default-length)
+              log-string (escape-html
+                           (if start
+                             (page-file path start length)
+                             (page-file path length)))
+              start (or start (- file-length length))]
+          (if grep
+            (html [:pre#logContent
+                   (if grep
+                     (filter #(.contains % grep)
+                             (.split log-string "\n"))
+                     log-string)])
+            (let [pager-data (pager-links fname start length file-length)]
+              (html (concat pager-data
+                            (download-link fname)
+                            [[:pre#logContent log-string]]
+                            pager-data)))))
+        (-> (resp/response "Page not found")
+            (resp/status 404))))
+    (unauthorized-user-html user)))
+
+(defn download-log-file [fname req resp user ^String root-dir]
+  (let [file (.getCanonicalFile (File. root-dir fname))]
+    (if (= (File. root-dir) (.getParentFile file))
+      (if (or (blank? (*STORM-CONF* UI-FILTER))
+              (authorized-log-user? user fname *STORM-CONF*))
+        (-> (resp/response file)
+            (resp/content-type "application/octet-stream"))
+        (unauthorized-user-html user))
+      (-> (resp/response "Page not found")
+          (resp/status 404)))))
+
+(defn log-template
+  ([body] (log-template body nil nil))
+  ([body fname user]
+    (html4
+     [:head
+      [:title (str (escape-html fname) " - Storm Log Viewer")]
+      (include-css "/css/bootstrap-1.4.0.css")
+      (include-css "/css/style.css")
+      ]
+     [:body
+      (concat
+        (when (not (blank? user)) [[:div.ui-user [:p "User: " user]]])
+        [[:h3 (escape-html fname)]]
+        (seq body))
+      ])))
+
+(def http-creds-handler (AuthUtils/GetUiHttpCredentialsPlugin *STORM-CONF*))
+
+(defn- parse-long-from-map [m k]
+  (try
+    (Long/parseLong (k m))
+    (catch NumberFormatException ex
+      (throw (InvalidRequestException.
+               (str "Could not make an integer out of the query parameter '"
+                    (name k) "'")
+               ex)))))
 
 (defroutes log-routes
   (GET "/log" [:as req & m]
-       (log-template (log-page (:file m) (:tail m) (:grep m) (:log-root req))))
+       (try
+         (let [servlet-request (:servlet-request req)
+               log-root (:log-root req)
+               user (.getUserName http-creds-handler servlet-request)
+               start (if (:start m) (parse-long-from-map m :start))
+               length (if (:length m) (parse-long-from-map m :length))]
+           (log-template (log-page (:file m) start length (:grep m) user log-root)
+                         (:file m) user))
+         (catch InvalidRequestException ex
+           (log-error ex)
+           (ring-response-from-exception ex))))
+  (GET "/download/:file" [:as {:keys [servlet-request servlet-response log-root]} file & m]
+       (try
+         (let [user (.getUserName http-creds-handler servlet-request)]
+           (download-log-file file servlet-request servlet-response user log-root))
+         (catch InvalidRequestException ex
+           (log-error ex)
+           (ring-response-from-exception ex))))
   (route/resources "/")
   (route/not-found "Page not found"))
 
-(def logapp
-  (handler/site log-routes)
- )
-
 (defn conf-middleware
   "For passing the storm configuration with each request."
   [app log-root]
   (fn [req]
     (app (assoc req :log-root log-root))))
 
-(defn start-logviewer [port log-root]
-  (run-jetty (conf-middleware logapp log-root) {:port port}))
+(defn start-logviewer! [conf log-root-dir]
+  (try
+    (let [header-buffer-size (int (.get conf UI-HEADER-BUFFER-BYTES))
+          filter-class (conf UI-FILTER)
+          filter-params (conf UI-FILTER-PARAMS)
+          logapp (handler/api log-routes) ;; query params as map
+          middle (conf-middleware logapp log-root-dir)
+          filters-confs (if (conf UI-FILTER)
+                          [{:filter-class filter-class
+                            :filter-params (or (conf UI-FILTER-PARAMS) {})}]
+                          [])
+          filters-confs (concat filters-confs
+                          [{:filter-class "org.mortbay.servlet.GzipFilter"
+                            :filter-name "Gzipper"
+                            :filter-params {}}])]
+      (run-jetty middle 
+                        {:port (int (conf LOGVIEWER-PORT))
+                         :join? false
+                         :configurator (fn [server]
+                                         (doseq [connector (.getConnectors server)]
+                                           (.setHeaderBufferSize connector header-buffer-size))
+                                         (config-filter server middle filters-confs))}))
+  (catch Exception ex
+    (log-error ex))))
 
 (defn -main []
   (let [conf (read-storm-config)
         log-root (log-root-dir (conf LOGVIEWER-APPENDER-NAME))]
-    (start-logviewer (int (conf LOGVIEWER-PORT)) log-root)))
+    (start-log-cleaner! conf log-root)
+    (start-logviewer! conf log-root)))
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index bf22a1b..9502745 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -14,19 +14,18 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.daemon.nimbus
-  (:import [org.apache.thrift.server THsHaServer THsHaServer$Args])
-  (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
-  (:import [org.apache.thrift.exception])
-  (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
   (:import [java.nio ByteBuffer])
   (:import [java.io FileNotFoundException])
   (:import [java.nio.channels Channels WritableByteChannel])
+  (:import [backtype.storm.security.auth ThriftServer ThriftConnectionType ReqContext AuthUtils])
   (:use [backtype.storm.scheduler.DefaultScheduler])
   (:import [backtype.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails
             Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
+  (:import [backtype.storm.generated AuthorizationException])
   (:use [backtype.storm bootstrap util])
   (:use [backtype.storm.config :only [validate-configs-with-schemas]])
   (:use [backtype.storm.daemon common])
+  (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
   (:gen-class
     :methods [^{:static true} [launch [backtype.storm.scheduler.INimbus] void]]))
 
@@ -59,13 +58,22 @@
     scheduler
     ))
 
+(def NIMBUS-ZK-ACLS
+  [(first ZooDefs$Ids/CREATOR_ALL_ACL) 
+   (ACL. (bit-or ZooDefs$Perms/READ ZooDefs$Perms/CREATE) ZooDefs$Ids/ANYONE_ID_UNSAFE)])
+
 (defn nimbus-data [conf inimbus]
   (let [forced-scheduler (.getForcedScheduler inimbus)]
     {:conf conf
      :inimbus inimbus
+     :authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
      :submitted-count (atom 0)
-     :storm-cluster-state (cluster/mk-storm-cluster-state conf)
+     :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
+                                                                       (Utils/isZkAuthenticationConfiguredStormServer
+                                                                         conf)
+                                                                       NIMBUS-ZK-ACLS))
      :submit-lock (Object.)
+     :cred-update-lock (Object.)
      :heartbeats-cache (atom {})
      :downloaders (file-cache-map conf)
      :uploaders (file-cache-map conf)
@@ -76,6 +84,8 @@
                                  (halt-process! 20 "Error when processing an event")
                                  ))
      :scheduler (mk-scheduler conf inimbus)
+     :id->sched-status (atom {})
+     :cred-renewers (AuthUtils/GetCredentialRenewers conf)
      }))
 
 (defn inbox [nimbus]
@@ -302,6 +312,7 @@
 
 (defn- setup-storm-code [conf storm-id tmp-jar-location storm-conf topology]
   (let [stormroot (master-stormdist-root conf storm-id)]
+   (log-message "nimbus file location:" stormroot)
    (FileUtils/forceMkdir (File. stormroot))
    (FileUtils/cleanDirectory (File. stormroot))
    (setup-jar conf tmp-jar-location stormroot)
@@ -590,6 +601,7 @@
         new-scheduler-assignments (.getAssignments cluster)
         ;; add more information to convert SchedulerAssignment to Assignment
         new-topology->executor->node+port (compute-topology->executor->node+port new-scheduler-assignments)]
+    (reset! (:id->sched-status nimbus) (.getStatusMap cluster))
     ;; print some useful information.
     (doseq [[topology-id executor->node+port] new-topology->executor->node+port
             :let [old-executor->node+port (-> topology-id
@@ -663,6 +675,7 @@
                                        topologies
                                        scratch-topology-id)
         
+        topology->executor->node+port (merge (into {} (for [id assigned-topology-ids] {id nil})) topology->executor->node+port)
         
         now-secs (current-time-secs)
         
@@ -725,7 +738,8 @@
                                   (current-time-secs)
                                   {:type topology-initial-status}
                                   (storm-conf TOPOLOGY-WORKERS)
-                                  num-executors))))
+                                  num-executors
+                                  (storm-conf TOPOLOGY-SUBMITTER-USER)))))
 
 ;; Master:
 ;; job submit:
@@ -745,6 +759,19 @@
       (throw (AlreadyAliveException. (str storm-name " is already active"))))
     ))
 
+(defn check-authorization! 
+  ([nimbus storm-name storm-conf operation context]
+     (let [aclHandler (:authorization-handler nimbus)
+           ctx (or context (ReqContext/context))
+           check-conf (if storm-conf storm-conf (if storm-name {TOPOLOGY-NAME storm-name}))]
+       (log-message "[req " (.requestID ctx) "] Access from: " (.remoteAddress ctx) " principal:" (.principal ctx) " op:" operation)
+       (if aclHandler
+         (if-not (.permit aclHandler ctx operation check-conf)
+           (throw (AuthorizationException. (str operation (if storm-name (str " on topology " storm-name)) " is not authorized")))
+           ))))
+  ([nimbus storm-name storm-conf operation]
+     (check-authorization! nimbus storm-name storm-conf operation (ReqContext/context))))
+
 (defn code-ids [conf]
   (-> conf
       master-stormdist-root
@@ -830,6 +857,24 @@
           (swap! (:heartbeats-cache nimbus) dissoc id))
         ))))
 
+(defn renew-credentials [nimbus]
+  (let [storm-cluster-state (:storm-cluster-state nimbus)
+        renewers (:cred-renewers nimbus)
+        update-lock (:cred-update-lock nimbus)
+        assigned-ids (set (.active-storms storm-cluster-state))]
+    (when-not (empty? assigned-ids)
+      (doseq [id assigned-ids]
+        (locking update-lock
+          (let [orig-creds (.credentials storm-cluster-state id nil)]
+            (if orig-creds
+              (let [new-creds (HashMap. orig-creds)]
+                (doseq [renewer renewers]
+                  (log-message "Renewing Creds For " id " with " renewer)
+		  (.renew renewer new-creds))
+                (when-not (= orig-creds new-creds)
+                  (.set-credentials! storm-cluster-state id new-creds)
+              )))))))))
+
 (defn- file-older-than? [now seconds file]
   (<= (+ (.lastModified file) (to-millis seconds)) (to-millis now)))
 
@@ -872,26 +917,63 @@
     (throw (InvalidTopologyException. 
             ("Topology name cannot be blank"))))))
 
-(defn- try-read-storm-conf [conf storm-id]
+;; We will only file at <Storm dist root>/<Topology ID>/<File>
+;; to be accessed via Thrift
+;; ex., storm-local/nimbus/stormdist/aa-1-1377104853/stormjar.jar
+(defn check-file-access [conf file-path]
+  (log-debug "check file access:" file-path)
+  (try
+    (if (not= (.getCanonicalFile (File. (master-stormdist-root conf)))
+          (-> (File. file-path) .getCanonicalFile .getParentFile .getParentFile))
+      (throw (AuthorizationException. (str "Invalid file path: " file-path))))
+    (catch Exception e
+      (throw (AuthorizationException. (str "Invalid file path: " file-path))))))
+
+(defn try-read-storm-conf [conf storm-id]
   (try-cause
     (read-storm-conf conf storm-id)
     (catch FileNotFoundException e
-       (throw (NotAliveException. storm-id)))
+       (throw (NotAliveException. (str storm-id))))
   )
 )
 
-(defn- try-read-storm-topology [conf storm-id]
+(defn try-read-storm-conf-from-name [conf storm-name nimbus]
+  (let [storm-cluster-state (:storm-cluster-state nimbus)
+        id (get-storm-id storm-cluster-state storm-name)]
+   (try-read-storm-conf conf id)))
+
+(defn try-read-storm-topology [conf storm-id]
   (try-cause
     (read-storm-topology conf storm-id)
     (catch FileNotFoundException e
-       (throw (NotAliveException. storm-id)))
+       (throw (NotAliveException. (str storm-id))))
   )
 )
 
+(defn validate-topology-size [topo-conf nimbus-conf topology]
+  (let [workers-count (get topo-conf TOPOLOGY-WORKERS)
+        workers-allowed (get nimbus-conf NIMBUS-SLOTS-PER-TOPOLOGY)
+        num-executors (->> (all-components topology) (map-val num-start-executors))
+        executors-count (reduce + (vals num-executors))
+        executors-allowed (get nimbus-conf NIMBUS-EXECUTORS-PER-TOPOLOGY)]
+    (when (and 
+           (not (nil? executors-allowed))
+           (> executors-count executors-allowed))
+      (throw 
+       (InvalidTopologyException. 
+        (str "Failed to submit topology. Topology requests more than " executors-allowed " executors."))))
+    (when (and
+           (not (nil? workers-allowed))
+           (> workers-count workers-allowed))
+      (throw 
+       (InvalidTopologyException. 
+        (str "Failed to submit topology. Topology requests more than " workers-allowed " workers."))))))
+
 (defserverfn service-handler [conf inimbus]
   (.prepare inimbus conf (master-inimbus-dir conf))
   (log-message "Starting Nimbus with conf " conf)
-  (let [nimbus (nimbus-data conf inimbus)]
+  (let [nimbus (nimbus-data conf inimbus)
+       principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf)]
     (.prepare ^backtype.storm.nimbus.ITopologyValidator (:validator nimbus) conf)
     (cleanup-corrupt-topologies! nimbus)
     (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
@@ -912,6 +994,12 @@
                         (fn []
                           (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS))
                           ))    
+    (schedule-recurring (:timer nimbus)
+                        0
+                        (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS)
+                        (fn []
+                          (renew-credentials nimbus)))
+
     (reify Nimbus$Iface
       (^void submitTopologyWithOpts
         [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology
@@ -919,6 +1007,7 @@
         (try
           (assert (not-nil? submitOptions))
           (validate-topology-name! storm-name)
+          (check-authorization! nimbus storm-name nil "submitTopology")
           (check-storm-active! nimbus storm-name false)
           (let [topo-conf (from-json serializedConf)]
             (try
@@ -931,21 +1020,44 @@
                        topology))
           (swap! (:submitted-count nimbus) inc)
           (let [storm-id (str storm-name "-" @(:submitted-count nimbus) "-" (current-time-secs))
-                storm-conf (normalize-conf
+                credentials (.get_creds submitOptions)
+                credentials (when credentials (.get_creds credentials))
+                topo-conf (from-json serializedConf)
+                storm-conf-submitted (normalize-conf
                             conf
-                            (-> serializedConf
-                                from-json
-                                (assoc STORM-ID storm-id)
+                            (-> topo-conf
+                              (assoc STORM-ID storm-id)
                               (assoc TOPOLOGY-NAME storm-name))
                             topology)
+                req (ReqContext/context)
+                principal (.principal req)
+                submitter-principal (if principal (.toString principal))
+                submitter-user (.toLocal principal-to-local principal)
+                topo-acl (distinct (remove nil? (conj (.get storm-conf-submitted TOPOLOGY-USERS) submitter-principal, submitter-user)))
+                storm-conf (-> storm-conf-submitted
+                               (assoc TOPOLOGY-SUBMITTER-PRINCIPAL (if submitter-principal submitter-principal ""))
+                               (assoc TOPOLOGY-SUBMITTER-USER (if submitter-user submitter-user "")) ;Don't let the user set who we launch as
+                               (assoc TOPOLOGY-USERS topo-acl)
+                               (assoc STORM-ZOOKEEPER-SUPERACL (.get conf STORM-ZOOKEEPER-SUPERACL)))
+                storm-conf (if (Utils/isZkAuthenticationConfiguredStormServer conf)
+                                storm-conf
+                                (dissoc storm-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
                 total-storm-conf (merge conf storm-conf)
                 topology (normalize-topology total-storm-conf topology)
                 storm-cluster-state (:storm-cluster-state nimbus)]
+            (if (and (conf SUPERVISOR-RUN-WORKER-AS-USER) (or (nil? submitter-user) (.isEmpty (.trim submitter-user)))) 
+              (throw (AuthorizationException. "Could not determine the user to run this topology as.")))
             (system-topology! total-storm-conf topology) ;; this validates the structure of the topology
+            (validate-topology-size topo-conf conf topology)
+            (when (and (Utils/isZkAuthenticationConfiguredStormServer conf)
+                       (not (Utils/isZkAuthenticationConfiguredTopology storm-conf)))
+                (throw (IllegalArgumentException. "The cluster is configured for zookeeper authentication, but no payload was provided.")))
             (log-message "Received topology submission for " storm-name " with conf " storm-conf)
             ;; lock protects against multiple topologies being submitted at once and
             ;; cleanup thread killing topology in b/w assignment and starting the topology
             (locking (:submit-lock nimbus)
+              ;;cred-update-lock is not needed here because creds are being added for the first time.
+              (.set-credentials! storm-cluster-state storm-id credentials storm-conf)
               (setup-storm-code conf storm-id uploadedJarLocation storm-conf topology)
               (.setup-heartbeats! storm-cluster-state storm-id)
               (let [thrift-status->kw-status {TopologyInitialStatus/INACTIVE :inactive
@@ -962,10 +1074,12 @@
                                  (SubmitOptions. TopologyInitialStatus/ACTIVE)))
       
       (^void killTopology [this ^String name]
-        (.killTopologyWithOpts this name (KillOptions.)))
+         (.killTopologyWithOpts this name (KillOptions.)))
 
       (^void killTopologyWithOpts [this ^String storm-name ^KillOptions options]
         (check-storm-active! nimbus storm-name true)
+        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
+          (check-authorization! nimbus storm-name topology-conf "killTopology"))
         (let [wait-amt (if (.is_set_wait_secs options)
                          (.get_wait_secs options)                         
                          )]
@@ -974,6 +1088,8 @@
 
       (^void rebalance [this ^String storm-name ^RebalanceOptions options]
         (check-storm-active! nimbus storm-name true)
+        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
+          (check-authorization! nimbus storm-name topology-conf "rebalance"))
         (let [wait-amt (if (.is_set_wait_secs options)
                          (.get_wait_secs options))
               num-workers (if (.is_set_num_workers options)
@@ -989,13 +1105,26 @@
           ))
 
       (activate [this storm-name]
+        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
+          (check-authorization! nimbus storm-name topology-conf "activate"))
         (transition-name! nimbus storm-name :activate true)
         )
 
       (deactivate [this storm-name]
+        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
+          (check-authorization! nimbus storm-name topology-conf "deactivate"))
         (transition-name! nimbus storm-name :inactivate true))
 
+      (uploadNewCredentials [this storm-name credentials]
+        (let [storm-cluster-state (:storm-cluster-state nimbus)
+              storm-id (get-storm-id storm-cluster-state storm-name)
+              topology-conf (try-read-storm-conf conf storm-id)
+              creds (when credentials (.get_creds credentials))]
+          (check-authorization! nimbus storm-name topology-conf "uploadNewCredentials")
+          (locking (:cred-update-lock nimbus) (.set-credentials! storm-cluster-state storm-id creds topology-conf))))
+
       (beginFileUpload [this]
+        (check-authorization! nimbus nil nil "fileUpload")
         (let [fileloc (str (inbox nimbus) "/stormjar-" (uuid) ".jar")]
           (.put (:uploaders nimbus)
                 fileloc
@@ -1005,6 +1134,7 @@
           ))
 
       (^void uploadChunk [this ^String location ^ByteBuffer chunk]
+        (check-authorization! nimbus nil nil "fileUpload")
         (let [uploaders (:uploaders nimbus)
               ^WritableByteChannel channel (.get uploaders location)]
           (when-not channel
@@ -1015,6 +1145,7 @@
           ))
 
       (^void finishFileUpload [this ^String location]
+        (check-authorization! nimbus nil nil "fileUpload")
         (let [uploaders (:uploaders nimbus)
               ^WritableByteChannel channel (.get uploaders location)]
           (when-not channel
@@ -1026,6 +1157,8 @@
           ))
 
       (^String beginFileDownload [this ^String file]
+        (check-authorization! nimbus nil nil "fileDownload")
+        (check-file-access (:conf nimbus) file)
         (let [is (BufferFileInputStream. file)
               id (uuid)]
           (.put (:downloaders nimbus) id is)
@@ -1033,6 +1166,7 @@
           ))
 
       (^ByteBuffer downloadChunk [this ^String id]
+        (check-authorization! nimbus nil nil "fileDownload")
         (let [downloaders (:downloaders nimbus)
               ^BufferFileInputStream is (.get downloaders id)]
           (when-not is
@@ -1046,18 +1180,29 @@
             )))
 
       (^String getNimbusConf [this]
+        (check-authorization! nimbus nil nil "getNimbusConf")
         (to-json (:conf nimbus)))
 
       (^String getTopologyConf [this ^String id]
-        (to-json (try-read-storm-conf conf id)))
+        (let [topology-conf (try-read-storm-conf conf id)
+              storm-name (topology-conf TOPOLOGY-NAME)]
+              (check-authorization! nimbus storm-name topology-conf "getTopologyConf")
+              (to-json topology-conf)))
 
       (^StormTopology getTopology [this ^String id]
-        (system-topology! (try-read-storm-conf conf id) (try-read-storm-topology conf id)))
+        (let [topology-conf (try-read-storm-conf conf id)
+              storm-name (topology-conf TOPOLOGY-NAME)]
+              (check-authorization! nimbus storm-name topology-conf "getTopology")
+              (system-topology! topology-conf (try-read-storm-topology conf id))))
 
       (^StormTopology getUserTopology [this ^String id]
-        (try-read-storm-topology conf id))
+        (let [topology-conf (try-read-storm-conf conf id)
+              storm-name (topology-conf TOPOLOGY-NAME)]
+              (check-authorization! nimbus storm-name topology-conf "getUserTopology")
+              (try-read-storm-topology topology-conf id)))
 
       (^ClusterSummary getClusterInfo [this]
+        (check-authorization! nimbus nil nil "getClusterInfo")
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               supervisor-infos (all-supervisor-info storm-cluster-state)
               ;; TODO: need to get the port info about supervisors...
@@ -1073,9 +1218,9 @@
                                             ))
               nimbus-uptime ((:uptime nimbus))
               bases (topology-bases storm-cluster-state)
-              topology-summaries (dofor [[id base] bases]
-                                        (let [assignment (.assignment-info storm-cluster-state id nil)]
-                                          (TopologySummary. id
+              topology-summaries (dofor [[id base] bases :when base]
+	                                  (let [assignment (.assignment-info storm-cluster-state id nil)
+                                                topo-summ (TopologySummary. id
                                                             (:storm-name base)
                                                             (->> (:executor->node+port assignment)
                                                                  keys
@@ -1089,7 +1234,10 @@
                                                                  set
                                                                  count)
                                                             (time-delta (:launch-time-secs base))
-                                                            (extract-status-str base))
+                                                            (extract-status-str base))]
+                                               (when-let [owner (:owner base)] (.set_owner topo-summ owner))
+                                               (when-let [sched-status (.get @(:id->sched-status nimbus) id)] (.set_sched_status topo-summ sched-status))
+                                               topo-summ
                                           ))]
           (ClusterSummary. supervisor-summaries
                            nimbus-uptime
@@ -1098,8 +1246,14 @@
       
       (^TopologyInfo getTopologyInfo [this ^String storm-id]
         (let [storm-cluster-state (:storm-cluster-state nimbus)
-              task->component (storm-task-info (try-read-storm-topology conf storm-id) (try-read-storm-conf conf storm-id))
+              topology-conf (try-read-storm-conf conf storm-id)
+              storm-name (topology-conf TOPOLOGY-NAME)
+              _ (check-authorization! nimbus storm-name topology-conf "getTopologyInfo")
+              task->component (storm-task-info (try-read-storm-topology conf storm-id) topology-conf)
               base (.storm-base storm-cluster-state storm-id nil)
+              storm-name (if base (:storm-name base) (throw (NotAliveException. (str storm-id))))
+              launch-time-secs (if base (:launch-time-secs base) (throw (NotAliveException. (str storm-id))))
+              task->component (storm-task-info (try-read-storm-topology conf storm-id) topology-conf)
               assignment (.assignment-info storm-cluster-state storm-id nil)
               beats (.executor-beats storm-cluster-state storm-id (:executor->node+port assignment))
               all-components (-> task->component reverse-map keys)
@@ -1120,14 +1274,16 @@
                                                                 (nil-to-zero (:uptime heartbeat)))
                                             (.set_stats stats))
                                           ))
-              ]
-          (TopologyInfo. storm-id
-                         (:storm-name base)
-                         (time-delta (:launch-time-secs base))
-                         executor-summaries
-                         (extract-status-str base)
-                         errors
-                         )
+              topo-info  (TopologyInfo. storm-id
+                           storm-name
+                           (time-delta launch-time-secs)
+                           executor-summaries
+                           (extract-status-str base)
+                           errors
+                           )]
+            (when-let [owner (:owner base)] (.set_owner topo-info owner))
+            (when-let [sched-status (.get @(:id->sched-status nimbus) storm-id)] (.set_sched_status topo-info sched-status))
+            topo-info
           ))
       
       Shutdownable
@@ -1146,16 +1302,13 @@
 (defn launch-server! [conf nimbus]
   (validate-distributed-mode! conf)
   (let [service-handler (service-handler conf nimbus)
-        options (-> (TNonblockingServerSocket. (int (conf NIMBUS-THRIFT-PORT)))
-                    (THsHaServer$Args.)
-                    (.workerThreads 64)
-                    (.protocolFactory (TBinaryProtocol$Factory. false true (conf NIMBUS-THRIFT-MAX-BUFFER-SIZE)))
-                    (.processor (Nimbus$Processor. service-handler))
-                    )
-       server (THsHaServer. (do (set! (. options maxReadBufferBytes)(conf NIMBUS-THRIFT-MAX-BUFFER-SIZE)) options))]
+        ;;TODO need to honor NIMBUS-THRIFT-MAX-BUFFER-SIZE for different transports
+        server (ThriftServer. conf (Nimbus$Processor. service-handler) 
+                              ThriftConnectionType/NIMBUS)]
     (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.shutdown service-handler) (.stop server))))
     (log-message "Starting Nimbus server...")
-    (.serve server)))
+    (.serve server)
+    service-handler))
 
 ;; distributed implementation
 
@@ -1175,7 +1328,10 @@
   )
 
 (defn -launch [nimbus]
-  (launch-server! (read-storm-config) nimbus))
+  (let [conf (merge
+               (read-storm-config)
+               (read-yaml-config "storm-cluster-auth.yaml" false))]
+  (launch-server! conf nimbus)))
 
 (defn standalone-nimbus []
   (reify INimbus
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 7566a79..2fe7fb8 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -14,10 +14,14 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.daemon.supervisor
+  (:import [java.io OutputStreamWriter BufferedWriter IOException])
   (:import [backtype.storm.scheduler ISupervisor])
   (:use [backtype.storm bootstrap])
   (:use [backtype.storm.daemon common])
   (:require [backtype.storm.daemon [worker :as worker]])
+  (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
+  (:import [org.yaml.snakeyaml Yaml]
+           [org.yaml.snakeyaml.constructor SafeConstructor])
   (:gen-class
     :methods [^{:static true} [launch [backtype.storm.scheduler.ISupervisor] void]]))
 
@@ -62,7 +66,7 @@
   "Returns map from port to struct containing :storm-id and :executors"
   [assignments-snapshot assignment-id]
   (->> (dofor [sid (keys assignments-snapshot)] (read-my-executors assignments-snapshot sid assignment-id))
-       (apply merge-with (fn [& ignored] (throw-runtime "Should not have multiple topologies assigned to one port")))))
+       (apply merge-with (fn [& params] (throw-runtime (str "Should not have multiple topologies assigned to one port " params))))))
 
 (defn- read-storm-code-locations
   [assignments-snapshot]
@@ -98,6 +102,18 @@
          (= (disj (set (:executors worker-heartbeat)) Constants/SYSTEM_EXECUTOR_ID)
             (set (:executors local-assignment))))))
 
+(let [dead-workers (atom #{})]
+  (defn get-dead-workers []
+    @dead-workers)
+  (defn add-dead-worker [worker]
+    (swap! dead-workers conj worker))
+  (defn remove-dead-worker [worker]
+    (swap! dead-workers disj worker)))
+
+(defn is-worker-hb-timed-out? [now hb conf]
+  (> (- now (:time-secs hb))
+     (conf SUPERVISOR-WORKER-TIMEOUT-SECS)))
+
 (defn read-allocated-workers
   "Returns map from worker id to worker heartbeat. if the heartbeat is nil, then the worker is dead (timed out or never wrote heartbeat)"
   [supervisor assigned-executors now]
@@ -114,8 +130,11 @@
                          (or (not (contains? approved-ids id))
                              (not (matches-an-assignment? hb assigned-executors)))
                            :disallowed
-                         (> (- now (:time-secs hb))
-                            (conf SUPERVISOR-WORKER-TIMEOUT-SECS))
+                         (or 
+                          (when (get (get-dead-workers) id)
+                            (log-message "Worker Process " id " has died!")
+                            true)
+                          (is-worker-hb-timed-out? now hb conf))
                            :timed-out
                          true
                            :valid)]
@@ -151,35 +170,87 @@
 (defn generate-supervisor-id []
   (uuid))
 
-(defn try-cleanup-worker [conf id]
+(defnk worker-launcher [conf user args :environment {} :log-prefix nil :exit-code-callback nil]
+  (let [_ (when (clojure.string/blank? user)
+            (throw (java.lang.IllegalArgumentException.
+                     "User cannot be blank when calling worker-launcher.")))
+        wl-initial (conf SUPERVISOR-WORKER-LAUNCHER)
+        storm-home (System/getProperty "storm.home")
+        wl (if wl-initial wl-initial (str storm-home "/bin/worker-launcher"))
+        command (concat [wl user] args)]
+    (log-message "Running as user:" user " command:" (pr-str command))
+    (launch-process command :environment environment :log-prefix log-prefix :exit-code-callback exit-code-callback)
+  ))
+
+(defnk worker-launcher-and-wait [conf user args :environment {} :log-prefix nil]
+  (let [process (worker-launcher conf user args :environment environment)]
+    (if log-prefix
+      (read-and-log-stream log-prefix (.getInputStream process)))
+      (try
+        (.waitFor process)
+      (catch InterruptedException e
+        (log-message log-prefix " interrupted.")))
+      (.exitValue process)))
+
+(defn- rmr-as-user
+  "Launches a process owned by the given user that deletes the given path
+  recursively.  Throws RuntimeException if the directory is not removed."
+  [conf id user path]
+  (worker-launcher-and-wait conf
+                            user
+                            ["rmr" path]
+                            :log-prefix (str "rmr " id))
+  (if (exists-file? path)
+    (throw (RuntimeException. (str path " was not deleted")))))
+
+(defn try-cleanup-worker [conf id user]
   (try
-    (rmr (worker-heartbeats-root conf id))
-    ;; this avoids a race condition with worker or subprocess writing pid around same time
-    (rmpath (worker-pids-root conf id))
-    (rmpath (worker-root conf id))
+    (if (.exists (File. (worker-root conf id)))
+      (do
+        (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
+          (rmr-as-user conf id user (worker-root conf id))
+          (do
+            (rmr (worker-heartbeats-root conf id))
+            ;; this avoids a race condition with worker or subprocess writing pid around same time
+            (rmpath (worker-pids-root conf id))
+            (rmpath (worker-root conf id))))
+        (remove-worker-user! conf id)
+        (remove-dead-worker id)
+      ))
+  (catch IOException e
+    (log-warn-error e "Failed to cleanup worker " id ". Will retry later"))
   (catch RuntimeException e
     (log-warn-error e "Failed to cleanup worker " id ". Will retry later")
     )
   (catch java.io.FileNotFoundException e (log-message (.getMessage e)))
-  (catch java.io.IOException e (log-message (.getMessage e)))
     ))
 
 (defn shutdown-worker [supervisor id]
   (log-message "Shutting down " (:supervisor-id supervisor) ":" id)
   (let [conf (:conf supervisor)
         pids (read-dir-contents (worker-pids-root conf id))
-        thread-pid (@(:worker-thread-pids-atom supervisor) id)]
+        thread-pid (@(:worker-thread-pids-atom supervisor) id)
+        as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)
+        user (get-worker-user conf id)]
     (when thread-pid
       (psim/kill-process thread-pid))
     (doseq [pid pids]
-      (ensure-process-killed! pid)
-      (try
-        (rmpath (worker-pid-path conf id pid))
-        (catch Exception e)) ;; on windows, the supervisor may still holds the lock on the worker directory
-      )
-    (try-cleanup-worker conf id))
+      (if as-user
+        (worker-launcher-and-wait conf user ["signal" pid "9"] :log-prefix (str "kill -9 " pid))
+        (ensure-process-killed! pid))
+      (if as-user
+        (rmr-as-user conf id user (worker-pid-path conf id pid))
+        (try
+          (rmpath (worker-pid-path conf id pid))
+          (catch Exception e)) ;; on windows, the supervisor may still holds the lock on the worker directory
+      ))
+    (try-cleanup-worker conf id user))
   (log-message "Shut down " (:supervisor-id supervisor) ":" id))
 
+(def SUPERVISOR-ZK-ACLS
+  [(first ZooDefs$Ids/CREATOR_ALL_ACL) 
+   (ACL. (bit-or ZooDefs$Perms/READ ZooDefs$Perms/CREATE) ZooDefs$Ids/ANYONE_ID_UNSAFE)])
+
 (defn supervisor-data [conf shared-context ^ISupervisor isupervisor]
   {:conf conf
    :shared-context shared-context
@@ -187,7 +258,10 @@
    :active (atom true)
    :uptime (uptime-computer)
    :worker-thread-pids-atom (atom {})
-   :storm-cluster-state (cluster/mk-storm-cluster-state conf)
+   :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
+                                                                     (Utils/isZkAuthenticationConfiguredStormServer
+                                                                       conf)
+                                                                     SUPERVISOR-ZK-ACLS))
    :local-state (supervisor-state conf)
    :supervisor-id (.getSupervisorId isupervisor)
    :assignment-id (.getAssignmentId isupervisor)
@@ -240,7 +314,8 @@
         (shutdown-worker supervisor id)
         ))
     (doseq [id (vals new-worker-ids)]
-      (local-mkdirs (worker-pids-root conf id)))
+      (local-mkdirs (worker-pids-root conf id))
+      (local-mkdirs (worker-heartbeats-root conf id)))
     (.put local-state LS-APPROVED-WORKERS
           (merge
            (select-keys (.get local-state LS-APPROVED-WORKERS)
@@ -422,8 +497,11 @@
   (.shutdown supervisor)
   )
 
-;; distributed implementation
+(defn setup-storm-code-dir [conf storm-conf dir]
+ (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
+  (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
 
+;; distributed implementation
 (defmethod download-storm-code
     :distributed [conf storm-id master-code-dir]
     ;; Downloading to permanent location is atomic
@@ -436,7 +514,31 @@
       (Utils/downloadFromMaster conf (master-stormconf-path master-code-dir) (supervisor-stormconf-path tmproot))
       (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
       (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
-      ))
+      (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)      
+    ))
+
+(defn write-log-metadata-to-yaml-file! [storm-id port data conf]
+  (let [file (get-log-metadata-file storm-id port)]
+    ;;run worker as user needs the directory to have special permissions
+    ;; or it is insecure
+    (when (and (not (conf SUPERVISOR-RUN-WORKER-AS-USER))
+               (not (.exists (.getParentFile file))))
+      (.mkdirs (.getParentFile file)))
+    (let [writer (java.io.FileWriter. file)
+        yaml (Yaml.)]
+      (try
+        (.dump yaml data writer)
+        (finally
+          (.close writer))))))
+
+(defn write-log-metadata! [storm-conf user worker-id storm-id port conf]
+  (let [data {TOPOLOGY-SUBMITTER-USER user
+              "worker-id" worker-id
+              LOGS-USERS (sort (distinct (remove nil?
+                                           (concat
+                                             (storm-conf LOGS-USERS)
+                                             (storm-conf TOPOLOGY-USERS)))))}]
+    (write-log-metadata-to-yaml-file! storm-id port data conf)))
 
 (defn jlp [stormroot conf]
   (let [resource-root (str stormroot File/separator RESOURCES-SUBDIR)
@@ -445,11 +547,23 @@
         arch-resource-root (str resource-root File/separator os "-" arch)]
     (str arch-resource-root File/pathSeparator resource-root File/pathSeparator (conf JAVA-LIBRARY-PATH)))) 
 
-(defn- substitute-worker-childopts [value port]
-  (let [sub-fn (fn [s] (.replaceAll s "%ID%" (str port)))]
-    (if (list? value)
-      (map sub-fn value)
-      (-> value sub-fn (.split " ")))))
+(defn substitute-childopts
+  [childopts worker-id storm-id port]
+  (
+    let [replacement-map {"%ID%"          (str port)
+                          "%WORKER-ID%"   (str worker-id)
+                          "%STORM-ID%"    (str storm-id)
+                          "%WORKER-PORT%" (str port)}
+         sub-fn (fn [s] 
+                  (reduce (fn [string entry]
+                    (apply clojure.string/replace string entry))
+                     s replacement-map))]
+    (if-not (nil? childopts)
+      (if (sequential? childopts)
+        (map sub-fn childopts)
+        (-> childopts sub-fn (.split " ")))
+      nil)
+    ))
 
 (defn java-cmd []
   (let [java-home (.get (System/getenv) "JAVA_HOME")]
@@ -462,21 +576,25 @@
 (defmethod launch-worker
     :distributed [supervisor storm-id port worker-id]
     (let [conf (:conf supervisor)
+          run-worker-as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)
           storm-home (System/getProperty "storm.home")
           stormroot (supervisor-stormdist-root conf storm-id)
           jlp (jlp stormroot conf)
           stormjar (supervisor-stormjar-path stormroot)
           storm-conf (read-supervisor-storm-conf conf storm-id)
           classpath (add-to-classpath (current-classpath) [stormjar])
-          worker-childopts (when-let [s (conf WORKER-CHILDOPTS)]
-                             (substitute-worker-childopts s port))
-          topo-worker-childopts (when-let [s (storm-conf TOPOLOGY-WORKER-CHILDOPTS)]
-                                  (substitute-worker-childopts s port))
-          logfilename (str "worker-" port ".log")
+          top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
+          gc-opts (substitute-childopts (if top-gc-opts top-gc-opts (conf WORKER-GC-CHILDOPTS)) worker-id storm-id port)
+          user (storm-conf TOPOLOGY-SUBMITTER-USER)
+          logfilename (logs-filename storm-id port)
+
+          worker-childopts (substitute-childopts (conf WORKER-CHILDOPTS) worker-id storm-id port)
+          topo-worker-childopts (substitute-childopts (storm-conf TOPOLOGY-WORKER-CHILDOPTS) worker-id storm-id port)
           command (concat
                     [(java-cmd) "-server"]
                     worker-childopts
                     topo-worker-childopts
+                    gc-opts
                     [(str "-Djava.library.path=" jlp)
                      (str "-Dlogfile.name=" logfilename)
                      (str "-Dstorm.home=" storm-home)
@@ -490,13 +608,21 @@
                      (:assignment-id supervisor)
                      port
                      worker-id])
-          command (->> command (map str) (filter (complement empty?)))
-          shell-cmd (->> command
-                         (map #(str \' (clojure.string/escape % {\' "\\'"}) \'))
-                         (clojure.string/join " "))]
-      (log-message "Launching worker with command: " shell-cmd)
-      (launch-process command :environment {"LD_LIBRARY_PATH" jlp})
-      ))
+          command (->> command (map str) (filter (complement empty?)))]
+
+      (log-message "Launching worker with command: " (shell-cmd command))
+      (write-log-metadata! storm-conf user worker-id storm-id port conf)
+      (set-worker-user! conf worker-id user)
+      (let [log-prefix (str "Worker Process " worker-id)
+           callback (fn [exit-code] 
+                          (log-message log-prefix " exited with code: " exit-code)
+                          (add-dead-worker worker-id))]
+        (remove-dead-worker worker-id) 
+        (if run-worker-as-user
+          (let [worker-dir (worker-root conf worker-id)]
+            (worker-launcher conf user ["worker" worker-dir (write-script worker-dir command :environment {"LD_LIBRARY_PATH" jlp})] :log-prefix log-prefix :exit-code-callback callback))
+          (launch-process command :environment {"LD_LIBRARY_PATH" jlp} :log-prefix log-prefix :exit-code-callback callback)
+      ))))
 
 ;; local implementation
 
@@ -536,6 +662,7 @@
                                    (:assignment-id supervisor)
                                    port
                                    worker-id)]
+      (set-worker-user! conf worker-id "")
       (psim/register-process pid worker)
       (swap! (:worker-thread-pids-atom supervisor) assoc worker-id pid)
       ))
diff --git a/storm-core/src/clj/backtype/storm/daemon/task.clj b/storm-core/src/clj/backtype/storm/daemon/task.clj
index 3650150..6a61cea 100644
--- a/storm-core/src/clj/backtype/storm/daemon/task.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/task.clj
@@ -126,12 +126,9 @@
         emit-sampler (mk-stats-sampler storm-conf)
         stream->component->grouper (:stream->component->grouper executor-data)
         user-context (:user-context task-data)
-        executor-stats (:stats executor-data)
-        debug? (= true (storm-conf TOPOLOGY-DEBUG))]
+        executor-stats (:stats executor-data)]
         
     (fn ([^Integer out-task-id ^String stream ^List values]
-          (when debug?
-            (log-message "Emitting direct: " out-task-id "; " component-id " " stream " " values))
           (let [target-component (.getComponentId worker-context out-task-id)
                 component->grouping (get stream->component->grouper stream)
                 grouping (get component->grouping target-component)
@@ -148,8 +145,6 @@
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
-           (when debug?
-             (log-message "Emitting: " component-id " " stream " " values))
            (let [out-tasks (ArrayList.)]
              (fast-map-iter [[out-component grouper] (get stream->component->grouper stream)]
                (when (= :direct grouper)
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index 16765d9..d09ea35 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -22,6 +22,9 @@
   (:import [backtype.storm.utils TransferDrainer])
   (:import [backtype.storm.messaging TransportFactory])
   (:import [backtype.storm.messaging TaskMessage IContext IConnection])
+  (:import [backtype.storm.security.auth AuthUtils])
+  (:import [javax.security.auth Subject])
+  (:import [java.security PrivilegedExceptionAction])
   (:gen-class))
 
 (bootstrap)
@@ -108,29 +111,42 @@
               (log-warn "Received invalid messages for unknown tasks. Dropping... ")
               )))))))
 
+(defn- assert-can-serialize [^KryoTupleSerializer serializer tuple-batch]
+  "Check that all of the tuples can be serialized by serializing them."
+  (fast-list-iter [[task tuple :as pair] tuple-batch]
+    (.serialize serializer tuple)))
+
 (defn mk-transfer-fn [worker]
   (let [local-tasks (-> worker :task-ids set)
         local-transfer (:transfer-local-fn worker)
         ^DisruptorQueue transfer-queue (:transfer-queue worker)
-        task->node+port (:cached-task->node+port worker)]
-    (fn [^KryoTupleSerializer serializer tuple-batch]
-      (let [local (ArrayList.)
-            remoteMap (HashMap.)]
-        (fast-list-iter [[task tuple :as pair] tuple-batch]
-          (if (local-tasks task)
-            (.add local pair)
-            
-            ;;Using java objects directly to avoid performance issues in java code
-            (let [node+port (get @task->node+port task)]
-              (when (not (.get remoteMap node+port))
-                (.put remoteMap node+port (ArrayList.)))
-              (let [remote (.get remoteMap node+port)]
-                (.add remote (TaskMessage. task (.serialize serializer tuple)))
-                 ))))
-        
-        (local-transfer local)
-        (disruptor/publish transfer-queue remoteMap)
-          ))))
+        task->node+port (:cached-task->node+port worker)
+        try-serialize-local ((:conf worker) TOPOLOGY-TESTING-ALWAYS-TRY-SERIALIZE)
+        transfer-fn
+          (fn [^KryoTupleSerializer serializer tuple-batch]
+            (let [local (ArrayList.)
+                  remoteMap (HashMap.)]
+              (fast-list-iter [[task tuple :as pair] tuple-batch]
+                (if (local-tasks task)
+                  (.add local pair) 
+
+                  ;;Using java objects directly to avoid performance issues in java code
+                  (let [node+port (get @task->node+port task)]
+                    (when (not (.get remoteMap node+port))
+                      (.put remoteMap node+port (ArrayList.)))
+                    (let [remote (.get remoteMap node+port)]
+                      (.add remote (TaskMessage. task (.serialize serializer tuple)))
+                     )))) 
+                (local-transfer local)
+                (disruptor/publish transfer-queue remoteMap)
+              ))]
+    (if try-serialize-local
+      (do 
+        (log-warn "WILL TRY TO SERIALIZE ALL TUPLES (Turn off " TOPOLOGY-TESTING-ALWAYS-TRY-SERIALIZE " for production)")
+        (fn [^KryoTupleSerializer serializer tuple-batch]
+          (assert-can-serialize serializer tuple-batch)
+          (transfer-fn serializer tuple-batch)))
+      transfer-fn)))
 
 (defn- mk-receive-queue-map [storm-conf executors]
   (->> executors
@@ -174,21 +190,17 @@
                        )
             :timer-name timer-name))
 
-(defn worker-data [conf mq-context storm-id assignment-id port worker-id]
-  (let [cluster-state (cluster/mk-distributed-cluster-state conf)
-        storm-cluster-state (cluster/mk-storm-cluster-state cluster-state)
-        storm-conf (read-supervisor-storm-conf conf storm-id)
-        executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port))
-        transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
-                                                  :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
-        executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
-        
-        receive-queue-map (->> executor-receive-queue-map
-                               (mapcat (fn [[e queue]] (for [t (executor-id->tasks e)] [t queue])))
-                               (into {}))
-
-        topology (read-supervisor-topology conf storm-id)]
-    (recursive-map
+(defn recursive-map-worker-data [conf mq-context storm-id assignment-id port
+                                  storm-conf
+                                  worker-id 
+                                  cluster-state 
+                                  storm-cluster-state
+                                  executors 
+                                  transfer-queue
+                                  executor-receive-queue-map 
+                                  receive-queue-map
+                                  topology]
+  (recursive-map
       :conf conf
       :mq-context (if mq-context
                       mq-context
@@ -207,6 +219,7 @@
       :system-topology (system-topology! storm-conf topology)
       :heartbeat-timer (mk-halting-timer "heartbeat-timer")
       :refresh-connections-timer (mk-halting-timer "refresh-connections-timer")
+      :refresh-credentials-timer (mk-halting-timer "refresh-credentials-timer")
       :refresh-active-timer (mk-halting-timer "refresh-active-timer")
       :executor-heartbeat-timer (mk-halting-timer "executor-heartbeat-timer")
       :user-timer (mk-halting-timer "user-timer")
@@ -230,7 +243,31 @@
       :transfer-local-fn (mk-transfer-local-fn <>)
       :receiver-thread-count (get storm-conf WORKER-RECEIVER-THREAD-COUNT)
       :transfer-fn (mk-transfer-fn <>)
-      )))
+      ))
+
+(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state]
+  (let [executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port))
+        transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
+                                                  :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
+        executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
+        
+        receive-queue-map (->> executor-receive-queue-map
+                               (mapcat (fn [[e queue]] (for [t (executor-id->tasks e)] [t queue])))
+                               (into {}))
+
+        topology (read-supervisor-topology conf storm-id)]
+        (recursive-map-worker-data 
+          conf mq-context storm-id assignment-id port
+          storm-conf
+          worker-id 
+          cluster-state 
+          storm-cluster-state
+          executors 
+          transfer-queue
+          executor-receive-queue-map 
+          receive-queue-map
+          topology)
+  ))
 
 (defn- endpoint->string [[node port]]
   (str port "/" node))
@@ -339,6 +376,11 @@
     (.shutdownNow (get dr WorkerTopologyContext/SHARED_EXECUTOR))
     (log-message "Shut down default resources")))
 
+(defn- override-login-config-with-system-property [conf]
+  (if-let [login_conf_file (System/getProperty "java.security.auth.login.config")]
+    (assoc conf "java.security.auth.login.config" login_conf_file)
+    conf))
+
 ;; TODO: should worker even take the storm-id as input? this should be
 ;; deducable from cluster state (by searching through assignments)
 ;; what about if there's inconsistency in assignments? -> but nimbus
@@ -353,7 +395,17 @@
   ;; process. supervisor will register it in this case
   (when (= :distributed (cluster-mode conf))
     (touch (worker-pid-path conf worker-id (process-pid))))
-  (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id)
+  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
+        storm-conf (override-login-config-with-system-property storm-conf)
+        acls (Utils/getWorkerACL storm-conf)
+        cluster-state (cluster/mk-distributed-cluster-state conf :auth-conf storm-conf :acls acls)
+        storm-cluster-state (cluster/mk-storm-cluster-state cluster-state :acls acls)
+        initial-credentials (.credentials storm-cluster-state storm-id nil)
+        auto-creds (AuthUtils/GetAutoCredentials storm-conf)
+        subject (AuthUtils/populateSubject nil auto-creds initial-credentials)]
+      (Subject/doAs subject (reify PrivilegedExceptionAction 
+        (run [this]
+          (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state)
         heartbeat-fn #(do-heartbeat worker)
 
         ;; do this here so that the worker process dies if this fails
@@ -366,13 +418,12 @@
         _ (schedule-recurring (:heartbeat-timer worker) 0 (conf WORKER-HEARTBEAT-FREQUENCY-SECS) heartbeat-fn)
         _ (schedule-recurring (:executor-heartbeat-timer worker) 0 (conf TASK-HEARTBEAT-FREQUENCY-SECS) #(do-executor-heartbeats worker :executors @executors))
 
-        
         refresh-connections (mk-refresh-connections worker)
 
         _ (refresh-connections nil)
         _ (refresh-storm-active worker nil)
  
-        _ (reset! executors (dofor [e (:executors worker)] (executor/mk-executor worker e)))
+        _ (reset! executors (dofor [e (:executors worker)] (executor/mk-executor worker e initial-credentials)))
         receive-thread-shutdown (launch-receive-thread worker)
         
         transfer-tuples (mk-transfer-tuples-handler worker)
@@ -403,6 +454,7 @@
                     (log-message "Shut down transfer thread")
                     (cancel-timer (:heartbeat-timer worker))
                     (cancel-timer (:refresh-connections-timer worker))
+                    (cancel-timer (:refresh-credentials-timer worker))
                     (cancel-timer (:refresh-active-timer worker))
                     (cancel-timer (:executor-heartbeat-timer worker))
                     (cancel-timer (:user-timer worker))
@@ -426,19 +478,29 @@
                (and
                  (timer-waiting? (:heartbeat-timer worker))
                  (timer-waiting? (:refresh-connections-timer worker))
+                 (timer-waiting? (:refresh-credentials-timer worker))
                  (timer-waiting? (:refresh-active-timer worker))
                  (timer-waiting? (:executor-heartbeat-timer worker))
                  (timer-waiting? (:user-timer worker))
                  ))
-             )]
-    
+             )
+        credentials (atom initial-credentials)
+        check-credentials-changed (fn []
+                                    (let [new-creds (.credentials (:storm-cluster-state worker) storm-id nil)]
+                                      (when-not (= new-creds @credentials) ;;This does not have to be atomic, worst case we update when one is not needed
+                                        (AuthUtils/updateSubject subject auto-creds new-creds)
+                                        (dofor [e @executors] (.credentials-changed e new-creds))
+                                        (reset! credentials new-creds))))
+      ]
+    (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed)))
+    (schedule-recurring (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS) check-credentials-changed)
     (schedule-recurring (:refresh-connections-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) refresh-connections)
     (schedule-recurring (:refresh-active-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) (partial refresh-storm-active worker))
 
     (log-message "Worker has topology config " (:storm-conf worker))
     (log-message "Worker " worker-id " for storm " storm-id " on " assignment-id ":" port " has finished loading")
     ret
-    ))
+    ))))))
 
 (defmethod mk-suicide-fn
   :local [conf]
diff --git a/storm-core/src/clj/backtype/storm/testing.clj b/storm-core/src/clj/backtype/storm/testing.clj
index 32f7f88..39f3759 100644
--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -187,10 +187,15 @@
 
 (defmacro while-timeout [timeout-ms condition & body]
   `(let [end-time# (+ (System/currentTimeMillis) ~timeout-ms)]
+     (log-debug "Looping until " '~condition)
      (while ~condition
        (when (> (System/currentTimeMillis) end-time#)
-         (throw (AssertionError. (str "Test timed out (" ~timeout-ms "ms)"))))
-       ~@body)))
+         (let [thread-dump# (Utils/threadDump)]
+           (log-message "Condition " '~condition  " not met in " ~timeout-ms "ms")
+           (log-message thread-dump#)
+           (throw (AssertionError. (str "Test timed out (" ~timeout-ms "ms) " '~condition)))))
+       ~@body)
+     (log-debug "Condition met " '~condition)))
 
 (defn wait-until-cluster-waiting
   "Wait until the cluster is idle. Should be used with time simulation."
@@ -204,7 +209,7 @@
                   ; because a worker may already be dead
                   workers)]
     (while-timeout TEST-TIMEOUT-MS (not (every? (memfn waiting?) daemons))
-                   (Thread/sleep 10)
+                   (Thread/sleep (rand-int 20))
                    ;;      (doseq [d daemons]
                    ;;        (if-not ((memfn waiting?) d)
                    ;;          (println d)))
@@ -278,7 +283,9 @@
 (defn mk-capture-launch-fn [capture-atom]
   (fn [supervisor storm-id port worker-id]
     (let [supervisor-id (:supervisor-id supervisor)
+          conf (:conf supervisor)
           existing (get @capture-atom [supervisor-id port] [])]
+      (set-worker-user! conf worker-id "")
       (swap! capture-atom assoc [supervisor-id port] (conj existing storm-id)))))
 
 (defn find-worker-id
@@ -474,6 +481,9 @@
     (submit-local-topology (:nimbus cluster-map) storm-name storm-conf topology)
 
     (let [storm-id (common/get-storm-id state storm-name)]
+      ;;Give the topology time to come up without using it to wait for the spouts to complete
+      (simulate-wait cluster-map)
+
       (while-timeout TEST-TIMEOUT-MS (not (every? exhausted? (spout-objects spouts)))
                      (simulate-wait cluster-map))
 
@@ -575,19 +585,18 @@
   ([tracked-topology]
    (tracked-wait tracked-topology 1))
   ([tracked-topology amt]
-   (let [target (+ amt @(:last-spout-emit tracked-topology))
-         track-id (-> tracked-topology :cluster ::track-id)
-         waiting? (fn []
-                    (or (not= target (global-amt track-id "spout-emitted"))
-                        (not= (global-amt track-id "transferred")
-                              (global-amt track-id "processed"))
-                        ))]
-     (while-timeout TEST-TIMEOUT-MS (waiting?)
-                    ;; (println "Spout emitted: " (global-amt track-id "spout-emitted"))
-                    ;; (println "Processed: " (global-amt track-id "processed"))
-                    ;; (println "Transferred: " (global-amt track-id "transferred"))
-                    (Thread/sleep 500))
-     (reset! (:last-spout-emit tracked-topology) target))))
+    (let [target (+ amt @(:last-spout-emit tracked-topology))
+          track-id (-> tracked-topology :cluster ::track-id)
+          waiting? (fn []
+                     (or (not= target (global-amt track-id "spout-emitted"))
+                         (not= (global-amt track-id "transferred")                                 
+                               (global-amt track-id "processed"))))]
+      (while-timeout TEST-TIMEOUT-MS (waiting?)
+                     ;; (println "Spout emitted: " (global-amt track-id "spout-emitted"))
+                     ;; (println "Processed: " (global-amt track-id "processed"))
+                     ;; (println "Transferred: " (global-amt track-id "transferred"))
+                    (Thread/sleep (rand-int 200)))
+      (reset! (:last-spout-emit tracked-topology) target))))
 
 (defnk test-tuple
   [values
@@ -619,3 +628,10 @@
                   (HashMap.)
                   (atom false))]
     (TupleImpl. context values 1 stream)))
+
+(defmacro with-timeout
+  [millis unit & body]
+  `(let [f# (future ~@body)]
+     (try
+       (.get f# ~millis ~unit)
+       (finally (future-cancel f#)))))
diff --git a/storm-core/src/clj/backtype/storm/testing4j.clj b/storm-core/src/clj/backtype/storm/testing4j.clj
index ff939a9..1ce0b23 100644
--- a/storm-core/src/clj/backtype/storm/testing4j.clj
+++ b/storm-core/src/clj/backtype/storm/testing4j.clj
@@ -15,7 +15,7 @@
 ;; limitations under the License.
 (ns backtype.storm.testing4j
   (:import [java.util Map List Collection ArrayList])
-  (:require [backtype.storm LocalCluster])
+  (:require [backtype.storm [LocalCluster :as LocalCluster]])
   (:import [backtype.storm Config ILocalCluster LocalCluster])
   (:import [backtype.storm.generated StormTopology])
   (:import [backtype.storm.daemon nimbus])
diff --git a/storm-core/src/clj/backtype/storm/thrift.clj b/storm-core/src/clj/backtype/storm/thrift.clj
index f778d23..ce0a5ff 100644
--- a/storm-core/src/clj/backtype/storm/thrift.clj
+++ b/storm-core/src/clj/backtype/storm/thrift.clj
@@ -21,13 +21,12 @@
             ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
             GlobalStreamId ComponentObject ComponentObject$_Fields
             ShellComponent])
-  (:import [backtype.storm.utils Utils])
+  (:import [backtype.storm.utils Utils NimbusClient])
   (:import [backtype.storm Constants])
   (:import [backtype.storm.grouping CustomStreamGrouping])
   (:import [backtype.storm.topology TopologyBuilder])
   (:import [backtype.storm.clojure RichShellBolt RichShellSpout])
-  (:import [org.apache.thrift.protocol TBinaryProtocol TProtocol])
-  (:import [org.apache.thrift.transport TTransport TFramedTransport TSocket])
+  (:import [org.apache.thrift.transport TTransport])
   (:use [backtype.storm util config log]))
 
 (defn instantiate-java-object
@@ -69,27 +68,26 @@
 (defn nimbus-client-and-conn
   [host port]
   (log-message "Connecting to Nimbus at " host ":" port)
-  (let [transport (TFramedTransport. (TSocket. host port))
-        prot (TBinaryProtocol. transport)
-        client (Nimbus$Client. prot)]
-    (.open transport)
-    [client transport]))
+  (let [conf (read-storm-config)
+        nimbusClient (NimbusClient. conf host port nil)
+        client (.getClient nimbusClient)
+        transport (.transport nimbusClient)]
+        [client transport] ))
 
 (defmacro with-nimbus-connection
   [[client-sym host port] & body]
-  `(let [[^Nimbus$Client ~client-sym ^TTransport conn#]
-         (nimbus-client-and-conn ~host ~port)]
-     (try
-       ~@body
-       (finally (.close conn#)))))
+  `(let [[^Nimbus$Client ~client-sym ^TTransport conn#] (nimbus-client-and-conn ~host ~port)]
+    (try
+      ~@body
+    (finally (.close conn#)))))
 
 (defmacro with-configured-nimbus-connection
   [client-sym & body]
   `(let [conf# (read-storm-config)
          host# (conf# NIMBUS-HOST)
          port# (conf# NIMBUS-THRIFT-PORT)]
-     (with-nimbus-connection [~client-sym host# port#]
-       ~@body )))
+    (with-nimbus-connection [~client-sym host# port#]
+      ~@body)))
 
 (defn direct-output-fields
   [fields]
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index 68bc4c0..e177ca7 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -23,13 +23,14 @@
   (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID
                                               ACKER-ACK-STREAM-ID ACKER-FAIL-STREAM-ID system-id?]]])
   (:use [ring.adapter.jetty :only [run-jetty]])
-  (:use [clojure.string :only [trim]])
+  (:use [clojure.string :only [blank? lower-case trim]])
   (:import [backtype.storm.utils Utils])
   (:import [backtype.storm.generated ExecutorSpecificStats
             ExecutorStats ExecutorSummary TopologyInfo SpoutStats BoltStats
             ErrorInfo ClusterSummary SupervisorSummary TopologySummary
             Nimbus$Client StormTopology GlobalStreamId RebalanceOptions
             KillOptions])
+  (:import [backtype.storm.security.auth AuthUtils])
   (:import [java.io File])
   (:require [compojure.route :as route]
             [compojure.handler :as handler]
@@ -46,6 +47,34 @@
      [~nimbus-sym (*STORM-CONF* NIMBUS-HOST) (*STORM-CONF* NIMBUS-THRIFT-PORT)]
      ~@body))
 
+(defn authorized-ui-user?
+  [user conf topology-conf]
+  (let [ui-users (concat (conf UI-USERS)
+                         (conf NIMBUS-ADMINS)
+                         (topology-conf UI-USERS)
+                         (topology-conf TOPOLOGY-USERS))]
+    (or (blank? (conf UI-FILTER))
+        (and (not (blank? user))
+          (some #(= % user) ui-users)))))
+
+(defn assert-authorized-ui-user
+  [user conf topology-conf]
+  (if (not (authorized-ui-user? user conf topology-conf))
+    ;;TODO need a better exception here so the UI can appear better
+    (throw (RuntimeException. (str "User " user " is not authorized.")))))
+
+(defn- ui-actions-enabled?
+  []
+  (= "true" (lower-case (*STORM-CONF* UI-ACTIONS-ENABLED))))
+
+(defn assert-authorized-topology-user
+  [user]
+  ;;TODO eventually we will want to use the Authorizatin handler from nimbus, but for now
+  ;; Disable the calls conditionally
+  (if (not (ui-actions-enabled?))
+    ;;TODO need a better exception here so the UI can appear better
+    (throw (RuntimeException. (str "Topology actions for the UI have been disabled")))))
+
 (defn get-filled-stats
   [summs]
   (->> summs
@@ -264,9 +293,10 @@
               (bolt-comp-summs id))]
     (sort-by #(-> ^ExecutorSummary % .get_executor_info .get_task_start) ret)))
 
-(defn worker-log-link [host port]
-  (url-format "http://%s:%s/log?file=worker-%s.log"
-              host (*STORM-CONF* LOGVIEWER-PORT) port))
+(defn worker-log-link [host port topology-id]
+  (let [fname (logs-filename topology-id port)]
+    (url-format (str "http://%s:%s/log?file=%s")
+          host (*STORM-CONF* LOGVIEWER-PORT) fname)))
 
 (defn compute-executor-capacity
   [^ExecutorSummary e]
@@ -423,8 +453,9 @@
                                  :checked (is-ack-stream (get m :stream))}))))))]
     (map (fn [row]
            {:row row}) (partition 4 4 nil streams))))
-
-(defn mk-visualization-data [id window include-sys?]
+  
+(defn mk-visualization-data
+  [id window include-sys? user]
   (with-nimbus
     nimbus
     (let [window (if window window ":all-time")
@@ -439,22 +470,23 @@
           bolt-comp-summs (group-by-comp bolt-summs)
           bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?)
                                       bolt-comp-summs)
-          topology-conf (from-json
+          topology-conf (from-json 
                           (.getTopologyConf ^Nimbus$Client nimbus id))]
-      (visualization-data
-        (merge (hashmap-to-persistent spouts)
-               (hashmap-to-persistent bolts))
-        spout-comp-summs bolt-comp-summs window id))))
+      (assert-authorized-ui-user user *STORM-CONF* topology-conf)
+      (visualization-data 
+       (merge (hashmap-to-persistent spouts)
+              (hashmap-to-persistent bolts))
+       spout-comp-summs bolt-comp-summs window id))))
 
 (defn cluster-configuration []
   (with-nimbus nimbus
     (.getNimbusConf ^Nimbus$Client nimbus)))
 
 (defn cluster-summary
-  ([]
+  ([user]
      (with-nimbus nimbus
-        (cluster-summary (.getClusterInfo ^Nimbus$Client nimbus))))
-  ([^ClusterSummary summ]
+        (cluster-summary (.getClusterInfo ^Nimbus$Client nimbus) user)))
+  ([^ClusterSummary summ user]
      (let [sups (.get_supervisors summ)
         used-slots (reduce + (map #(.get_num_used_workers ^SupervisorSummary %) sups))
         total-slots (reduce + (map #(.get_num_workers ^SupervisorSummary %) sups))
@@ -465,14 +497,15 @@
         total-executors (->> (.get_topologies summ)
                              (map #(.get_num_executors ^TopologySummary %))
                              (reduce +))]
-       {"stormVersion" (read-storm-version)
+       {"user" user
+        "stormVersion" (read-storm-version)
         "nimbusUptime" (pretty-uptime-sec (.get_nimbus_uptime_secs summ))
         "supervisors" (count sups)
         "slotsTotal" total-slots
         "slotsUsed"  used-slots
         "slotsFree" free-slots
         "executorsTotal" total-executors
-        "tasksTotal" total-tasks})))
+        "tasksTotal" total-tasks })))
 
 (defn supervisor-summary
   ([]
@@ -498,12 +531,14 @@
    {"topologies"
     (for [^TopologySummary t summs]
       {"id" (.get_id t)
+       "owner" (.get_owner t)
        "name" (.get_name t)
        "status" (.get_status t)
        "uptime" (pretty-uptime-sec (.get_uptime_secs t))
        "tasksTotal" (.get_num_tasks t)
        "workersTotal" (.get_num_workers t)
-       "executorsTotal" (.get_num_executors t)})}))
+       "executorsTotal" (.get_num_executors t)
+       "schedulerInfo" (.get_sched_status t)})}))
 
 (defn topology-stats [id window stats]
   (let [times (stats-times (:emitted stats))
@@ -559,12 +594,14 @@
         workers (set (for [^ExecutorSummary e executors]
                        [(.get_host e) (.get_port e)]))]
       {"id" (.get_id summ)
+       "owner" (.get_owner summ)
        "name" (.get_name summ)
        "status" (.get_status summ)
        "uptime" (pretty-uptime-sec (.get_uptime_secs summ))
        "tasksTotal" (sum-tasks executors)
        "workersTotal" (count workers)
-       "executorsTotal" (count executors)}))
+       "executorsTotal" (count executors)
+       "schedulerInfo" (.get_sched_status summ)}))
 
 (defn spout-summary-json [topology-id id stats window]
   (let [times (stats-times (:emitted stats))
@@ -580,7 +617,7 @@
         "acked" (get-in stats [:acked k])
         "failed" (get-in stats [:failed k])})))
 
-(defn topology-page [id window include-sys?]
+(defn topology-page [id window include-sys? user]
   (with-nimbus nimbus
     (let [window (if window window ":all-time")
           window-hint (window-hint window)
@@ -603,16 +640,19 @@
                                               bolt-comp-summs
                                               window
                                               id)]
+      (assert-authorized-ui-user user *STORM-CONF* topology-conf)
       (merge
        (topology-summary summ)
-       {"window" window
+       {"user" user
+        "window" window
         "windowHint" window-hint
         "msgTimeout" msg-timeout
         "topologyStats" (topology-stats id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
         "spouts" (spout-comp id spout-comp-summs (.get_errors summ) window include-sys?)
         "bolts" (bolt-comp id bolt-comp-summs (.get_errors summ) window include-sys?)
         "configuration" topology-conf
-        "visualizationTable" (stream-boxes visualizer-data)}))))
+        "visualizationTable" (stream-boxes visualizer-data)
+        "uiActionsEnabled" (ui-actions-enabled?)}))))
 
 (defn spout-output-stats
   [stream-summary window]
@@ -644,7 +684,7 @@
      "completeLatency" (float-str (:complete-latencies stats))
      "acked" (nil-to-zero (:acked stats))
      "failed" (nil-to-zero (:failed stats))
-     "workerLogLink" (worker-log-link (.get_host e) (.get_port e))}))
+     "workerLogLink" (worker-log-link (.get_host e) (.get_port e) topology-id)}))
 
 (defn component-errors
   [errors-list]
@@ -737,7 +777,7 @@
      "processLatency" (float-str (:process-latencies stats))
      "acked" (nil-to-zero (:acked stats))
      "failed" (nil-to-zero (:failed stats))
-     "workerLogLink" (worker-log-link (.get_host e) (.get_port e))}))
+     "workerLogLink" (worker-log-link (.get_host e) (.get_port e) topology-id)}))
 
 (defn bolt-stats
   [window ^TopologyInfo topology-info component executors include-sys?]
@@ -752,18 +792,21 @@
                        (.get_id topology-info) executors window include-sys?)}))
 
 (defn component-page
-  [topology-id component window include-sys?]
+  [topology-id component window include-sys? user]
   (with-nimbus nimbus
     (let [window (if window window ":all-time")
           summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
           topology (.getTopology ^Nimbus$Client nimbus topology-id)
+          topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id))
           type (component-type topology component)
           summs (component-task-summs summ topology component)
           spec (cond (= type :spout) (spout-stats window summ component summs include-sys?)
                      (= type :bolt) (bolt-stats window summ component summs include-sys?))
           errors (component-errors (get (.get_errors summ) component))]
+      (assert-authorized-ui-user user *STORM-CONF* topology-conf)
       (merge
-       {"id" component
+        {"user" user
+         "id" component
          "name" (.get_name summ)
          "executors" (count summs)
          "tasks" (sum-tasks summs)
@@ -782,57 +825,72 @@
    :headers {"Content-Type" "application/json"}
    :body (to-json data)})
 
+(def http-creds-handler (AuthUtils/GetUiHttpCredentialsPlugin *STORM-CONF*))
+
 (defroutes main-routes
   (GET "/api/v1/cluster/configuration" []
        (cluster-configuration))
-  (GET "/api/v1/cluster/summary" []
-       (json-response (cluster-summary)))
+  (GET "/api/v1/cluster/summary" [:as {:keys [cookies servlet-request]}]
+       (let [user (.getUserName http-creds-handler servlet-request)]
+         (json-response (cluster-summary user))))
   (GET "/api/v1/supervisor/summary" []
        (json-response (supervisor-summary)))
   (GET "/api/v1/topology/summary" []
        (json-response (all-topologies-summary)))
-  (GET  "/api/v1/topology/:id" [id & m]
-        (let [id (url-decode id)]
-          (json-response (topology-page id (:window m) (check-include-sys? (:sys m))))))
+  (GET  "/api/v1/topology/:id" [:as {:keys [cookies servlet-request]} id & m]
+        (let [id (url-decode id)
+              user (.getUserName http-creds-handler servlet-request)]
+          (json-response (topology-page id (:window m) (check-include-sys? (:sys m)) user))))
   (GET "/api/v1/topology/:id/visualization" [:as {:keys [cookies servlet-request]} id & m]
-       (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m)))))
-  (GET "/api/v1/topology/:id/component/:component" [id component & m]
+        (let [id (url-decode id)
+              user (.getUserName http-creds-handler servlet-request)]
+          (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m)) user))))
+  (GET "/api/v1/topology/:id/component/:component" [:as {:keys [cookies servlet-request]} id component & m]
        (let [id (url-decode id)
-             component (url-decode component)]
-         (json-response (component-page id component (:window m) (check-include-sys? (:sys m))))))
-  (POST "/api/v1/topology/:id/activate" [id]
+             component (url-decode component)
+             user (.getUserName http-creds-handler servlet-request)]
+         (json-response (component-page id component (:window m) (check-include-sys? (:sys m)) user))))
+  (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id]
     (with-nimbus nimbus
       (let [id (url-decode id)
             tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-            name (.get_name tplg)]
+            name (.get_name tplg)
+            user (.getUserName http-creds-handler servlet-request)]
+        (assert-authorized-topology-user user)
         (.activate nimbus name)
         (log-message "Activating topology '" name "'")))
     (resp/redirect (str "/api/v1/topology/" id)))
 
-  (POST "/api/v1/topology/:id/deactivate" [id]
-    (with-nimbus nimbus
-      (let [id (url-decode id)
-            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-            name (.get_name tplg)]
-        (.deactivate nimbus name)
-        (log-message "Deactivating topology '" name "'")))
-    (resp/redirect (str "/api/v1/topology/" id)))
-  (POST "/api/v1/topology/:id/rebalance/:wait-time" [id wait-time]
+  (POST "/api/v1/topology/:id/deactivate" [:as {:keys [cookies servlet-request]} id]
     (with-nimbus nimbus
       (let [id (url-decode id)
             tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
             name (.get_name tplg)
-            options (RebalanceOptions.)]
+            user (.getUserName http-creds-handler servlet-request)]
+        (assert-authorized-topology-user user)
+        (.deactivate nimbus name)
+        (log-message "Deactivating topology '" name "'")))
+    (resp/redirect (str "/api/v1/topology/" id)))
+  (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
+    (with-nimbus nimbus
+      (let [id (url-decode id)
+            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+            name (.get_name tplg)
+            options (RebalanceOptions.)
+            user (.getUserName http-creds-handler servlet-request)]
+        (assert-authorized-topology-user user)
         (.set_wait_secs options (Integer/parseInt wait-time))
         (.rebalance nimbus name options)
         (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
     (resp/redirect (str "/api/v1/topology/" id)))
-  (POST "/api/v1/topology/:id/kill/:wait-time" [id wait-time]
+  (POST "/api/v1/topology/:id/kill/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
     (with-nimbus nimbus
       (let [id (url-decode id)
             tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
             name (.get_name tplg)
-            options (KillOptions.)]
+            options (KillOptions.)
+            user (.getUserName http-creds-handler servlet-request)]
+        (assert-authorized-topology-user user)
         (.set_wait_secs options (Integer/parseInt wait-time))
         (.killTopologyWithOpts nimbus name options)
         (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
@@ -866,7 +924,18 @@
 
 (defn start-server!
   []
-  (run-jetty app {:port (Integer. (*STORM-CONF* UI-PORT))
-                  :join? false}))
+  (try
+    (let [conf *STORM-CONF*
+          header-buffer-size (int (.get conf UI-HEADER-BUFFER-BYTES))
+          filters-confs [{:filter-class (conf UI-FILTER)
+                          :filter-params (conf UI-FILTER-PARAMS)}]]
+      (run-jetty app {:port (conf UI-PORT)
+                          :join? false
+                          :configurator (fn [server]
+                                          (doseq [connector (.getConnectors server)]
+                                            (.setHeaderBufferSize connector header-buffer-size))
+                                          (config-filter server app filters-confs))}))
+   (catch Exception ex
+     (log-error ex))))
 
 (defn -main [] (start-server!))
diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj
index c2de7c7..8f339b7 100644
--- a/storm-core/src/clj/backtype/storm/ui/helpers.clj
+++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj
@@ -16,10 +16,15 @@
 (ns backtype.storm.ui.helpers
   (:use compojure.core)
   (:use [hiccup core page-helpers])
-  (:use [clojure [string :only [join]]])
-  (:use [backtype.storm.util :only [uuid defnk url-encode]])
+  (:use [clojure
+         [string :only [blank? join]]
+         [walk :only [keywordize-keys]]])
+  (:use [backtype.storm config log])
+  (:use [backtype.storm.util :only [clojurify-structure uuid defnk url-encode]])
   (:use [clj-time coerce format])
   (:import [backtype.storm.generated ExecutorInfo ExecutorSummary])
+  (:import [org.mortbay.jetty.security SslSocketConnector])
+  (:require [ring.util servlet])
   (:require [compojure.route :as route]
             [compojure.handler :as handler]))
 
@@ -146,3 +151,40 @@
 
 (defn pretty-executor-info [^ExecutorInfo e]
   (str "[" (.get_task_start e) "-" (.get_task_end e) "]"))
+
+(defn unauthorized-user-html [user]
+  [[:h2 "User '" (escape-html user) "' is not authorized."]])
+
+(defn- mk-ssl-connector [port ks-path ks-password ks-type]
+  (doto (SslSocketConnector.)
+    (.setExcludeCipherSuites (into-array String ["SSL_RSA_WITH_RC4_128_MD5" "SSL_RSA_WITH_RC4_128_SHA"]))
+    (.setAllowRenegotiate false)
+    (.setKeystore ks-path)
+    (.setKeystoreType ks-type)
+    (.setKeyPassword ks-password)
+    (.setPassword ks-password)
+    (.setPort port)))
+
+(defn config-ssl [server port ks-path ks-password ks-type]
+  (when (> port 0)
+    (.addConnector server (mk-ssl-connector port ks-path ks-password ks-type))))
+
+(defn config-filter [server handler filters-confs]
+  (if filters-confs
+    (let [servlet-holder (org.mortbay.jetty.servlet.ServletHolder.
+                           (ring.util.servlet/servlet handler))
+          context (doto (org.mortbay.jetty.servlet.Context. server "/")
+                    (.addServlet servlet-holder "/"))]
+      (doseq [{:keys [filter-name filter-class filter-params]} filters-confs]
+        (if filter-class
+          (let [filter-holder (doto (org.mortbay.jetty.servlet.FilterHolder.)
+                                (.setClassName filter-class)
+                                (.setName (or filter-name filter-class))
+                                (.setInitParameters (or filter-params {})))]
+            (.addFilter context filter-holder "/*" org.mortbay.jetty.Handler/ALL))))
+      (.addHandler server context))))
+
+(defn ring-response-from-exception [ex]
+  {:headers {}
+   :status 400
+   :body (.getMessage ex)})
diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj
index 91b0713..ed2e30b 100644
--- a/storm-core/src/clj/backtype/storm/util.clj
+++ b/storm-core/src/clj/backtype/storm/util.clj
@@ -25,12 +25,15 @@
   (:import [java.util.zip ZipFile])
   (:import [java.util.concurrent.locks ReentrantReadWriteLock])
   (:import [java.util.concurrent Semaphore])
-  (:import [java.io File FileOutputStream StringWriter PrintWriter IOException])
+  (:import [java.io File FileOutputStream RandomAccessFile StringWriter
+            PrintWriter BufferedReader InputStreamReader IOException])
   (:import [java.lang.management ManagementFactory])
   (:import [org.apache.commons.exec DefaultExecutor CommandLine])
   (:import [org.apache.commons.io FileUtils])
   (:import [org.apache.commons.exec ExecuteException])
   (:import [org.json.simple JSONValue])
+  (:import [org.yaml.snakeyaml Yaml]
+           [org.yaml.snakeyaml.constructor SafeConstructor])
   (:require [clojure [string :as str]])
   (:import [clojure.lang RT])
   (:require [clojure [set :as set]])
@@ -395,12 +398,17 @@
     (catch ExecuteException e
       (log-message "Error when trying to kill " pid ". Process is probably already dead."))))
 
-(defnk launch-process [command :environment {}]
-  (let [builder (ProcessBuilder. command)
-        process-env (.environment builder)]
-    (doseq [[k v] environment]
-      (.put process-env k v))
-    (.start builder)))
+(defn read-and-log-stream
+  [prefix stream]
+  (try
+    (let [reader (BufferedReader. (InputStreamReader. stream))]
+      (loop []
+        (if-let [line (.readLine reader)]
+                (do
+                  (log-warn (str prefix ":" line))
+                  (recur)))))
+    (catch IOException e
+      (log-warn "Error while trying to log stream" e))))
 
 (defn sleep-secs [secs]
   (when (pos? secs)
@@ -460,6 +468,42 @@
         [this]
         (Time/isThreadWaiting thread)))))
 
+(defn shell-cmd
+  [command]
+  (->> command
+    (map #(str \' (clojure.string/escape % {\' "'\"'\"'"}) \'))
+      (clojure.string/join " ")))
+
+(defnk write-script
+  [dir command :environment {}]
+  (let [script-src (str "#!/bin/bash\n" (clojure.string/join "" (map (fn [[k v]] (str (shell-cmd ["export" (str k "=" v)]) ";\n")) environment)) "\nexec " (shell-cmd command) ";")
+        script-path (str dir "/storm-worker-script.sh")
+        - (spit script-path script-src)]
+    script-path
+  ))
+
+(defnk launch-process
+  [command :environment {} :log-prefix nil :exit-code-callback nil]
+  (let [builder (ProcessBuilder. command)
+        process-env (.environment builder)]
+    (.redirectErrorStream builder true)
+    (doseq [[k v] environment]
+      (.put process-env k v))
+    (let [process (.start builder)]
+      (if (or log-prefix exit-code-callback)
+        (async-loop
+         (fn []
+           (if log-prefix
+             (read-and-log-stream log-prefix (.getInputStream process)))
+           (when exit-code-callback
+             (try
+               (.waitFor process)
+               (catch InterruptedException e
+                 (log-message log-prefix " interrupted.")))
+             (exit-code-callback (.exitValue process)))
+           nil)))                    
+      process)))
+   
 (defn exists-file?
   [path]
   (.exists (File. path)))
@@ -922,5 +966,34 @@
               (list form x)))
   ([x form & more] `(-<> (-<> ~x ~form) ~@more)))
 
+(def LOG-DIR
+  (.getCanonicalPath 
+                (clojure.java.io/file (System/getProperty "storm.home") "logs")))
+
+(defn- logs-rootname [storm-id port]
+  (str storm-id "-worker-" port))
+
+(defn logs-filename [storm-id port]
+  (str (logs-rootname storm-id port) ".log"))
+
+(defn logs-metadata-filename [storm-id port]
+  (str (logs-rootname storm-id port) ".yaml"))
+
+(def worker-log-filename-pattern #"((.*-\d+-\d+)-worker-(\d+)).log")
+
+(defn get-log-metadata-file
+  ([fname]
+    (if-let [[_ _ id port] (re-matches worker-log-filename-pattern fname)]
+      (get-log-metadata-file id port)))
+  ([id port]
+    (clojure.java.io/file LOG-DIR "metadata" (logs-metadata-filename id port))))
+
+(defn clojure-from-yaml-file [yamlFile]
+  (try
+    (let [obj (.load (Yaml. (SafeConstructor.)) (java.io.FileReader. yamlFile))]
+      (clojurify-structure obj))
+    (catch Exception ex
+      (log-error ex))))
+
 (defn hashmap-to-persistent [^HashMap m]
   (zipmap (.keySet m) (.values m)))
diff --git a/storm-core/src/clj/backtype/storm/zookeeper.clj b/storm-core/src/clj/backtype/storm/zookeeper.clj
index 46d1c69..dc9472c 100644
--- a/storm-core/src/clj/backtype/storm/zookeeper.clj
+++ b/storm-core/src/clj/backtype/storm/zookeeper.clj
@@ -85,12 +85,13 @@
    :sequential CreateMode/PERSISTENT_SEQUENTIAL})
 
 (defn create-node
-  ([^CuratorFramework zk ^String path ^bytes data mode]
-   (try
-     (.. zk (create) (withMode (zk-create-modes mode)) (withACL ZooDefs$Ids/OPEN_ACL_UNSAFE) (forPath (normalize-path path) data))
-     (catch Exception e (throw (wrap-in-runtime e)))))
-  ([^CuratorFramework zk ^String path ^bytes data]
-   (create-node zk path data :persistent)))
+  ([^CuratorFramework zk ^String path ^bytes data mode acls]
+    (let [mode  (zk-create-modes mode)]
+      (try
+        (.. zk (create) (withMode mode) (withACL acls) (forPath (normalize-path path) data))
+        (catch Exception e (throw (wrap-in-runtime e))))))
+  ([^CuratorFramework zk ^String path ^bytes data acls]
+    (create-node zk path data :persistent acls)))
 
 (defn exists-node?
   [^CuratorFramework zk ^String path watch?]
@@ -109,12 +110,12 @@
              (catch Exception e (throw (wrap-in-runtime e)))))
 
 (defn mkdirs
-  [^CuratorFramework zk ^String path]
+  [^CuratorFramework zk ^String path acls]
   (let [path (normalize-path path)]
     (when-not (or (= path "/") (exists-node? zk path false))
-      (mkdirs zk (parent-path path))
+      (mkdirs zk (parent-path path) acls)
       (try-cause
-        (create-node zk path (barr 7) :persistent)
+        (create-node zk path (barr 7) :persistent acls)
         (catch KeeperException$NodeExistsException e
           ;; this can happen when multiple clients doing mkdir at same time
           ))
diff --git a/storm-core/src/clj/storm/trident/testing.clj b/storm-core/src/clj/storm/trident/testing.clj
index 3207173..ac5fcab 100644
--- a/storm-core/src/clj/storm/trident/testing.clj
+++ b/storm-core/src/clj/storm/trident/testing.clj
@@ -16,6 +16,7 @@
 (ns storm.trident.testing
   (:require [backtype.storm.LocalDRPC :as LocalDRPC])
   (:import [storm.trident.testing FeederBatchSpout FeederCommitterBatchSpout MemoryMapState MemoryMapState$Factory TuplifyArgs])
+  (:require [backtype.storm [LocalDRPC]])
   (:import [backtype.storm LocalDRPC])
   (:import [backtype.storm.tuple Fields])
   (:import [backtype.storm.generated KillOptions])
diff --git a/storm-core/src/dev/drpc-simple-acl-test-scenario.yaml b/storm-core/src/dev/drpc-simple-acl-test-scenario.yaml
new file mode 100644
index 0000000..82c03c0
--- /dev/null
+++ b/storm-core/src/dev/drpc-simple-acl-test-scenario.yaml
@@ -0,0 +1,11 @@
+# For the function "jump", alice can perform client operations, and bob can
+# perform invocation operations.
+drpc.authorizer.acl:
+  "jump":
+    "client.users":
+      - "alice"
+      - "bob"
+    "invocation.user": "charlie"
+  "partial":
+    "client.users":
+      - "alice"
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index ff309a5..3b3f7e5 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -20,6 +20,7 @@
 import backtype.storm.ConfigValidation;
 import backtype.storm.serialization.IKryoDecorator;
 import backtype.storm.serialization.IKryoFactory;
+import backtype.storm.utils.Utils;
 import com.esotericsoftware.kryo.Serializer;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -44,6 +45,13 @@
  */
 public class Config extends HashMap<String, Object> {
     /**
+     * This is part of a temporary workaround to a ZK bug, it is the 'scheme:acl' for
+     * the user Nimbus and Supervisors use to authenticate with ZK.
+     */
+    public static final String STORM_ZOOKEEPER_SUPERACL = "storm.zookeeper.superACL";
+    public static final Object STORM_ZOOKEEPER_SUPERACL_SCHEMA = String.class;
+
+    /**
      * The transporter for communication among Storm tasks
      */
     public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport";
@@ -144,7 +152,13 @@
     public static final Object STORM_LOCAL_HOSTNAME_SCHEMA = String.class;
 
     /**
-     * The transport plug-in for Thrift client/server communication
+     * The plugin that will convert a principal to a local user.
+     */
+    public static final String STORM_PRINCIPAL_TO_LOCAL_PLUGIN = "storm.principal.tolocal";
+    public static final Object STORM_PRINCIPAL_TO_LOCAL_PLUGIN_SCHEMA = String.class;
+
+    /**
+     * The default transport plug-in for Thrift client/server communication
      */
     public static final String STORM_THRIFT_TRANSPORT_PLUGIN = "storm.thrift.transport";
     public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
@@ -157,6 +171,13 @@
     public static final Object TOPOLOGY_TUPLE_SERIALIZER_SCHEMA = String.class;
 
     /**
+     * Try to serialize all tuples, even for local transfers.  This should only be used
+     * for testing, as a sanity check that all of your tuples are setup properly.
+     */
+    public static final String TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE = "topology.testing.always.try.serialize";
+    public static final Object TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE_SCHEMA = Boolean.class;
+
+    /**
      * Whether or not to use ZeroMQ for messaging in local mode. If this is set 
      * to false, then Storm will use a pure-Java messaging system. The purpose 
      * of this flag is to make it easy to run Storm in local mode by eliminating 
@@ -204,30 +225,72 @@
     public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING_SCHEMA = Number.class;
 
     /**
-     * The Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
+     * The cluster Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
      */
     public static final String STORM_ZOOKEEPER_AUTH_SCHEME="storm.zookeeper.auth.scheme";
     public static final Object STORM_ZOOKEEPER_AUTH_SCHEME_SCHEMA = String.class;
 
     /**
-     * A string representing the payload for Zookeeper authentication. It gets serialized using UTF-8 encoding during authentication.
+     * A string representing the payload for cluster Zookeeper authentication.
+     * It gets serialized using UTF-8 encoding during authentication.
+     * Note that if this is set to something with a secret (as when using
+     * digest authentication) then it should only be set in the
+     * storm-cluster-auth.yaml file.
+     * This file storm-cluster-auth.yaml should then be protected with
+     * appropriate permissions that deny access from workers.
      */
     public static final String STORM_ZOOKEEPER_AUTH_PAYLOAD="storm.zookeeper.auth.payload";
     public static final Object STORM_ZOOKEEPER_AUTH_PAYLOAD_SCHEMA = String.class;
 
     /**
+     * The topology Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
+     */
+    public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME="storm.zookeeper.topology.auth.scheme";
+    public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME_SCHEMA = String.class;
+
+    /**
+     * A string representing the payload for topology Zookeeper authentication. It gets serialized using UTF-8 encoding during authentication.
+     */
+    public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD="storm.zookeeper.topology.auth.payload";
+    public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD_SCHEMA = String.class;
+
+    /**
      * The id assigned to a running topology. The id is the storm name with a unique nonce appended.
      */
     public static final String STORM_ID = "storm.id";
     public static final Object STORM_ID_SCHEMA = String.class;
 
     /**
+     * The number of times to retry a Nimbus operation.
+     */
+    public static final String STORM_NIMBUS_RETRY_TIMES="storm.nimbus.retry.times";
+    public static final Object STORM_NIMBUS_RETRY_TIMES_SCHEMA = Number.class;
+    
+    /**
+     * The starting interval between exponential backoff retries of a Nimbus operation.
+     */
+    public static final String STORM_NIMBUS_RETRY_INTERVAL="storm.nimbus.retry.interval.millis";
+    public static final Object STORM_NIMBUS_RETRY_INTERVAL_SCHEMA = Number.class;
+
+    /**
+     * The ceiling of the interval between retries of a client connect to Nimbus operation.
+     */
+    public static final String STORM_NIMBUS_RETRY_INTERVAL_CEILING="storm.nimbus.retry.intervalceiling.millis";
+    public static final Object STORM_NIMBUS_RETRY_INTERVAL_CEILING_SCHEMA = Number.class;
+
+    /**
      * The host that the master server is running on.
      */
     public static final String NIMBUS_HOST = "nimbus.host";
     public static final Object NIMBUS_HOST_SCHEMA = String.class;
 
     /**
+     * The Nimbus transport plug-in for Thrift client/server communication
+     */
+    public static final String NIMBUS_THRIFT_TRANSPORT_PLUGIN = "nimbus.thrift.transport";
+    public static final Object NIMBUS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
+
+    /**
      * Which port the Thrift interface of Nimbus should run on. Clients should
      * connect to this port to upload jars and submit topologies.
      */
@@ -235,12 +298,32 @@
     public static final Object NIMBUS_THRIFT_PORT_SCHEMA = Number.class;
 
     /**
+     * The number of threads that should be used by the nimbus thrift server.
+     */
+    public static final String NIMBUS_THRIFT_THREADS = "nimbus.thrift.threads";
+    public static final Object NIMBUS_THRIFT_THREADS_SCHEMA = Number.class;
+
+    /**
+     * A list of users that are cluster admins and can run any command.  To use this set
+     * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer  
+     */
+    public static final String NIMBUS_ADMINS = "nimbus.admins";
+    public static final Object NIMBUS_ADMINS_SCHEMA = ConfigValidation.StringsValidator;
+
+    /**
+     * 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 backtype.storm.security.auth.authorizer.SimpleACLAuthorizer  
+     */
+    public static final String NIMBUS_SUPERVISOR_USERS = "nimbus.supervisor.users";
+    public static final Object NIMBUS_SUPERVISOR_USERS_SCHEMA = ConfigValidation.StringsValidator;
+
+    /**
      * The maximum buffer size thrift should use when reading messages.
      */
     public static final String NIMBUS_THRIFT_MAX_BUFFER_SIZE = "nimbus.thrift.max_buffer_size";
     public static final Object NIMBUS_THRIFT_MAX_BUFFER_SIZE_SCHEMA = Number.class;
 
-
     /**
      * This parameter is used by the storm-deploy project to configure the
      * jvm options for the nimbus daemon.
@@ -331,6 +414,18 @@
     public static final Object NIMBUS_AUTHORIZER_SCHEMA = String.class;
 
     /**
+     * How often nimbus should wake up to renew credentials if needed.
+     */
+    public static final String NIMBUS_CREDENTIAL_RENEW_FREQ_SECS = "nimbus.credential.renewers.freq.secs";
+    public static final Object NIMBUS_CREDENTIAL_RENEW_FREQ_SECS_SCHEMA = Number.class;
+
+    /**
+     * A list of credential renewers that nimbus should load.
+     */
+    public static final String NIMBUS_CREDENTIAL_RENEWERS = "nimbus.credential.renewers.classes";
+    public static final Object NIMBUS_CREDENTIAL_RENEWERS_SCHEMA = ConfigValidation.StringsValidator;
+
+    /**
      * Storm UI binds to this port.
      */
     public static final String UI_PORT = "ui.port";
@@ -349,6 +444,24 @@
     public static final Object LOGVIEWER_CHILDOPTS_SCHEMA = String.class;
 
     /**
+     * How often to clean up old log files
+     */
+    public static final String LOGVIEWER_CLEANUP_INTERVAL_SECS = "logviewer.cleanup.interval.secs";
+    public static final Object LOGVIEWER_CLEANUP_INTERVAL_SECS_SCHEMA = ConfigValidation.PositiveIntegerValidator;
+
+    /**
+     * How many minutes since a log was last modified for the log to be considered for clean-up
+     */
+    public static final String LOGVIEWER_CLEANUP_AGE_MINS = "logviewer.cleanup.age.mins";
+    public static final Object LOGVIEWER_CLEANUP_AGE_MINS_SCHEMA = ConfigValidation.PositiveIntegerValidator;
+
+    /**
+     * A list of users allowed to view logs via the Log Viewer 
+     */
+    public static final String LOGS_USERS = "logs.users";
+    public static final Object LOGS_USERS_SCHEMA = ConfigValidation.StringsValidator;
+
+    /**
      * Appender name used by log viewer to determine log directory.
      */
     public static final String LOGVIEWER_APPENDER_NAME = "logviewer.appender.name";
@@ -361,36 +474,156 @@
     public static final Object UI_CHILDOPTS_SCHEMA = String.class;
 
     /**
+     * A class implementing javax.servlet.Filter for authenticating/filtering UI requests
+     */
+    public static final String UI_FILTER = "ui.filter";
+    public static final Object UI_FILTER_SCHEMA = String.class;
+
+    /**
+     * Initialization parameters for the javax.servlet.Filter
+     */
+    public static final String UI_FILTER_PARAMS = "ui.filter.params";
+    public static final Object UI_FILTER_PARAMS_SCHEMA = Map.class;
+
+    /**
+     * The size of the header buffer for the UI in bytes
+     */
+    public static final String UI_HEADER_BUFFER_BYTES = "ui.header.buffer.bytes";
+    public static final Object UI_HEADER_BUFFER_BYTES_SCHEMA = Number.class;
+
+    /**
+     * A list of users allowed to view topologies via the UI
+     */
+    public static final String UI_USERS = "ui.users";
+    public static final Object UI_USERS_SCHEMA = ConfigValidation.StringsValidator;
+
+    /**
+     * Whether or not actions should be enabled.  When disabled, requests to
+     * modify the state of topologies via HTTP will not be honored.
+     *
+     * Defaults to true.
+     */
+    public static final String UI_ACTIONS_ENABLED = "ui.actions.enabled";
+    public static final Object UI_ACTIONS_ENABLED_SCHEMA = Boolean.class;
+
+    /**
      * List of DRPC servers so that the DRPCSpout knows who to talk to.
      */
     public static final String DRPC_SERVERS = "drpc.servers";
     public static final Object DRPC_SERVERS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
+     * This port is used by Storm DRPC for receiving HTTP DPRC requests from clients.
+     */
+    public static final String DRPC_HTTP_PORT = "drpc.http.port";
+    public static final Object DRPC_HTTP_PORT_SCHEMA = Number.class;
+
+    /**
+     * This port is used by Storm DRPC for receiving HTTPS (SSL) DPRC requests from clients.
+     */
+    public static final String DRPC_HTTPS_PORT = "drpc.https.port";
+    public static final Object DRPC_HTTPS_PORT_SCHEMA = Number.class;
+
+    /**
+     * Path to the keystore used by Storm DRPC for setting up HTTPS (SSL).
+     */
+    public static final String DRPC_HTTPS_KEYSTORE_PATH = "drpc.https.keystore.path";
+    public static final Object DRPC_HTTPS_KEYSTORE_PATH_SCHEMA = String.class;
+
+    /**
+     * Password to the keystore used by Storm DRPC for setting up HTTPS (SSL).
+     */
+    public static final String DRPC_HTTPS_KEYSTORE_PASSWORD = "drpc.https.keystore.password";
+    public static final Object DRPC_HTTPS_KEYSTORE_PASSWORD_SCHEMA = String.class;
+
+    /**
+     * Type of keystore used by Storm DRPC for setting up HTTPS (SSL).
+     * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details.
+     */
+    public static final String DRPC_HTTPS_KEYSTORE_TYPE = "drpc.https.keystore.type";
+    public static final Object DRPC_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class;
+
+    /**
+     * The DRPC transport plug-in for Thrift client/server communication
+     */
+    public static final String DRPC_THRIFT_TRANSPORT_PLUGIN = "drpc.thrift.transport";
+    public static final Object DRPC_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
+
+    /**
      * This port is used by Storm DRPC for receiving DPRC requests from clients.
      */
     public static final String DRPC_PORT = "drpc.port";
     public static final Object DRPC_PORT_SCHEMA = Number.class;
 
     /**
+     * Class name for authorization plugin for DRPC client
+     */
+    public static final String DRPC_AUTHORIZER = "drpc.authorizer";
+    public static final Object DRPC_AUTHORIZER_SCHEMA = String.class;
+
+    /**
+     * The Access Control List for the DRPC Authorizer.
+     * @see DRPCSimpleAclAuthorizer
+     */
+    public static final String DRPC_AUTHORIZER_ACL = "drpc.authorizer.acl";
+    public static final Object DRPC_AUTHORIZER_ACL_SCHEMA = Map.class;
+
+    /**
+     * File name of the DRPC Authorizer ACL.
+     * @see DRPCSimpleAclAuthorizer
+     */
+    public static final String DRPC_AUTHORIZER_ACL_FILENAME = "drpc.authorizer.acl.filename";
+    public static final Object DRPC_AUTHORIZER_ACL_FILENAME_SCHEMA = String.class;
+
+    /**
+     * Whether the DRPCSimpleAclAuthorizer should deny requests for operations
+     * involving functions that have no explicit ACL entry. When set to false
+     * (the default) DRPC functions that have no entry in the ACL will be
+     * permitted, which is appropriate for a development environment. When set
+     * to true, explicit ACL entries are required for every DRPC function, and
+     * any request for functions will be denied.
+     * @see DRPCSimpleAclAuthorizer
+     */
+    public static final String DRPC_AUTHORIZER_ACL_STRICT = "drpc.authorizer.acl.strict";
+    public static final Object DRPC_AUTHORIZER_ACL_STRICT_SCHEMA = Boolean.class;
+
+    /**
      * DRPC thrift server worker threads 
      */
     public static final String DRPC_WORKER_THREADS = "drpc.worker.threads";
     public static final Object DRPC_WORKER_THREADS_SCHEMA = Number.class;
 
     /**
+     * The maximum buffer size thrift should use when reading messages for DRPC.
+     */
+    public static final String DRPC_MAX_BUFFER_SIZE = "drpc.max_buffer_size";
+    public static final Object DRPC_MAX_BUFFER_SIZE_SCHEMA = Number.class;
+
+    /**
      * DRPC thrift server queue size 
      */
     public static final String DRPC_QUEUE_SIZE = "drpc.queue.size";
     public static final Object DRPC_QUEUE_SIZE_SCHEMA = Number.class;
 
     /**
+     * The DRPC invocations transport plug-in for Thrift client/server communication
+     */
+    public static final String DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN = "drpc.invocations.thrift.transport";
+    public static final Object DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
+
+    /**
      * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back. 
      */
     public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port";
     public static final Object DRPC_INVOCATIONS_PORT_SCHEMA = Number.class;
 
     /**
+     * DRPC invocations thrift server worker threads 
+     */
+    public static final String DRPC_INVOCATIONS_THREADS = "drpc.invocations.threads";
+    public static final Object DRPC_INVOCATIONS_THREADS_SCHEMA = Number.class;
+
+    /**
      * The timeout on DRPC requests within the DRPC server. Defaults to 10 minutes. Note that requests can also
      * timeout based on the socket timeout on the DRPC client, and separately based on the topology message
      * timeout for the topology implementing the DRPC function.
@@ -405,7 +638,19 @@
     public static final Object DRPC_CHILDOPTS_SCHEMA = String.class;
 
     /**
-     * the metadata configed on the supervisor
+     * Class name of the HTTP credentials plugin for the UI.
+     */
+    public static final String UI_HTTP_CREDS_PLUGIN = "ui.http.creds.plugin";
+    public static final Object UI_HTTP_CREDS_PLUGIN_SCHEMA = String.class;
+
+    /**
+     * Class name of the HTTP credentials plugin for DRPC.
+     */
+    public static final String DRPC_HTTP_CREDS_PLUGIN = "drpc.http.creds.plugin";
+    public static final Object DRPC_HTTP_CREDS_PLUGIN_SCHEMA = String.class;
+
+    /**
+     * the metadata configured on the supervisor
      */
     public static final String SUPERVISOR_SCHEDULER_META = "supervisor.scheduler.meta";
     public static final Object SUPERVISOR_SCHEDULER_META_SCHEMA = Map.class;
@@ -416,7 +661,31 @@
      */
     public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
     public static final Object SUPERVISOR_SLOTS_PORTS_SCHEMA = ConfigValidation.NumbersValidator;
+    
+    /**
+     * A number representing the maximum number of workers any single topology can acquire.
+     */
+    public static final String NIMBUS_SLOTS_PER_TOPOLOGY = "nimbus.slots.perTopology";
+    public static final Object NIMBUS_SLOTS_PER_TOPOLOGY_SCHEMA = Number.class;
 
+    /**
+     * A class implementing javax.servlet.Filter for DRPC HTTP requests
+     */
+    public static final String DRPC_HTTP_FILTER = "drpc.http.filter";
+    public static final Object DRPC_HTTP_FILTER_SCHEMA = String.class;
+
+    /**
+     * Initialization parameters for the javax.servlet.Filter of the DRPC HTTP
+     * service
+     */
+    public static final String DRPC_HTTP_FILTER_PARAMS = "drpc.http.filter.params";
+    public static final Object DRPC_HTTP_FILTER_PARAMS_SCHEMA = Map.class;
+
+    /**
+     * A number representing the maximum number of executors any single topology can acquire.
+     */
+    public static final String NIMBUS_EXECUTORS_PER_TOPOLOGY = "nimbus.executors.perTopology";
+    public static final Object NIMBUS_EXECUTORS_PER_TOPOLOGY_SCHEMA = Number.class;
 
     /**
      * This parameter is used by the storm-deploy project to configure the
@@ -425,7 +694,6 @@
     public static final String SUPERVISOR_CHILDOPTS = "supervisor.childopts";
     public static final Object SUPERVISOR_CHILDOPTS_SCHEMA = String.class;
 
-
     /**
      * How long a worker can go without heartbeating before the supervisor tries to
      * restart the worker process.
@@ -433,7 +701,6 @@
     public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
     public static final Object SUPERVISOR_WORKER_TIMEOUT_SECS_SCHEMA = Number.class;
 
-
     /**
      * How long a worker can go without heartbeating during the initial launch before
      * the supervisor tries to restart the worker process. This value override
@@ -443,7 +710,6 @@
     public static final String SUPERVISOR_WORKER_START_TIMEOUT_SECS = "supervisor.worker.start.timeout.secs";
     public static final Object SUPERVISOR_WORKER_START_TIMEOUT_SECS_SCHEMA = Number.class;
 
-
     /**
      * Whether or not the supervisor should launch workers assigned to it. Defaults
      * to true -- and you should probably never change this value. This configuration
@@ -452,7 +718,6 @@
     public static final String SUPERVISOR_ENABLE = "supervisor.enable";
     public static final Object SUPERVISOR_ENABLE_SCHEMA = Boolean.class;
 
-
     /**
      * how often the supervisor sends a heartbeat to the master.
      */
@@ -468,13 +733,35 @@
     public static final Object SUPERVISOR_MONITOR_FREQUENCY_SECS_SCHEMA = Number.class;
 
     /**
+     * Should the supervior try to run the worker as the lauching user or not.  Defaults to false. 
+     */
+    public static final String SUPERVISOR_RUN_WORKER_AS_USER = "supervisor.run.worker.as.user";
+    public static final Object SUPERVISOR_RUN_WORKER_AS_USER_SCHEMA = Boolean.class;
+
+    /**
+     * Full path to the worker-laucher executable that will be used to lauch workers when 
+     * SUPERVISOR_RUN_WORKER_AS_USER is set to true.
+     */ 
+    public static final String SUPERVISOR_WORKER_LAUNCHER = "supervisor.worker.launcher";
+    public static final Object SUPERVISOR_WORKER_LAUNCHER_SCHEMA = String.class;
+
+    /**
      * The jvm opts provided to workers launched by this supervisor. All "%ID%" substrings are replaced
-     * with an identifier for this worker.
+     * with an identifier for this worker. Also, "%WORKER-ID%", "%STORM-ID%" and "%WORKER-PORT%" are
+     * replaced with appropriate runtime values for this worker.
      */
     public static final String WORKER_CHILDOPTS = "worker.childopts";
     public static final Object WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
 
     /**
+     * The jvm opts provided to workers launched by this supervisor for GC. All "%ID%" substrings are replaced
+     * with an identifier for this worker.  Because the JVM complains about multiple GC opts the topology
+     * can override this default value by setting topology.worker.gc.childopts.
+     */
+    public static final String WORKER_GC_CHILDOPTS = "worker.gc.childopts";
+    public static final Object WORKER_GC_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
+
+    /**
      * control how many worker receiver threads we need per worker
      */
     public static final String WORKER_RECEIVER_THREAD_COUNT = "topology.worker.receiver.thread.count";
@@ -504,6 +791,19 @@
     public static final Object TASK_REFRESH_POLL_SECS_SCHEMA = Number.class;
 
 
+    /**
+     * How often a task should sync credentials, worst case.
+     */
+    public static final String TASK_CREDENTIALS_POLL_SECS = "task.credentials.poll.secs";
+    public static final Object TASK_CREDENTIALS_POLL_SECS_SCHEMA = Number.class;
+
+
+    /**
+     * A list of users that are allowed to interact with the topology.  To use this set
+     * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer  
+     */
+    public static final String TOPOLOGY_USERS = "topology.users";
+    public static final Object TOPOLOGY_USERS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * True if Storm should timeout messages or not. Defaults to true. This is meant to be used
@@ -682,6 +982,12 @@
     public static final Object TOPOLOGY_WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
 
     /**
+     * Topology-specific options GC for the worker child process. This overrides WORKER_GC_CHILDOPTS.
+     */
+    public static final String TOPOLOGY_WORKER_GC_CHILDOPTS="topology.worker.gc.childopts";
+    public static final Object TOPOLOGY_WORKER_GC_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
+
+    /**
      * This config is available for TransactionalSpouts, and contains the id ( a String) for
      * the transactional topology. This id is used to store the state of the transactional
      * topology in Zookeeper.
@@ -769,10 +1075,34 @@
     /**
      * Name of the topology. This config is automatically set by Storm when the topology is submitted.
      */
-    public static final String TOPOLOGY_NAME="topology.name";
+    public final static String TOPOLOGY_NAME="topology.name";  
     public static final Object TOPOLOGY_NAME_SCHEMA = String.class;
 
     /**
+     * The principal who submitted a topology
+     */
+    public final static String TOPOLOGY_SUBMITTER_PRINCIPAL = "topology.submitter.principal";
+    public static final Object TOPOLOGY_SUBMITTER_PRINCIPAL_SCHEMA = String.class;
+
+    /**
+     * The local user name of the user who submitted a topology.
+     */
+    public static final String TOPOLOGY_SUBMITTER_USER = "topology.submitter.user";
+    public static final Object TOPOLOGY_SUBMITTER_USER_SCHEMA = String.class;
+    
+    /**
+     * Array of components that scheduler should try to place on separate hosts.
+     */
+    public static final String TOPOLOGY_SPREAD_COMPONENTS = "topology.spread.components";
+    public static final Object TOPOLOGY_SPREAD_COMPONENTS_SCHEMA = ConfigValidation.StringsValidator;
+   
+    /**
+     * A list of IAutoCredentials that the topology should load and use.
+     */
+    public static final String TOPOLOGY_AUTO_CREDENTIALS = "topology.auto-credentials";
+    public static final Object TOPOLOGY_AUTO_CREDENTIALS_SCHEMA = ConfigValidation.StringsValidator;
+ 
+    /**
      * Max pending tuples in one ShellBolt
      */
     public static final String TOPOLOGY_SHELLBOLT_MAX_PENDING="topology.shellbolt.max.pending";
@@ -841,8 +1171,22 @@
      * to backtype.storm.scheduler.IsolationScheduler to make use of the isolation scheduler.
      */
     public static final String ISOLATION_SCHEDULER_MACHINES = "isolation.scheduler.machines";
-    public static final Object ISOLATION_SCHEDULER_MACHINES_SCHEMA = Map.class;
+    public static final Object ISOLATION_SCHEDULER_MACHINES_SCHEMA = ConfigValidation.MapOfStringToNumberValidator;
+    
+    /**
+     * A map from the user name to the number of machines that should that user is allowed to use. Set storm.scheduler
+     * to backtype.storm.scheduler.multitenant.MultitenantScheduler
+     */
+    public static final String MULTITENANT_SCHEDULER_USER_POOLS = "multitenant.scheduler.user.pools";
+    public static final Object MULTITENANT_SCHEDULER_USER_POOLS_SCHEMA = ConfigValidation.MapOfStringToNumberValidator;
 
+    /**
+     * The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler
+     * to backtype.storm.scheduler.multitenant.MultitenantScheduler
+     */
+    public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines";
+    public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class;
+    
     public static void setDebug(Map conf, boolean isOn) {
         conf.put(Config.TOPOLOGY_DEBUG, isOn);
     } 
diff --git a/storm-core/src/jvm/backtype/storm/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/ConfigValidation.java
index 3accb82..e990921 100644
--- a/storm-core/src/jvm/backtype/storm/ConfigValidation.java
+++ b/storm-core/src/jvm/backtype/storm/ConfigValidation.java
@@ -18,6 +18,8 @@
 package backtype.storm;
 import java.util.Map;
 
+import java.util.Map;
+
 /**
  * Provides functionality for validating configuration fields.
  */
@@ -35,51 +37,147 @@
          */
         public void validateField(String name, Object field) throws IllegalArgumentException;
     }
+    
+    /**
+     * Declares a method for validating configuration values that is nestable.
+     */
+    public static abstract class NestableFieldValidator implements FieldValidator {
+        @Override
+        public void validateField(String name, Object field) throws IllegalArgumentException {
+            validateField(null, name, field);
+        }
+        
+        /**
+         * Validates the given field.
+         * @param pd describes the parent wrapping this validator.
+         * @param name the name of the field.
+         * @param field The field to be validated.
+         * @throws IllegalArgumentException if the field fails validation.
+         */
+        public abstract void validateField(String pd, String name, Object field) throws IllegalArgumentException;
+    }
 
     /**
-     * Returns a new FieldValidator for a List of the given Class.
-     * @param cls the Class of elements composing the list
-     * @return a FieldValidator for a list of the given class
+     * Returns a new NestableFieldValidator for a given class.
+     * @param cls the Class the field should be a type of
+     * @param nullAllowed whether or not a value of null is valid
+     * @return a NestableFieldValidator for that class
      */
-    static FieldValidator FieldListValidatorFactory(final Class cls) {
-        return new FieldValidator() {
+    public static NestableFieldValidator fv(final Class cls, final boolean nullAllowed) {
+        return new NestableFieldValidator() {
             @Override
-            public void validateField(String name, Object field)
+            public void validateField(String pd, String name, Object field)
                     throws IllegalArgumentException {
-                if (field == null) {
-                    // A null value is acceptable.
+                if (nullAllowed && field == null) {
+                    return;
+                }
+                if (! cls.isInstance(field)) {
+                    throw new IllegalArgumentException(
+                        pd + name + " must be a " + cls.getName() + ". ("+field+")");
+                }
+            }
+        };
+    }
+    
+    /**
+     * Returns a new NestableFieldValidator for a List of the given Class.
+     * @param cls the Class of elements composing the list
+     * @param nullAllowed whether or not a value of null is valid
+     * @return a NestableFieldValidator for a list of the given class
+     */
+    public static NestableFieldValidator listFv(Class cls, boolean nullAllowed) {
+      return listFv(fv(cls, false), nullAllowed);
+    }
+    
+    /**
+     * Returns a new NestableFieldValidator for a List where each item is validated by validator.
+     * @param validator used to validate each item in the list
+     * @param nullAllowed whether or not a value of null is valid
+     * @return a NestableFieldValidator for a list with each item validated by a different validator.
+     */
+    public static NestableFieldValidator listFv(final NestableFieldValidator validator, 
+            final boolean nullAllowed) {
+        return new NestableFieldValidator() {
+            @Override
+            public void validateField(String pd, String name, Object field)
+                    throws IllegalArgumentException {
+                if (nullAllowed && field == null) {
                     return;
                 }
                 if (field instanceof Iterable) {
                     for (Object e : (Iterable)field) {
-                        if (! cls.isInstance(e)) {
-                            throw new IllegalArgumentException(
-                                    "Each element of the list " + name + " must be a " +
-                                    cls.getName() + ".");
-                        }
+                        validator.validateField(pd + "Each element of the list ", name, e);
                     }
                     return;
                 }
                 throw new IllegalArgumentException(
-                        "Field " + name + " must be an Iterable of " + cls.getName());
+                        "Field " + name + " must be an Iterable but was " +
+                        ((field == null) ? "null" :  ("a " + field.getClass())));
             }
         };
     }
 
     /**
+     * Returns a new NestableFieldValidator for a Map of key to val.
+     * @param key the Class of keys in the map
+     * @param val the Class of values in the map
+     * @param nullAllowed whether or not a value of null is valid
+     * @return a NestableFieldValidator for a Map of key to val
+     */
+    public static NestableFieldValidator mapFv(Class key, Class val, 
+            boolean nullAllowed) {
+        return mapFv(fv(key, false), fv(val, false), nullAllowed);
+    }
+ 
+    /**
+     * Returns a new NestableFieldValidator for a Map.
+     * @param key a validator for the keys in the map
+     * @param val a validator for the values in the map
+     * @param nullAllowed whether or not a value of null is valid
+     * @return a NestableFieldValidator for a Map
+     */   
+    public static NestableFieldValidator mapFv(final NestableFieldValidator key, 
+            final NestableFieldValidator val, final boolean nullAllowed) {
+        return new NestableFieldValidator() {
+            @SuppressWarnings("unchecked")
+            @Override
+            public void validateField(String pd, String name, Object field)
+                    throws IllegalArgumentException {
+                if (nullAllowed && field == null) {
+                    return;
+                }
+                if (field instanceof Map) {
+                    for (Map.Entry<Object, Object> entry: ((Map<Object, Object>)field).entrySet()) {
+                      key.validateField("Each key of the map ", name, entry.getKey());
+                      val.validateField("Each value in the map ", name, entry.getValue());
+                    }
+                    return;
+                }
+                throw new IllegalArgumentException(
+                        "Field " + name + " must be a Map");
+            }
+        };
+    }
+    
+    /**
      * Validates a list of Numbers.
      */
-    public static Object NumbersValidator = FieldListValidatorFactory(Number.class);
+    public static Object NumbersValidator = listFv(Number.class, true);
 
     /**
-     * Validates is a list of Strings.
+     * Validates a list of Strings.
      */
-    public static Object StringsValidator = FieldListValidatorFactory(String.class);
+    public static Object StringsValidator = listFv(String.class, true);
+    
+    /**
+     * Validates a map of Strings to Numbers.
+     */
+    public static Object MapOfStringToNumberValidator = mapFv(String.class, Number.class, true);
 
     /**
      * Validates is a list of Maps.
      */
-    public static Object MapsValidator = FieldListValidatorFactory(Map.class);
+    public static Object MapsValidator = listFv(Map.class, true);
 
     /**
      * Validates a power of 2.
@@ -105,6 +203,28 @@
     };
 
     /**
+     * Validates a positive integer.
+     */
+    public static Object PositiveIntegerValidator = new FieldValidator() {
+        @Override
+        public void validateField(String name, Object o) throws IllegalArgumentException {
+            if (o == null) {
+                // A null value is acceptable.
+                return;
+            }
+            final long i;
+            if (o instanceof Number &&
+                    (i = ((Number)o).longValue()) == ((Number)o).doubleValue())
+            {
+                if (i > 0) {
+                    return;
+                }
+            }
+            throw new IllegalArgumentException("Field " + name + " must be a positive integer.");
+        }
+    };
+
+    /**
      * Validates Kryo Registration
      */
     public static Object KryoRegValidator = new FieldValidator() {
@@ -141,7 +261,7 @@
      */
     public static Object StringOrStringListValidator = new FieldValidator() {
 
-        private FieldValidator fv = FieldListValidatorFactory(String.class);
+        private FieldValidator fv = listFv(String.class, false);
 
         @Override
         public void validateField(String name, Object o) throws IllegalArgumentException {
diff --git a/storm-core/src/jvm/backtype/storm/Constants.java b/storm-core/src/jvm/backtype/storm/Constants.java
index 39d3ffa..35c252f 100644
--- a/storm-core/src/jvm/backtype/storm/Constants.java
+++ b/storm-core/src/jvm/backtype/storm/Constants.java
@@ -31,5 +31,6 @@
     public static final String METRICS_COMPONENT_ID_PREFIX = "__metrics";
     public static final String METRICS_STREAM_ID = "__metrics";
     public static final String METRICS_TICK_STREAM_ID = "__metrics_tick";
+    public static final String CREDENTIALS_CHANGED_STREAM_ID = "__credentials";
 }
-    
\ No newline at end of file
+    
diff --git a/storm-core/src/jvm/backtype/storm/ICredentialsListener.java b/storm-core/src/jvm/backtype/storm/ICredentialsListener.java
new file mode 100644
index 0000000..1a7bc1b
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/ICredentialsListener.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package backtype.storm;
+
+import java.util.Map;
+
+/**
+ * Allows a bolt or a spout to be informed when the credentials of the topology have changed.
+ */
+public interface ICredentialsListener {
+    /**
+     * Called when the credentials of a topology have changed.
+     * @param credentials the new credentials, could be null.
+     */
+    public void setCredentials(Map<String,String> credentials);
+}
diff --git a/storm-core/src/jvm/backtype/storm/ILocalCluster.java b/storm-core/src/jvm/backtype/storm/ILocalCluster.java
index 818dfb0..7d5aa35 100644
--- a/storm-core/src/jvm/backtype/storm/ILocalCluster.java
+++ b/storm-core/src/jvm/backtype/storm/ILocalCluster.java
@@ -26,6 +26,7 @@
 import backtype.storm.generated.RebalanceOptions;
 import backtype.storm.generated.StormTopology;
 import backtype.storm.generated.TopologyInfo;
+import backtype.storm.generated.Credentials;
 
 import java.util.Map;
 
@@ -33,6 +34,7 @@
 public interface ILocalCluster {
     void submitTopology(String topologyName, Map conf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException;
     void submitTopologyWithOpts(String topologyName, Map conf, StormTopology topology, SubmitOptions submitOpts) throws AlreadyAliveException, InvalidTopologyException;
+    void uploadNewCredentials(String topologyName, Credentials creds);
     void killTopology(String topologyName) throws NotAliveException;
     void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException;
     void activate(String topologyName) throws NotAliveException;
diff --git a/storm-core/src/jvm/backtype/storm/StormSubmitter.java b/storm-core/src/jvm/backtype/storm/StormSubmitter.java
index 5dfb34b..d5da103 100644
--- a/storm-core/src/jvm/backtype/storm/StormSubmitter.java
+++ b/storm-core/src/jvm/backtype/storm/StormSubmitter.java
@@ -19,6 +19,8 @@
 
 import java.io.File;
 import java.nio.ByteBuffer;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -28,13 +30,9 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import backtype.storm.generated.AlreadyAliveException;
-import backtype.storm.generated.ClusterSummary;
-import backtype.storm.generated.InvalidTopologyException;
-import backtype.storm.generated.Nimbus;
-import backtype.storm.generated.StormTopology;
-import backtype.storm.generated.SubmitOptions;
-import backtype.storm.generated.TopologySummary;
+import backtype.storm.security.auth.IAutoCredentials;
+import backtype.storm.security.auth.AuthUtils;
+import backtype.storm.generated.*;
 import backtype.storm.utils.BufferFileInputStream;
 import backtype.storm.utils.NimbusClient;
 import backtype.storm.utils.Utils;
@@ -49,12 +47,99 @@
 
     private static final int THRIFT_CHUNK_SIZE_BYTES = 307200;
     
-    private static Nimbus.Iface localNimbus = null;
+    private static ILocalCluster localNimbus = null;
 
-    public static void setLocalNimbus(Nimbus.Iface localNimbusHandler) {
+    public static void setLocalNimbus(ILocalCluster localNimbusHandler) {
         StormSubmitter.localNimbus = localNimbusHandler;
     }
 
+    private static String generateZookeeperDigestSecretPayload() {
+        return Utils.secureRandomLong() + ":" + Utils.secureRandomLong();
+    }
+
+    public static final Pattern zkDigestPattern = Pattern.compile("\\S+:\\S+");
+
+    public static boolean validateZKDigestPayload(String payload) {
+        if (payload != null) {
+            Matcher m = zkDigestPattern.matcher(payload);
+            return m.matches();
+        }
+        return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    public static Map prepareZookeeperAuthentication(Map conf) {
+        Map toRet = new HashMap();
+
+        // Is the topology ZooKeeper authentication configuration unset?
+        if (! conf.containsKey(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD) ||
+                conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD) == null || 
+                !  validateZKDigestPayload((String)
+                    conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD))) {
+
+            String secretPayload = generateZookeeperDigestSecretPayload();
+            toRet.put(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD, secretPayload);
+            LOG.info("Generated ZooKeeper secret payload for MD5-digest: " + secretPayload);
+        }
+        
+        // This should always be set to digest.
+        toRet.put(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME, "digest");
+
+        return toRet;
+    }
+
+    private static Map<String,String> populateCredentials(Map conf, Map<String, String> creds) {
+        Map<String,String> ret = new HashMap<String,String>();
+        for (IAutoCredentials autoCred: AuthUtils.GetAutoCredentials(conf)) {
+            LOG.info("Running "+autoCred);
+            autoCred.populateCredentials(ret);
+        }
+        if (creds != null) {
+            ret.putAll(creds);
+        }
+        return ret;
+    }
+
+    /**
+     * Push a new set of credentials to the running topology.
+     * @param name the name of the topology to push credentials to.
+     * @param stormConf the topology-specific configuration, if desired. See {@link Config}. 
+     * @param credentials the credentials to push.
+     * @throws AuthorizationException if you are not authorized ot push credentials.
+     * @throws NotAliveException if the topology is not alive
+     * @throws InvalidTopologyException if any other error happens
+     */
+    public static void pushCredentials(String name, Map stormConf, Map<String, String> credentials) 
+            throws AuthorizationException, NotAliveException, InvalidTopologyException {
+        stormConf = new HashMap(stormConf);
+        stormConf.putAll(Utils.readCommandLineOpts());
+        Map conf = Utils.readStormConfig();
+        conf.putAll(stormConf);
+        Map<String,String> fullCreds = populateCredentials(conf, credentials);
+        if (fullCreds.isEmpty()) {
+            LOG.warn("No credentials were found to push to "+name);
+            return;
+        }
+        try {
+            if(localNimbus!=null) {
+                LOG.info("Pushing Credentials to topology " + name + " in local mode");
+                localNimbus.uploadNewCredentials(name, new Credentials(fullCreds));
+            } else {
+                NimbusClient client = NimbusClient.getConfiguredClient(conf);
+                try {
+                    LOG.info("Uploading new credentials to " +  name);
+                    client.getClient().uploadNewCredentials(name, new Credentials(fullCreds));
+                } finally {
+                    client.close();
+                }
+            }
+            LOG.info("Finished submitting topology: " +  name);
+        } catch(TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+ 
+
     /**
      * Submits a topology to run on the cluster. A topology runs forever or until 
      * explicitly killed.
@@ -65,8 +150,10 @@
      * @param topology the processing to execute.
      * @throws AlreadyAliveException if a topology with this name is already running
      * @throws InvalidTopologyException if an invalid topology was submitted
+     * @throws AuthorizationException if authorization is failed
      */
-    public static void submitTopology(String name, Map stormConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException {
+    public static void submitTopology(String name, Map stormConf, StormTopology topology) 
+            throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
         submitTopology(name, stormConf, topology, null, null);
     }    
 
@@ -82,8 +169,11 @@
      * @param progressListener to track the progress of the jar upload process
      * @throws AlreadyAliveException if a topology with this name is already running
      * @throws InvalidTopologyException if an invalid topology was submitted
+     * @throws AuthorizationException if authorization is failed
      */
-    public static void submitTopology(String name, Map stormConf, StormTopology topology, SubmitOptions opts, ProgressListener progressListener) throws AlreadyAliveException, InvalidTopologyException {
+    @SuppressWarnings("unchecked")
+    public static void submitTopology(String name, Map stormConf, StormTopology topology, SubmitOptions opts,
+             ProgressListener progressListener) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
         if(!Utils.isValidConf(stormConf)) {
             throw new IllegalArgumentException("Storm conf is not valid. Must be json-serializable");
         }
@@ -91,24 +181,45 @@
         stormConf.putAll(Utils.readCommandLineOpts());
         Map conf = Utils.readStormConfig();
         conf.putAll(stormConf);
+        stormConf.putAll(prepareZookeeperAuthentication(conf));
+
+        Map<String,String> passedCreds = new HashMap<String, String>();
+        if (opts != null) {
+            Credentials tmpCreds = opts.get_creds();
+            if (tmpCreds != null) {
+                passedCreds = tmpCreds.get_creds();
+            }
+        }
+        Map<String,String> fullCreds = populateCredentials(conf, passedCreds);
+        if (!fullCreds.isEmpty()) {
+            if (opts == null) {
+                opts = new SubmitOptions(TopologyInitialStatus.ACTIVE);
+            }
+            opts.set_creds(new Credentials(fullCreds));
+        }
         try {
-            String serConf = JSONValue.toJSONString(stormConf);
             if(localNimbus!=null) {
                 LOG.info("Submitting topology " + name + " in local mode");
-                localNimbus.submitTopology(name, null, serConf, topology);
+                if(opts!=null) {
+                    localNimbus.submitTopologyWithOpts(name, stormConf, topology, opts);                    
+                } else {
+                    // this is for backwards compatibility
+                    localNimbus.submitTopology(name, stormConf, topology);                                            
+                }
             } else {
+                String serConf = JSONValue.toJSONString(stormConf);
                 NimbusClient client = NimbusClient.getConfiguredClient(conf);
                 if(topologyNameExists(conf, name)) {
                     throw new RuntimeException("Topology with name `" + name + "` already exists on cluster");
                 }
-                submitJar(conf, progressListener);
+                String jar = submitJar(conf, progressListener);
                 try {
                     LOG.info("Submitting topology " +  name + " in distributed mode with conf " + serConf);
                     if(opts!=null) {
-                        client.getClient().submitTopologyWithOpts(name, submittedJar, serConf, topology, opts);
+                        client.getClient().submitTopologyWithOpts(name, jar, serConf, topology, opts);                    
                     } else {
                         // this is for backwards compatibility
-                        client.getClient().submitTopology(name, submittedJar, serConf, topology);
+                        client.getClient().submitTopology(name, jar, serConf, topology);                                            
                     }
                 } catch(InvalidTopologyException e) {
                     LOG.warn("Topology submission exception: "+e.get_msg());
@@ -136,9 +247,10 @@
      * @param topology the processing to execute.
      * @throws AlreadyAliveException if a topology with this name is already running
      * @throws InvalidTopologyException if an invalid topology was submitted
+     * @throws AuthorizationException if authorization is failed
      */
 
-    public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException {
+    public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
         submitTopologyWithProgressBar(name, stormConf, topology, null);
     }
 
@@ -153,9 +265,10 @@
      * @param opts to manipulate the starting of the topology
      * @throws AlreadyAliveException if a topology with this name is already running
      * @throws InvalidTopologyException if an invalid topology was submitted
+     * @throws AuthorizationException if authorization is failed
      */
 
-    public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology, SubmitOptions opts) throws AlreadyAliveException, InvalidTopologyException {
+    public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology, SubmitOptions opts) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
         // show a progress bar so we know we're not stuck (especially on slow connections)
         submitTopology(name, stormConf, topology, opts, new StormSubmitter.ProgressListener() {
             @Override
@@ -198,16 +311,8 @@
         }
     }
 
-    private static String submittedJar = null;
-
-    private static void submitJar(Map conf, ProgressListener listener) {
-        if(submittedJar==null) {
-            LOG.info("Jar not uploaded to master yet. Submitting jar...");
-            String localJar = System.getProperty("storm.jar");
-            submittedJar = submitJar(conf, localJar, listener);
-        } else {
-            LOG.info("Jar already uploaded to master. Not submitting jar.");
-        }
+    private static String submitJar(Map conf, ProgressListener listener) {
+        return  submitJar(conf, System.getProperty("storm.jar"), listener);
     }
 
     /**
diff --git a/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java b/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java
index fadebf6..987cde0 100644
--- a/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java
+++ b/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java
@@ -17,36 +17,33 @@
  */
 package backtype.storm.drpc;
 
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+
 import backtype.storm.generated.DRPCRequest;
 import backtype.storm.generated.DistributedRPCInvocations;
+import backtype.storm.generated.AuthorizationException;
+import backtype.storm.security.auth.ThriftClient;
+import backtype.storm.security.auth.ThriftConnectionType;
+import org.apache.thrift.transport.TTransportException;
 import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TSocket;
-import org.apache.thrift.transport.TTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-public class DRPCInvocationsClient implements DistributedRPCInvocations.Iface {
-    private TTransport conn;
-    private DistributedRPCInvocations.Client client;
+public class DRPCInvocationsClient extends ThriftClient implements DistributedRPCInvocations.Iface {
+    public static Logger LOG = LoggerFactory.getLogger(DRPCInvocationsClient.class);
+    private final AtomicReference<DistributedRPCInvocations.Client> client =
+       new AtomicReference<DistributedRPCInvocations.Client>();
     private String host;
-    private int port;    
+    private int port;
 
-    public DRPCInvocationsClient(String host, int port) {
-        try {
-            this.host = host;
-            this.port = port;
-            connect();
-        } catch(TException e) {
-            throw new RuntimeException(e);
-        }
+    public DRPCInvocationsClient(Map conf, String host, int port) throws TTransportException {
+        super(conf, ThriftConnectionType.DRPC_INVOCATIONS, host, port, null);
+        this.host = host;
+        this.port = port;
+        client.set(new DistributedRPCInvocations.Client(_protocol));
     }
-    
-    private void connect() throws TException {
-        conn = new TFramedTransport(new TSocket(host, port));
-        client = new DistributedRPCInvocations.Client(new TBinaryProtocol(conn));
-        conn.open();
-    }
-    
+        
     public String getHost() {
         return host;
     }
@@ -55,37 +52,57 @@
         return port;
     }       
 
-    public void result(String id, String result) throws TException {
+    public void reconnectClient() throws TException {
+        if (client.get() == null) {
+            reconnect();
+            client.set(new DistributedRPCInvocations.Client(_protocol));
+        }
+    }
+
+    public boolean isConnected() {
+        return client.get() != null;
+    }
+
+    public void result(String id, String result) throws TException, AuthorizationException {
+        DistributedRPCInvocations.Client c = client.get();
         try {
-            if(client==null) connect();
-            client.result(id, result);
+            if (c == null) {
+                throw new TException("Client is not connected...");
+            }
+            c.result(id, result);
         } catch(TException e) {
-            client = null;
+            client.compareAndSet(c, null);
             throw e;
         }
     }
 
-    public DRPCRequest fetchRequest(String func) throws TException {
+    public DRPCRequest fetchRequest(String func) throws TException, AuthorizationException {
+        DistributedRPCInvocations.Client c = client.get();
         try {
-            if(client==null) connect();
-            return client.fetchRequest(func);
+            if (c == null) {
+                throw new TException("Client is not connected...");
+            }
+            return c.fetchRequest(func);
         } catch(TException e) {
-            client = null;
+            client.compareAndSet(c, null);
             throw e;
         }
     }    
 
-    public void failRequest(String id) throws TException {
+    public void failRequest(String id) throws TException, AuthorizationException {
+        DistributedRPCInvocations.Client c = client.get();
         try {
-            if(client==null) connect();
-            client.failRequest(id);
+            if (c == null) {
+                throw new TException("Client is not connected...");
+            }
+            c.failRequest(id);
         } catch(TException e) {
-            client = null;
+            client.compareAndSet(c, null);
             throw e;
         }
     }
 
-    public void close() {
-        conn.close();
+    public DistributedRPCInvocations.Client getClient() {
+        return client.get();
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java b/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java
index 918cbc0..82fd6cd 100644
--- a/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java
+++ b/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java
@@ -21,6 +21,7 @@
 import backtype.storm.ILocalDRPC;
 import backtype.storm.generated.DRPCRequest;
 import backtype.storm.generated.DistributedRPCInvocations;
+import backtype.storm.generated.AuthorizationException;
 import backtype.storm.spout.SpoutOutputCollector;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.OutputFieldsDeclarer;
@@ -31,18 +32,30 @@
 import backtype.storm.utils.Utils;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
+import java.util.LinkedList;
 import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.Callable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
 import org.json.simple.JSONValue;
 
 public class DRPCSpout extends BaseRichSpout {
+    //ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS
+    static final long serialVersionUID = 2387848310969237877L;
+
     public static Logger LOG = LoggerFactory.getLogger(DRPCSpout.class);
     
     SpoutOutputCollector _collector;
     List<DRPCInvocationsClient> _clients = new ArrayList<DRPCInvocationsClient>();
+    transient LinkedList<Future<Void>> _futures = null;
+    transient ExecutorService _backround = null;
     String _function;
     String _local_drpc_id = null;
     
@@ -65,11 +78,60 @@
         _function = function;
         _local_drpc_id = drpc.getServiceId();
     }
-    
+   
+    private class Adder implements Callable<Void> {
+        private String server;
+        private int port;
+        private Map conf;
+
+        public Adder(String server, int port, Map conf) {
+            this.server = server;
+            this.port = port;
+            this.conf = conf;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            DRPCInvocationsClient c = new DRPCInvocationsClient(conf, server, port);
+            synchronized (_clients) {
+                _clients.add(c);
+            }
+            return null;
+        }
+    }
+
+    private void reconnect(final DRPCInvocationsClient c) {
+        _futures.add(_backround.submit(new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                c.reconnectClient();
+                return null;
+            }
+        }));
+    }
+
+    private void checkFutures() {
+        Iterator<Future<Void>> i = _futures.iterator();
+        while (i.hasNext()) {
+            Future<Void> f = i.next();
+            if (f.isDone()) {
+                i.remove();
+            }
+            try {
+                f.get();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+ 
     @Override
     public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
         _collector = collector;
         if(_local_drpc_id==null) {
+            _backround = Executors.newCachedThreadPool();
+            _futures = new LinkedList<Future<Void>>();
+
             int numTasks = context.getComponentTasks(context.getThisComponentId()).size();
             int index = context.getThisTaskIndex();
 
@@ -78,13 +140,14 @@
             if(servers == null || servers.isEmpty()) {
                 throw new RuntimeException("No DRPC servers configured for topology");   
             }
-            if(numTasks < servers.size()) {
-                for(String s: servers) {
-                    _clients.add(new DRPCInvocationsClient(s, port));
+            
+            if (numTasks < servers.size()) {
+                for (String s: servers) {
+                    _futures.add(_backround.submit(new Adder(s, port, conf)));
                 }
-            } else {
+            } else {        
                 int i = index % servers.size();
-                _clients.add(new DRPCInvocationsClient(servers.get(i), port));
+                _futures.add(_backround.submit(new Adder(servers.get(i), port, conf)));
             }
         }
         
@@ -101,8 +164,18 @@
     public void nextTuple() {
         boolean gotRequest = false;
         if(_local_drpc_id==null) {
-            for(int i=0; i<_clients.size(); i++) {
-                DRPCInvocationsClient client = _clients.get(i);
+            int size = 0;
+            synchronized (_clients) {
+                size = _clients.size(); //This will only ever grow, so no need to worry about falling off the end
+            }
+            for(int i=0; i<size; i++) {
+                DRPCInvocationsClient client;
+                synchronized (_clients) {
+                    client = _clients.get(i);
+                }
+                if (!client.isConnected()) {
+                    continue;
+                }
                 try {
                     DRPCRequest req = client.fetchRequest(_function);
                     if(req.get_request_id().length() > 0) {
@@ -114,10 +187,17 @@
                         _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)), new DRPCMessageId(req.get_request_id(), i));
                         break;
                     }
+                } catch (TException e) {
+                    reconnect(client);
+                    LOG.error("Failed to fetch DRPC result from DRPC server", e);
+                } catch (AuthorizationException aze) {
+                    reconnect(client);
+                    LOG.error("Not authorized to fetch DRPC result from DRPC server", aze);
                 } catch (Exception e) {
                     LOG.error("Failed to fetch DRPC result from DRPC server", e);
                 }
             }
+            checkFutures();
         } else {
             DistributedRPCInvocations.Iface drpc = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id);
             if(drpc!=null) { // can happen during shutdown of drpc while topology is still up
@@ -133,6 +213,8 @@
                     }
                 } catch (TException e) {
                     throw new RuntimeException(e);
+                } catch (AuthorizationException aze) {
+                    throw new RuntimeException(aze);
                 }
             }
         }
@@ -159,6 +241,8 @@
             client.failRequest(did.id);
         } catch (TException e) {
             LOG.error("Failed to fail request", e);
+        } catch (AuthorizationException aze) {
+            LOG.error("Not authorized to failREquest from DRPC server", aze);
         }
     }
 
diff --git a/storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java b/storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java
index 34cca98..3d50679 100644
--- a/storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java
+++ b/storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java
@@ -19,6 +19,7 @@
 
 import backtype.storm.Config;
 import backtype.storm.generated.DistributedRPCInvocations;
+import backtype.storm.generated.AuthorizationException;
 import backtype.storm.task.OutputCollector;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.OutputFieldsDeclarer;
@@ -33,18 +34,23 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
 import org.json.simple.JSONValue;
 
 
 public class ReturnResults extends BaseRichBolt {
+    //ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS
+    static final long serialVersionUID = -774882142710631591L;
+
     public static final Logger LOG = LoggerFactory.getLogger(ReturnResults.class);
     OutputCollector _collector;
     boolean local;
-
+    Map _conf; 
     Map<List, DRPCInvocationsClient> _clients = new HashMap<List, DRPCInvocationsClient>();
 
     @Override
     public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _conf = stormConf;
         _collector = collector;
         local = stormConf.get(Config.STORM_CLUSTER_MODE).equals("local");
     }
@@ -68,17 +74,40 @@
                 }};
             
                 if(!_clients.containsKey(server)) {
-                    _clients.put(server, new DRPCInvocationsClient(host, port));
+                    try {
+                        _clients.put(server, new DRPCInvocationsClient(_conf, host, port));
+                    } catch (TTransportException ex) {
+                        throw new RuntimeException(ex);
+                    }
                 }
                 client = _clients.get(server);
             }
-                
+ 
             try {
                 client.result(id, result);
                 _collector.ack(input);
             } catch(TException e) {
                 LOG.error("Failed to return results to DRPC server", e);
                 _collector.fail(input);
+                if (client instanceof DRPCInvocationsClient) {
+                    try {
+                        LOG.info("reconnecting... ");
+                        ((DRPCInvocationsClient)client).reconnectClient(); //Blocking call
+                    } catch (TException e2) {
+                        throw new RuntimeException(e2);
+                    }
+                }
+            } catch (AuthorizationException aze) {
+                LOG.error("Not authorized to return results to DRPC server", aze);
+                _collector.fail(input);
+                if (client instanceof DRPCInvocationsClient) {
+                    try {
+                        LOG.info("reconnecting... ");
+                        ((DRPCInvocationsClient)client).reconnectClient(); //Blocking call
+                    } catch (TException e2) {
+                        throw new RuntimeException(e2);
+                    }
+                }
             }
         }
     }    
diff --git a/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java b/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
new file mode 100644
index 0000000..9efc9da
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
@@ -0,0 +1,328 @@
+/**
+ * Autogenerated by Thrift Compiler (0.7.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AuthorizationException extends Exception implements org.apache.thrift.TBase<AuthorizationException, AuthorizationException._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AuthorizationException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private String msg; // 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 {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        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.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AuthorizationException.class, metaDataMap);
+  }
+
+  public AuthorizationException() {
+  }
+
+  public AuthorizationException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AuthorizationException(AuthorizationException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public AuthorizationException deepCopy() {
+    return new AuthorizationException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    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 MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AuthorizationException)
+      return this.equals((AuthorizationException)that);
+    return false;
+  }
+
+  public boolean equals(AuthorizationException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_msg = true && (is_set_msg());
+    builder.append(present_msg);
+    if (present_msg)
+      builder.append(msg);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(AuthorizationException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    AuthorizationException typedOther = (AuthorizationException)other;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, typedOther.msg);
+      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 {
+    org.apache.thrift.protocol.TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == org.apache.thrift.protocol.TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // MSG
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.msg = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.msg != null) {
+      oprot.writeFieldBegin(MSG_FIELD_DESC);
+      oprot.writeString(this.msg);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("AuthorizationException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+  }
+
+  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);
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/Credentials.java b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
new file mode 100644
index 0000000..105cec1
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
@@ -0,0 +1,373 @@
+/**
+ * Autogenerated by Thrift Compiler (0.7.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Credentials implements org.apache.thrift.TBase<Credentials, Credentials._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Credentials");
+
+  private static final org.apache.thrift.protocol.TField CREDS_FIELD_DESC = new org.apache.thrift.protocol.TField("creds", org.apache.thrift.protocol.TType.MAP, (short)1);
+
+  private Map<String,String> creds; // 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 {
+    CREDS((short)1, "creds");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // CREDS
+          return CREDS;
+        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.CREDS, new org.apache.thrift.meta_data.FieldMetaData("creds", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Credentials.class, metaDataMap);
+  }
+
+  public Credentials() {
+  }
+
+  public Credentials(
+    Map<String,String> creds)
+  {
+    this();
+    this.creds = creds;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public Credentials(Credentials other) {
+    if (other.is_set_creds()) {
+      Map<String,String> __this__creds = new HashMap<String,String>();
+      for (Map.Entry<String, String> other_element : other.creds.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        String other_element_value = other_element.getValue();
+
+        String __this__creds_copy_key = other_element_key;
+
+        String __this__creds_copy_value = other_element_value;
+
+        __this__creds.put(__this__creds_copy_key, __this__creds_copy_value);
+      }
+      this.creds = __this__creds;
+    }
+  }
+
+  public Credentials deepCopy() {
+    return new Credentials(this);
+  }
+
+  @Override
+  public void clear() {
+    this.creds = null;
+  }
+
+  public int get_creds_size() {
+    return (this.creds == null) ? 0 : this.creds.size();
+  }
+
+  public void put_to_creds(String key, String val) {
+    if (this.creds == null) {
+      this.creds = new HashMap<String,String>();
+    }
+    this.creds.put(key, val);
+  }
+
+  public Map<String,String> get_creds() {
+    return this.creds;
+  }
+
+  public void set_creds(Map<String,String> creds) {
+    this.creds = creds;
+  }
+
+  public void unset_creds() {
+    this.creds = null;
+  }
+
+  /** Returns true if field creds is set (has been assigned a value) and false otherwise */
+  public boolean is_set_creds() {
+    return this.creds != null;
+  }
+
+  public void set_creds_isSet(boolean value) {
+    if (!value) {
+      this.creds = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case CREDS:
+      if (value == null) {
+        unset_creds();
+      } else {
+        set_creds((Map<String,String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case CREDS:
+      return get_creds();
+
+    }
+    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 CREDS:
+      return is_set_creds();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof Credentials)
+      return this.equals((Credentials)that);
+    return false;
+  }
+
+  public boolean equals(Credentials that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_creds = true && this.is_set_creds();
+    boolean that_present_creds = true && that.is_set_creds();
+    if (this_present_creds || that_present_creds) {
+      if (!(this_present_creds && that_present_creds))
+        return false;
+      if (!this.creds.equals(that.creds))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_creds = true && (is_set_creds());
+    builder.append(present_creds);
+    if (present_creds)
+      builder.append(creds);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(Credentials other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    Credentials typedOther = (Credentials)other;
+
+    lastComparison = Boolean.valueOf(is_set_creds()).compareTo(typedOther.is_set_creds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_creds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.creds, typedOther.creds);
+      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 {
+    org.apache.thrift.protocol.TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == org.apache.thrift.protocol.TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // CREDS
+          if (field.type == org.apache.thrift.protocol.TType.MAP) {
+            {
+              org.apache.thrift.protocol.TMap _map163 = iprot.readMapBegin();
+              this.creds = new HashMap<String,String>(2*_map163.size);
+              for (int _i164 = 0; _i164 < _map163.size; ++_i164)
+              {
+                String _key165; // required
+                String _val166; // required
+                _key165 = iprot.readString();
+                _val166 = iprot.readString();
+                this.creds.put(_key165, _val166);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.creds != null) {
+      oprot.writeFieldBegin(CREDS_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, this.creds.size()));
+        for (Map.Entry<String, String> _iter167 : this.creds.entrySet())
+        {
+          oprot.writeString(_iter167.getKey());
+          oprot.writeString(_iter167.getValue());
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Credentials(");
+    boolean first = true;
+
+    sb.append("creds:");
+    if (this.creds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.creds);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_creds()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'creds' is unset! Struct:" + toString());
+    }
+
+  }
+
+  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);
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java b/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
index 7922340..06c4f5c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
@@ -42,7 +42,7 @@
 
   public interface Iface {
 
-    public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift.TException;
+    public String execute(String functionName, String funcArgs) throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException;
 
   }
 
@@ -72,7 +72,7 @@
       super(iprot, oprot);
     }
 
-    public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift.TException
+    public String execute(String functionName, String funcArgs) throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException
     {
       send_execute(functionName, funcArgs);
       return recv_execute();
@@ -86,7 +86,7 @@
       sendBase("execute", args);
     }
 
-    public String recv_execute() throws DRPCExecutionException, org.apache.thrift.TException
+    public String recv_execute() throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException
     {
       execute_result result = new execute_result();
       receiveBase(result, "execute");
@@ -96,6 +96,9 @@
       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, "execute failed: unknown result");
     }
 
@@ -142,7 +145,7 @@
         prot.writeMessageEnd();
       }
 
-      public String getResult() throws DRPCExecutionException, org.apache.thrift.TException {
+      public String getResult() throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -184,6 +187,8 @@
           result.success = iface.execute(args.functionName, args.funcArgs);
         } catch (DRPCExecutionException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -590,14 +595,17 @@
 
     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 String success; // required
     private DRPCExecutionException 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");
+      E((short)1, "e"),
+      AZE((short)2, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -616,6 +624,8 @@
             return SUCCESS;
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -664,6 +674,8 @@
           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(execute_result.class, metaDataMap);
     }
@@ -673,11 +685,13 @@
 
     public execute_result(
       String success,
-      DRPCExecutionException e)
+      DRPCExecutionException e,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -690,6 +704,9 @@
       if (other.is_set_e()) {
         this.e = new DRPCExecutionException(other.e);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public execute_result deepCopy() {
@@ -700,6 +717,7 @@
     public void clear() {
       this.success = null;
       this.e = null;
+      this.aze = null;
     }
 
     public String get_success() {
@@ -748,6 +766,29 @@
       }
     }
 
+    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:
@@ -766,6 +807,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -777,6 +826,9 @@
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -792,6 +844,8 @@
         return is_set_success();
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -827,6 +881,15 @@
           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;
     }
 
@@ -844,6 +907,11 @@
       if (present_e)
         builder.append(e);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -875,6 +943,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -907,6 +985,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 2: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -927,6 +1013,10 @@
         oprot.writeFieldBegin(E_FIELD_DESC);
         this.e.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -952,6 +1042,14 @@
         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();
     }
diff --git a/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java b/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
index dc5bb45..e236e41 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
@@ -42,11 +42,11 @@
 
   public interface Iface {
 
-    public void result(String id, String result) throws org.apache.thrift.TException;
+    public void result(String id, String result) throws AuthorizationException, org.apache.thrift.TException;
 
-    public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift.TException;
+    public DRPCRequest fetchRequest(String functionName) throws AuthorizationException, org.apache.thrift.TException;
 
-    public void failRequest(String id) throws org.apache.thrift.TException;
+    public void failRequest(String id) throws AuthorizationException, org.apache.thrift.TException;
 
   }
 
@@ -80,7 +80,7 @@
       super(iprot, oprot);
     }
 
-    public void result(String id, String result) throws org.apache.thrift.TException
+    public void result(String id, String result) throws AuthorizationException, org.apache.thrift.TException
     {
       send_result(id, result);
       recv_result();
@@ -94,14 +94,17 @@
       sendBase("result", args);
     }
 
-    public void recv_result() throws org.apache.thrift.TException
+    public void recv_result() throws AuthorizationException, org.apache.thrift.TException
     {
       result_result result = new result_result();
       receiveBase(result, "result");
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift.TException
+    public DRPCRequest fetchRequest(String functionName) throws AuthorizationException, org.apache.thrift.TException
     {
       send_fetchRequest(functionName);
       return recv_fetchRequest();
@@ -114,17 +117,20 @@
       sendBase("fetchRequest", args);
     }
 
-    public DRPCRequest recv_fetchRequest() throws org.apache.thrift.TException
+    public DRPCRequest recv_fetchRequest() throws AuthorizationException, org.apache.thrift.TException
     {
       fetchRequest_result result = new fetchRequest_result();
       receiveBase(result, "fetchRequest");
       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, "fetchRequest failed: unknown result");
     }
 
-    public void failRequest(String id) throws org.apache.thrift.TException
+    public void failRequest(String id) throws AuthorizationException, org.apache.thrift.TException
     {
       send_failRequest(id);
       recv_failRequest();
@@ -137,10 +143,13 @@
       sendBase("failRequest", args);
     }
 
-    public void recv_failRequest() throws org.apache.thrift.TException
+    public void recv_failRequest() throws AuthorizationException, org.apache.thrift.TException
     {
       failRequest_result result = new failRequest_result();
       receiveBase(result, "failRequest");
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
@@ -187,7 +196,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws org.apache.thrift.TException {
+      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!");
         }
@@ -219,7 +228,7 @@
         prot.writeMessageEnd();
       }
 
-      public DRPCRequest getResult() throws org.apache.thrift.TException {
+      public DRPCRequest getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -251,7 +260,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws org.apache.thrift.TException {
+      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!");
         }
@@ -291,7 +300,11 @@
 
       protected result_result getResult(I iface, result_args args) throws org.apache.thrift.TException {
         result_result result = new result_result();
-        iface.result(args.id, args.result);
+        try {
+          iface.result(args.id, args.result);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -307,7 +320,11 @@
 
       protected fetchRequest_result getResult(I iface, fetchRequest_args args) throws org.apache.thrift.TException {
         fetchRequest_result result = new fetchRequest_result();
-        result.success = iface.fetchRequest(args.functionName);
+        try {
+          result.success = iface.fetchRequest(args.functionName);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -323,7 +340,11 @@
 
       protected failRequest_result getResult(I iface, failRequest_args args) throws org.apache.thrift.TException {
         failRequest_result result = new failRequest_result();
-        iface.failRequest(args.id);
+        try {
+          iface.failRequest(args.id);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -727,11 +748,13 @@
   public static class result_result implements org.apache.thrift.TBase<result_result, result_result._Fields>, java.io.Serializable, Cloneable   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("result_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 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<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -746,6 +769,8 @@
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -784,9 +809,14 @@
         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(result_result.class, metaDataMap);
     }
@@ -794,10 +824,20 @@
     public result_result() {
     }
 
+    public result_result(
+      AuthorizationException aze)
+    {
+      this();
+      this.aze = aze;
+    }
+
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public result_result(result_result other) {
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public result_result deepCopy() {
@@ -806,15 +846,50 @@
 
     @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();
     }
@@ -826,6 +901,8 @@
       }
 
       switch (field) {
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -843,6 +920,15 @@
       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;
     }
 
@@ -850,6 +936,11 @@
     public int hashCode() {
       HashCodeBuilder builder = new HashCodeBuilder();
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -861,6 +952,16 @@
       int lastComparison = 0;
       result_result typedOther = (result_result)other;
 
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -878,6 +979,14 @@
           break;
         }
         switch (field.id) {
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -890,6 +999,11 @@
     public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
       oprot.writeStructBegin(STRUCT_DESC);
 
+      if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -899,6 +1013,13 @@
       StringBuilder sb = new StringBuilder("result_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();
     }
@@ -1230,12 +1351,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("fetchRequest_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 DRPCRequest 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");
+      SUCCESS((short)0, "success"),
+      AZE((short)1, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -1252,6 +1376,8 @@
         switch(fieldId) {
           case 0: // SUCCESS
             return SUCCESS;
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -1298,6 +1424,8 @@
       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, DRPCRequest.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(fetchRequest_result.class, metaDataMap);
     }
@@ -1306,10 +1434,12 @@
     }
 
     public fetchRequest_result(
-      DRPCRequest success)
+      DRPCRequest success,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
+      this.aze = aze;
     }
 
     /**
@@ -1319,6 +1449,9 @@
       if (other.is_set_success()) {
         this.success = new DRPCRequest(other.success);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public fetchRequest_result deepCopy() {
@@ -1328,6 +1461,7 @@
     @Override
     public void clear() {
       this.success = null;
+      this.aze = null;
     }
 
     public DRPCRequest get_success() {
@@ -1353,6 +1487,29 @@
       }
     }
 
+    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:
@@ -1363,6 +1520,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -1371,6 +1536,9 @@
       case SUCCESS:
         return get_success();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -1384,6 +1552,8 @@
       switch (field) {
       case SUCCESS:
         return is_set_success();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -1410,6 +1580,15 @@
           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;
     }
 
@@ -1422,6 +1601,11 @@
       if (present_success)
         builder.append(success);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -1443,6 +1627,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -1468,6 +1662,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -1484,6 +1686,10 @@
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         this.success.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -1501,6 +1707,14 @@
         sb.append(this.success);
       }
       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();
     }
@@ -1831,11 +2045,13 @@
   public static class failRequest_result implements org.apache.thrift.TBase<failRequest_result, failRequest_result._Fields>, java.io.Serializable, Cloneable   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("failRequest_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 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<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -1850,6 +2066,8 @@
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -1888,9 +2106,14 @@
         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(failRequest_result.class, metaDataMap);
     }
@@ -1898,10 +2121,20 @@
     public failRequest_result() {
     }
 
+    public failRequest_result(
+      AuthorizationException aze)
+    {
+      this();
+      this.aze = aze;
+    }
+
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public failRequest_result(failRequest_result other) {
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public failRequest_result deepCopy() {
@@ -1910,15 +2143,50 @@
 
     @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();
     }
@@ -1930,6 +2198,8 @@
       }
 
       switch (field) {
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -1947,6 +2217,15 @@
       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;
     }
 
@@ -1954,6 +2233,11 @@
     public int hashCode() {
       HashCodeBuilder builder = new HashCodeBuilder();
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -1965,6 +2249,16 @@
       int lastComparison = 0;
       failRequest_result typedOther = (failRequest_result)other;
 
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -1982,6 +2276,14 @@
           break;
         }
         switch (field.id) {
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -1994,6 +2296,11 @@
     public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
       oprot.writeStructBegin(STRUCT_DESC);
 
+      if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -2003,6 +2310,13 @@
       StringBuilder sb = new StringBuilder("failRequest_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();
     }
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index e84e12d..7d04901 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
@@ -42,41 +42,43 @@
 
   public interface Iface {
 
-    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException;
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
-    public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException;
+    public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
-    public void killTopology(String name) throws NotAliveException, org.apache.thrift.TException;
+    public void killTopology(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
-    public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift.TException;
+    public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
-    public void activate(String name) throws NotAliveException, org.apache.thrift.TException;
+    public void activate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
-    public void deactivate(String name) throws NotAliveException, org.apache.thrift.TException;
+    public void deactivate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
-    public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException;
+    public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
-    public String beginFileUpload() throws org.apache.thrift.TException;
+    public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
-    public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift.TException;
+    public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException;
 
-    public void finishFileUpload(String location) throws org.apache.thrift.TException;
+    public void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException;
 
-    public String beginFileDownload(String file) throws org.apache.thrift.TException;
+    public void finishFileUpload(String location) throws AuthorizationException, org.apache.thrift.TException;
 
-    public ByteBuffer downloadChunk(String id) throws org.apache.thrift.TException;
+    public String beginFileDownload(String file) throws AuthorizationException, org.apache.thrift.TException;
 
-    public String getNimbusConf() throws org.apache.thrift.TException;
+    public ByteBuffer downloadChunk(String id) throws AuthorizationException, org.apache.thrift.TException;
 
-    public ClusterSummary getClusterInfo() throws org.apache.thrift.TException;
+    public String getNimbusConf() throws AuthorizationException, org.apache.thrift.TException;
 
-    public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift.TException;
+    public ClusterSummary getClusterInfo() throws AuthorizationException, org.apache.thrift.TException;
 
-    public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift.TException;
+    public TopologyInfo getTopologyInfo(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
-    public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift.TException;
+    public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
-    public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift.TException;
+    public StormTopology getTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public StormTopology getUserTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
   }
 
@@ -96,6 +98,8 @@
 
     public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.rebalance_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.uploadNewCredentials_call> resultHandler) throws org.apache.thrift.TException;
+
     public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.beginFileUpload_call> resultHandler) throws org.apache.thrift.TException;
 
     public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.uploadChunk_call> resultHandler) throws org.apache.thrift.TException;
@@ -140,7 +144,7 @@
       super(iprot, oprot);
     }
 
-    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
     {
       send_submitTopology(name, uploadedJarLocation, jsonConf, topology);
       recv_submitTopology();
@@ -156,7 +160,7 @@
       sendBase("submitTopology", args);
     }
 
-    public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException
+    public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
     {
       submitTopology_result result = new submitTopology_result();
       receiveBase(result, "submitTopology");
@@ -166,10 +170,13 @@
       if (result.ite != null) {
         throw result.ite;
       }
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException
+    public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
     {
       send_submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, options);
       recv_submitTopologyWithOpts();
@@ -186,7 +193,7 @@
       sendBase("submitTopologyWithOpts", args);
     }
 
-    public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException
+    public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
     {
       submitTopologyWithOpts_result result = new submitTopologyWithOpts_result();
       receiveBase(result, "submitTopologyWithOpts");
@@ -196,10 +203,13 @@
       if (result.ite != null) {
         throw result.ite;
       }
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public void killTopology(String name) throws NotAliveException, org.apache.thrift.TException
+    public void killTopology(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_killTopology(name);
       recv_killTopology();
@@ -212,17 +222,20 @@
       sendBase("killTopology", args);
     }
 
-    public void recv_killTopology() throws NotAliveException, org.apache.thrift.TException
+    public void recv_killTopology() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       killTopology_result result = new killTopology_result();
       receiveBase(result, "killTopology");
       if (result.e != null) {
         throw result.e;
       }
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift.TException
+    public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_killTopologyWithOpts(name, options);
       recv_killTopologyWithOpts();
@@ -236,17 +249,20 @@
       sendBase("killTopologyWithOpts", args);
     }
 
-    public void recv_killTopologyWithOpts() throws NotAliveException, org.apache.thrift.TException
+    public void recv_killTopologyWithOpts() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       killTopologyWithOpts_result result = new killTopologyWithOpts_result();
       receiveBase(result, "killTopologyWithOpts");
       if (result.e != null) {
         throw result.e;
       }
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public void activate(String name) throws NotAliveException, org.apache.thrift.TException
+    public void activate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_activate(name);
       recv_activate();
@@ -259,17 +275,20 @@
       sendBase("activate", args);
     }
 
-    public void recv_activate() throws NotAliveException, org.apache.thrift.TException
+    public void recv_activate() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       activate_result result = new activate_result();
       receiveBase(result, "activate");
       if (result.e != null) {
         throw result.e;
       }
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public void deactivate(String name) throws NotAliveException, org.apache.thrift.TException
+    public void deactivate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_deactivate(name);
       recv_deactivate();
@@ -282,17 +301,20 @@
       sendBase("deactivate", args);
     }
 
-    public void recv_deactivate() throws NotAliveException, org.apache.thrift.TException
+    public void recv_deactivate() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       deactivate_result result = new deactivate_result();
       receiveBase(result, "deactivate");
       if (result.e != null) {
         throw result.e;
       }
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException
+    public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
     {
       send_rebalance(name, options);
       recv_rebalance();
@@ -306,7 +328,7 @@
       sendBase("rebalance", args);
     }
 
-    public void recv_rebalance() throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException
+    public void recv_rebalance() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
     {
       rebalance_result result = new rebalance_result();
       receiveBase(result, "rebalance");
@@ -316,10 +338,43 @@
       if (result.ite != null) {
         throw result.ite;
       }
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public String beginFileUpload() throws org.apache.thrift.TException
+    public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_uploadNewCredentials(name, creds);
+      recv_uploadNewCredentials();
+    }
+
+    public void send_uploadNewCredentials(String name, Credentials creds) throws org.apache.thrift.TException
+    {
+      uploadNewCredentials_args args = new uploadNewCredentials_args();
+      args.set_name(name);
+      args.set_creds(creds);
+      sendBase("uploadNewCredentials", args);
+    }
+
+    public void recv_uploadNewCredentials() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
+    {
+      uploadNewCredentials_result result = new uploadNewCredentials_result();
+      receiveBase(result, "uploadNewCredentials");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.ite != null) {
+        throw result.ite;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException
     {
       send_beginFileUpload();
       return recv_beginFileUpload();
@@ -331,17 +386,20 @@
       sendBase("beginFileUpload", args);
     }
 
-    public String recv_beginFileUpload() throws org.apache.thrift.TException
+    public String recv_beginFileUpload() throws AuthorizationException, org.apache.thrift.TException
     {
       beginFileUpload_result result = new beginFileUpload_result();
       receiveBase(result, "beginFileUpload");
       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, "beginFileUpload failed: unknown result");
     }
 
-    public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift.TException
+    public void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException
     {
       send_uploadChunk(location, chunk);
       recv_uploadChunk();
@@ -355,14 +413,17 @@
       sendBase("uploadChunk", args);
     }
 
-    public void recv_uploadChunk() throws org.apache.thrift.TException
+    public void recv_uploadChunk() throws AuthorizationException, org.apache.thrift.TException
     {
       uploadChunk_result result = new uploadChunk_result();
       receiveBase(result, "uploadChunk");
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public void finishFileUpload(String location) throws org.apache.thrift.TException
+    public void finishFileUpload(String location) throws AuthorizationException, org.apache.thrift.TException
     {
       send_finishFileUpload(location);
       recv_finishFileUpload();
@@ -375,14 +436,17 @@
       sendBase("finishFileUpload", args);
     }
 
-    public void recv_finishFileUpload() throws org.apache.thrift.TException
+    public void recv_finishFileUpload() throws AuthorizationException, org.apache.thrift.TException
     {
       finishFileUpload_result result = new finishFileUpload_result();
       receiveBase(result, "finishFileUpload");
+      if (result.aze != null) {
+        throw result.aze;
+      }
       return;
     }
 
-    public String beginFileDownload(String file) throws org.apache.thrift.TException
+    public String beginFileDownload(String file) throws AuthorizationException, org.apache.thrift.TException
     {
       send_beginFileDownload(file);
       return recv_beginFileDownload();
@@ -395,17 +459,20 @@
       sendBase("beginFileDownload", args);
     }
 
-    public String recv_beginFileDownload() throws org.apache.thrift.TException
+    public String recv_beginFileDownload() throws AuthorizationException, org.apache.thrift.TException
     {
       beginFileDownload_result result = new beginFileDownload_result();
       receiveBase(result, "beginFileDownload");
       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, "beginFileDownload failed: unknown result");
     }
 
-    public ByteBuffer downloadChunk(String id) throws org.apache.thrift.TException
+    public ByteBuffer downloadChunk(String id) throws AuthorizationException, org.apache.thrift.TException
     {
       send_downloadChunk(id);
       return recv_downloadChunk();
@@ -418,17 +485,20 @@
       sendBase("downloadChunk", args);
     }
 
-    public ByteBuffer recv_downloadChunk() throws org.apache.thrift.TException
+    public ByteBuffer recv_downloadChunk() throws AuthorizationException, org.apache.thrift.TException
     {
       downloadChunk_result result = new downloadChunk_result();
       receiveBase(result, "downloadChunk");
       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, "downloadChunk failed: unknown result");
     }
 
-    public String getNimbusConf() throws org.apache.thrift.TException
+    public String getNimbusConf() throws AuthorizationException, org.apache.thrift.TException
     {
       send_getNimbusConf();
       return recv_getNimbusConf();
@@ -440,17 +510,20 @@
       sendBase("getNimbusConf", args);
     }
 
-    public String recv_getNimbusConf() throws org.apache.thrift.TException
+    public String recv_getNimbusConf() throws AuthorizationException, org.apache.thrift.TException
     {
       getNimbusConf_result result = new getNimbusConf_result();
       receiveBase(result, "getNimbusConf");
       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, "getNimbusConf failed: unknown result");
     }
 
-    public ClusterSummary getClusterInfo() throws org.apache.thrift.TException
+    public ClusterSummary getClusterInfo() throws AuthorizationException, org.apache.thrift.TException
     {
       send_getClusterInfo();
       return recv_getClusterInfo();
@@ -462,17 +535,20 @@
       sendBase("getClusterInfo", args);
     }
 
-    public ClusterSummary recv_getClusterInfo() throws org.apache.thrift.TException
+    public ClusterSummary recv_getClusterInfo() throws AuthorizationException, org.apache.thrift.TException
     {
       getClusterInfo_result result = new getClusterInfo_result();
       receiveBase(result, "getClusterInfo");
       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, "getClusterInfo failed: unknown result");
     }
 
-    public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift.TException
+    public TopologyInfo getTopologyInfo(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_getTopologyInfo(id);
       return recv_getTopologyInfo();
@@ -485,7 +561,7 @@
       sendBase("getTopologyInfo", args);
     }
 
-    public TopologyInfo recv_getTopologyInfo() throws NotAliveException, org.apache.thrift.TException
+    public TopologyInfo recv_getTopologyInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       getTopologyInfo_result result = new getTopologyInfo_result();
       receiveBase(result, "getTopologyInfo");
@@ -495,10 +571,13 @@
       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, "getTopologyInfo failed: unknown result");
     }
 
-    public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift.TException
+    public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_getTopologyConf(id);
       return recv_getTopologyConf();
@@ -511,7 +590,7 @@
       sendBase("getTopologyConf", args);
     }
 
-    public String recv_getTopologyConf() throws NotAliveException, org.apache.thrift.TException
+    public String recv_getTopologyConf() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       getTopologyConf_result result = new getTopologyConf_result();
       receiveBase(result, "getTopologyConf");
@@ -521,10 +600,13 @@
       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, "getTopologyConf failed: unknown result");
     }
 
-    public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift.TException
+    public StormTopology getTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_getTopology(id);
       return recv_getTopology();
@@ -537,7 +619,7 @@
       sendBase("getTopology", args);
     }
 
-    public StormTopology recv_getTopology() throws NotAliveException, org.apache.thrift.TException
+    public StormTopology recv_getTopology() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       getTopology_result result = new getTopology_result();
       receiveBase(result, "getTopology");
@@ -547,10 +629,13 @@
       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, "getTopology failed: unknown result");
     }
 
-    public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift.TException
+    public StormTopology getUserTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_getUserTopology(id);
       return recv_getUserTopology();
@@ -563,7 +648,7 @@
       sendBase("getUserTopology", args);
     }
 
-    public StormTopology recv_getUserTopology() throws NotAliveException, org.apache.thrift.TException
+    public StormTopology recv_getUserTopology() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       getUserTopology_result result = new getUserTopology_result();
       receiveBase(result, "getUserTopology");
@@ -573,6 +658,9 @@
       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, "getUserTopology failed: unknown result");
     }
 
@@ -625,7 +713,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException {
+      public void getResult() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -669,7 +757,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException {
+      public void getResult() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -701,7 +789,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws NotAliveException, org.apache.thrift.TException {
+      public void 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!");
         }
@@ -736,7 +824,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws NotAliveException, org.apache.thrift.TException {
+      public void 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!");
         }
@@ -768,7 +856,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws NotAliveException, org.apache.thrift.TException {
+      public void 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!");
         }
@@ -800,7 +888,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws NotAliveException, org.apache.thrift.TException {
+      public void 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!");
         }
@@ -835,7 +923,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException {
+      public void getResult() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -845,6 +933,41 @@
       }
     }
 
+    public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback<uploadNewCredentials_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      uploadNewCredentials_call method_call = new uploadNewCredentials_call(name, creds, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class uploadNewCredentials_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      private Credentials creds;
+      public uploadNewCredentials_call(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback<uploadNewCredentials_call> 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.name = name;
+        this.creds = creds;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("uploadNewCredentials", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        uploadNewCredentials_args args = new uploadNewCredentials_args();
+        args.set_name(name);
+        args.set_creds(creds);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NotAliveException, InvalidTopologyException, 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_uploadNewCredentials();
+      }
+    }
+
     public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback<beginFileUpload_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       beginFileUpload_call method_call = new beginFileUpload_call(resultHandler, this, ___protocolFactory, ___transport);
@@ -864,7 +987,7 @@
         prot.writeMessageEnd();
       }
 
-      public String getResult() throws org.apache.thrift.TException {
+      public String getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -899,7 +1022,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws org.apache.thrift.TException {
+      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!");
         }
@@ -931,7 +1054,7 @@
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws org.apache.thrift.TException {
+      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!");
         }
@@ -963,7 +1086,7 @@
         prot.writeMessageEnd();
       }
 
-      public String getResult() throws org.apache.thrift.TException {
+      public String getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -995,7 +1118,7 @@
         prot.writeMessageEnd();
       }
 
-      public ByteBuffer getResult() throws org.apache.thrift.TException {
+      public ByteBuffer getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -1024,7 +1147,7 @@
         prot.writeMessageEnd();
       }
 
-      public String getResult() throws org.apache.thrift.TException {
+      public String getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -1053,7 +1176,7 @@
         prot.writeMessageEnd();
       }
 
-      public ClusterSummary getResult() throws org.apache.thrift.TException {
+      public ClusterSummary getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
@@ -1085,7 +1208,7 @@
         prot.writeMessageEnd();
       }
 
-      public TopologyInfo getResult() throws NotAliveException, org.apache.thrift.TException {
+      public TopologyInfo 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!");
         }
@@ -1117,7 +1240,7 @@
         prot.writeMessageEnd();
       }
 
-      public String getResult() throws NotAliveException, org.apache.thrift.TException {
+      public String 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!");
         }
@@ -1149,7 +1272,7 @@
         prot.writeMessageEnd();
       }
 
-      public StormTopology getResult() throws NotAliveException, org.apache.thrift.TException {
+      public StormTopology 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!");
         }
@@ -1181,7 +1304,7 @@
         prot.writeMessageEnd();
       }
 
-      public StormTopology getResult() throws NotAliveException, org.apache.thrift.TException {
+      public StormTopology 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!");
         }
@@ -1211,6 +1334,7 @@
       processMap.put("activate", new activate());
       processMap.put("deactivate", new deactivate());
       processMap.put("rebalance", new rebalance());
+      processMap.put("uploadNewCredentials", new uploadNewCredentials());
       processMap.put("beginFileUpload", new beginFileUpload());
       processMap.put("uploadChunk", new uploadChunk());
       processMap.put("finishFileUpload", new finishFileUpload());
@@ -1242,6 +1366,8 @@
           result.e = e;
         } catch (InvalidTopologyException ite) {
           result.ite = ite;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1264,6 +1390,8 @@
           result.e = e;
         } catch (InvalidTopologyException ite) {
           result.ite = ite;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1284,6 +1412,8 @@
           iface.killTopology(args.name);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1304,6 +1434,8 @@
           iface.killTopologyWithOpts(args.name, args.options);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1324,6 +1456,8 @@
           iface.activate(args.name);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1344,6 +1478,8 @@
           iface.deactivate(args.name);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1366,6 +1502,32 @@
           result.e = e;
         } catch (InvalidTopologyException ite) {
           result.ite = ite;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
+    private static class uploadNewCredentials<I extends Iface> extends org.apache.thrift.ProcessFunction<I, uploadNewCredentials_args> {
+      public uploadNewCredentials() {
+        super("uploadNewCredentials");
+      }
+
+      protected uploadNewCredentials_args getEmptyArgsInstance() {
+        return new uploadNewCredentials_args();
+      }
+
+      protected uploadNewCredentials_result getResult(I iface, uploadNewCredentials_args args) throws org.apache.thrift.TException {
+        uploadNewCredentials_result result = new uploadNewCredentials_result();
+        try {
+          iface.uploadNewCredentials(args.name, args.creds);
+        } catch (NotAliveException e) {
+          result.e = e;
+        } catch (InvalidTopologyException ite) {
+          result.ite = ite;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1382,7 +1544,11 @@
 
       protected beginFileUpload_result getResult(I iface, beginFileUpload_args args) throws org.apache.thrift.TException {
         beginFileUpload_result result = new beginFileUpload_result();
-        result.success = iface.beginFileUpload();
+        try {
+          result.success = iface.beginFileUpload();
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -1398,7 +1564,11 @@
 
       protected uploadChunk_result getResult(I iface, uploadChunk_args args) throws org.apache.thrift.TException {
         uploadChunk_result result = new uploadChunk_result();
-        iface.uploadChunk(args.location, args.chunk);
+        try {
+          iface.uploadChunk(args.location, args.chunk);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -1414,7 +1584,11 @@
 
       protected finishFileUpload_result getResult(I iface, finishFileUpload_args args) throws org.apache.thrift.TException {
         finishFileUpload_result result = new finishFileUpload_result();
-        iface.finishFileUpload(args.location);
+        try {
+          iface.finishFileUpload(args.location);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -1430,7 +1604,11 @@
 
       protected beginFileDownload_result getResult(I iface, beginFileDownload_args args) throws org.apache.thrift.TException {
         beginFileDownload_result result = new beginFileDownload_result();
-        result.success = iface.beginFileDownload(args.file);
+        try {
+          result.success = iface.beginFileDownload(args.file);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -1446,7 +1624,11 @@
 
       protected downloadChunk_result getResult(I iface, downloadChunk_args args) throws org.apache.thrift.TException {
         downloadChunk_result result = new downloadChunk_result();
-        result.success = iface.downloadChunk(args.id);
+        try {
+          result.success = iface.downloadChunk(args.id);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -1462,7 +1644,11 @@
 
       protected getNimbusConf_result getResult(I iface, getNimbusConf_args args) throws org.apache.thrift.TException {
         getNimbusConf_result result = new getNimbusConf_result();
-        result.success = iface.getNimbusConf();
+        try {
+          result.success = iface.getNimbusConf();
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -1478,7 +1664,11 @@
 
       protected getClusterInfo_result getResult(I iface, getClusterInfo_args args) throws org.apache.thrift.TException {
         getClusterInfo_result result = new getClusterInfo_result();
-        result.success = iface.getClusterInfo();
+        try {
+          result.success = iface.getClusterInfo();
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
         return result;
       }
     }
@@ -1498,6 +1688,8 @@
           result.success = iface.getTopologyInfo(args.id);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1518,6 +1710,8 @@
           result.success = iface.getTopologyConf(args.id);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1538,6 +1732,8 @@
           result.success = iface.getTopology(args.id);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1558,6 +1754,8 @@
           result.success = iface.getUserTopology(args.id);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -2151,14 +2349,17 @@
 
     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 ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", 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)3);
 
     private AlreadyAliveException e; // required
     private InvalidTopologyException ite; // 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 {
       E((short)1, "e"),
-      ITE((short)2, "ite");
+      ITE((short)2, "ite"),
+      AZE((short)3, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -2177,6 +2378,8 @@
             return E;
           case 2: // ITE
             return ITE;
+          case 3: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -2225,6 +2428,8 @@
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", 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(submitTopology_result.class, metaDataMap);
     }
@@ -2234,11 +2439,13 @@
 
     public submitTopology_result(
       AlreadyAliveException e,
-      InvalidTopologyException ite)
+      InvalidTopologyException ite,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
       this.ite = ite;
+      this.aze = aze;
     }
 
     /**
@@ -2251,6 +2458,9 @@
       if (other.is_set_ite()) {
         this.ite = new InvalidTopologyException(other.ite);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public submitTopology_result deepCopy() {
@@ -2261,6 +2471,7 @@
     public void clear() {
       this.e = null;
       this.ite = null;
+      this.aze = null;
     }
 
     public AlreadyAliveException get_e() {
@@ -2309,6 +2520,29 @@
       }
     }
 
+    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 E:
@@ -2327,6 +2561,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -2338,6 +2580,9 @@
       case ITE:
         return get_ite();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -2353,6 +2598,8 @@
         return is_set_e();
       case ITE:
         return is_set_ite();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -2388,6 +2635,15 @@
           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;
     }
 
@@ -2405,6 +2661,11 @@
       if (present_ite)
         builder.append(ite);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -2436,6 +2697,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -2469,6 +2740,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 3: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -2489,6 +2768,10 @@
         oprot.writeFieldBegin(ITE_FIELD_DESC);
         this.ite.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -2514,6 +2797,14 @@
         sb.append(this.ite);
       }
       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();
     }
@@ -3220,14 +3511,17 @@
 
     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 ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", 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)3);
 
     private AlreadyAliveException e; // required
     private InvalidTopologyException ite; // 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 {
       E((short)1, "e"),
-      ITE((short)2, "ite");
+      ITE((short)2, "ite"),
+      AZE((short)3, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -3246,6 +3540,8 @@
             return E;
           case 2: // ITE
             return ITE;
+          case 3: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -3294,6 +3590,8 @@
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", 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(submitTopologyWithOpts_result.class, metaDataMap);
     }
@@ -3303,11 +3601,13 @@
 
     public submitTopologyWithOpts_result(
       AlreadyAliveException e,
-      InvalidTopologyException ite)
+      InvalidTopologyException ite,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
       this.ite = ite;
+      this.aze = aze;
     }
 
     /**
@@ -3320,6 +3620,9 @@
       if (other.is_set_ite()) {
         this.ite = new InvalidTopologyException(other.ite);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public submitTopologyWithOpts_result deepCopy() {
@@ -3330,6 +3633,7 @@
     public void clear() {
       this.e = null;
       this.ite = null;
+      this.aze = null;
     }
 
     public AlreadyAliveException get_e() {
@@ -3378,6 +3682,29 @@
       }
     }
 
+    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 E:
@@ -3396,6 +3723,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -3407,6 +3742,9 @@
       case ITE:
         return get_ite();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -3422,6 +3760,8 @@
         return is_set_e();
       case ITE:
         return is_set_ite();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -3457,6 +3797,15 @@
           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;
     }
 
@@ -3474,6 +3823,11 @@
       if (present_ite)
         builder.append(ite);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -3505,6 +3859,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -3538,6 +3902,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 3: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -3558,6 +3930,10 @@
         oprot.writeFieldBegin(ITE_FIELD_DESC);
         this.ite.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -3583,6 +3959,14 @@
         sb.append(this.ite);
       }
       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();
     }
@@ -3914,12 +4298,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("killTopology_result");
 
     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 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 {
-      E((short)1, "e");
+      E((short)1, "e"),
+      AZE((short)2, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -3936,6 +4323,8 @@
         switch(fieldId) {
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -3982,6 +4371,8 @@
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.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(killTopology_result.class, metaDataMap);
     }
@@ -3990,10 +4381,12 @@
     }
 
     public killTopology_result(
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -4003,6 +4396,9 @@
       if (other.is_set_e()) {
         this.e = new NotAliveException(other.e);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public killTopology_result deepCopy() {
@@ -4012,6 +4408,7 @@
     @Override
     public void clear() {
       this.e = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -4037,6 +4434,29 @@
       }
     }
 
+    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 E:
@@ -4047,6 +4467,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -4055,6 +4483,9 @@
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -4068,6 +4499,8 @@
       switch (field) {
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -4094,6 +4527,15 @@
           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;
     }
 
@@ -4106,6 +4548,11 @@
       if (present_e)
         builder.append(e);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -4127,6 +4574,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -4152,6 +4609,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 2: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -4168,6 +4633,10 @@
         oprot.writeFieldBegin(E_FIELD_DESC);
         this.e.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -4185,6 +4654,14 @@
         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();
     }
@@ -4610,12 +5087,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("killTopologyWithOpts_result");
 
     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 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 {
-      E((short)1, "e");
+      E((short)1, "e"),
+      AZE((short)2, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -4632,6 +5112,8 @@
         switch(fieldId) {
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -4678,6 +5160,8 @@
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.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(killTopologyWithOpts_result.class, metaDataMap);
     }
@@ -4686,10 +5170,12 @@
     }
 
     public killTopologyWithOpts_result(
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -4699,6 +5185,9 @@
       if (other.is_set_e()) {
         this.e = new NotAliveException(other.e);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public killTopologyWithOpts_result deepCopy() {
@@ -4708,6 +5197,7 @@
     @Override
     public void clear() {
       this.e = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -4733,6 +5223,29 @@
       }
     }
 
+    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 E:
@@ -4743,6 +5256,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -4751,6 +5272,9 @@
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -4764,6 +5288,8 @@
       switch (field) {
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -4790,6 +5316,15 @@
           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;
     }
 
@@ -4802,6 +5337,11 @@
       if (present_e)
         builder.append(e);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -4823,6 +5363,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -4848,6 +5398,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 2: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -4864,6 +5422,10 @@
         oprot.writeFieldBegin(E_FIELD_DESC);
         this.e.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -4881,6 +5443,14 @@
         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();
     }
@@ -5212,12 +5782,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("activate_result");
 
     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 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 {
-      E((short)1, "e");
+      E((short)1, "e"),
+      AZE((short)2, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -5234,6 +5807,8 @@
         switch(fieldId) {
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -5280,6 +5855,8 @@
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.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(activate_result.class, metaDataMap);
     }
@@ -5288,10 +5865,12 @@
     }
 
     public activate_result(
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -5301,6 +5880,9 @@
       if (other.is_set_e()) {
         this.e = new NotAliveException(other.e);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public activate_result deepCopy() {
@@ -5310,6 +5892,7 @@
     @Override
     public void clear() {
       this.e = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -5335,6 +5918,29 @@
       }
     }
 
+    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 E:
@@ -5345,6 +5951,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -5353,6 +5967,9 @@
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -5366,6 +5983,8 @@
       switch (field) {
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -5392,6 +6011,15 @@
           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;
     }
 
@@ -5404,6 +6032,11 @@
       if (present_e)
         builder.append(e);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -5425,6 +6058,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -5450,6 +6093,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 2: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -5466,6 +6117,10 @@
         oprot.writeFieldBegin(E_FIELD_DESC);
         this.e.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -5483,6 +6138,14 @@
         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();
     }
@@ -5814,12 +6477,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deactivate_result");
 
     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 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 {
-      E((short)1, "e");
+      E((short)1, "e"),
+      AZE((short)2, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -5836,6 +6502,8 @@
         switch(fieldId) {
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -5882,6 +6550,8 @@
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.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(deactivate_result.class, metaDataMap);
     }
@@ -5890,10 +6560,12 @@
     }
 
     public deactivate_result(
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -5903,6 +6575,9 @@
       if (other.is_set_e()) {
         this.e = new NotAliveException(other.e);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public deactivate_result deepCopy() {
@@ -5912,6 +6587,7 @@
     @Override
     public void clear() {
       this.e = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -5937,6 +6613,29 @@
       }
     }
 
+    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 E:
@@ -5947,6 +6646,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -5955,6 +6662,9 @@
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -5968,6 +6678,8 @@
       switch (field) {
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -5994,6 +6706,15 @@
           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;
     }
 
@@ -6006,6 +6727,11 @@
       if (present_e)
         builder.append(e);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -6027,6 +6753,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -6052,6 +6788,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 2: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -6068,6 +6812,10 @@
         oprot.writeFieldBegin(E_FIELD_DESC);
         this.e.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -6085,6 +6833,14 @@
         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();
     }
@@ -6511,14 +7267,17 @@
 
     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 ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", 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)3);
 
     private NotAliveException e; // required
     private InvalidTopologyException ite; // 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 {
       E((short)1, "e"),
-      ITE((short)2, "ite");
+      ITE((short)2, "ite"),
+      AZE((short)3, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -6537,6 +7296,8 @@
             return E;
           case 2: // ITE
             return ITE;
+          case 3: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -6585,6 +7346,8 @@
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", 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(rebalance_result.class, metaDataMap);
     }
@@ -6594,11 +7357,13 @@
 
     public rebalance_result(
       NotAliveException e,
-      InvalidTopologyException ite)
+      InvalidTopologyException ite,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
       this.ite = ite;
+      this.aze = aze;
     }
 
     /**
@@ -6611,6 +7376,9 @@
       if (other.is_set_ite()) {
         this.ite = new InvalidTopologyException(other.ite);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public rebalance_result deepCopy() {
@@ -6621,6 +7389,7 @@
     public void clear() {
       this.e = null;
       this.ite = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -6669,6 +7438,29 @@
       }
     }
 
+    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 E:
@@ -6687,6 +7479,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -6698,6 +7498,9 @@
       case ITE:
         return get_ite();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -6713,6 +7516,8 @@
         return is_set_e();
       case ITE:
         return is_set_ite();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -6748,6 +7553,15 @@
           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;
     }
 
@@ -6765,6 +7579,11 @@
       if (present_ite)
         builder.append(ite);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -6796,6 +7615,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -6829,6 +7658,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 3: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -6849,6 +7686,10 @@
         oprot.writeFieldBegin(ITE_FIELD_DESC);
         this.ite.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -6874,6 +7715,896 @@
         sb.append(this.ite);
       }
       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
+    }
+
+    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);
+      }
+    }
+
+  }
+
+  public static class uploadNewCredentials_args implements org.apache.thrift.TBase<uploadNewCredentials_args, uploadNewCredentials_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadNewCredentials_args");
+
+    private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDS_FIELD_DESC = new org.apache.thrift.protocol.TField("creds", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private String name; // required
+    private Credentials creds; // 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 {
+      NAME((short)1, "name"),
+      CREDS((short)2, "creds");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // NAME
+            return NAME;
+          case 2: // CREDS
+            return CREDS;
+          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.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.CREDS, new org.apache.thrift.meta_data.FieldMetaData("creds", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Credentials.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(uploadNewCredentials_args.class, metaDataMap);
+    }
+
+    public uploadNewCredentials_args() {
+    }
+
+    public uploadNewCredentials_args(
+      String name,
+      Credentials creds)
+    {
+      this();
+      this.name = name;
+      this.creds = creds;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public uploadNewCredentials_args(uploadNewCredentials_args other) {
+      if (other.is_set_name()) {
+        this.name = other.name;
+      }
+      if (other.is_set_creds()) {
+        this.creds = new Credentials(other.creds);
+      }
+    }
+
+    public uploadNewCredentials_args deepCopy() {
+      return new uploadNewCredentials_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.name = null;
+      this.creds = null;
+    }
+
+    public String get_name() {
+      return this.name;
+    }
+
+    public void set_name(String name) {
+      this.name = name;
+    }
+
+    public void unset_name() {
+      this.name = null;
+    }
+
+    /** Returns true if field name is set (has been assigned a value) and false otherwise */
+    public boolean is_set_name() {
+      return this.name != null;
+    }
+
+    public void set_name_isSet(boolean value) {
+      if (!value) {
+        this.name = null;
+      }
+    }
+
+    public Credentials get_creds() {
+      return this.creds;
+    }
+
+    public void set_creds(Credentials creds) {
+      this.creds = creds;
+    }
+
+    public void unset_creds() {
+      this.creds = null;
+    }
+
+    /** Returns true if field creds is set (has been assigned a value) and false otherwise */
+    public boolean is_set_creds() {
+      return this.creds != null;
+    }
+
+    public void set_creds_isSet(boolean value) {
+      if (!value) {
+        this.creds = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case NAME:
+        if (value == null) {
+          unset_name();
+        } else {
+          set_name((String)value);
+        }
+        break;
+
+      case CREDS:
+        if (value == null) {
+          unset_creds();
+        } else {
+          set_creds((Credentials)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case NAME:
+        return get_name();
+
+      case CREDS:
+        return get_creds();
+
+      }
+      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 NAME:
+        return is_set_name();
+      case CREDS:
+        return is_set_creds();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof uploadNewCredentials_args)
+        return this.equals((uploadNewCredentials_args)that);
+      return false;
+    }
+
+    public boolean equals(uploadNewCredentials_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_name = true && this.is_set_name();
+      boolean that_present_name = true && that.is_set_name();
+      if (this_present_name || that_present_name) {
+        if (!(this_present_name && that_present_name))
+          return false;
+        if (!this.name.equals(that.name))
+          return false;
+      }
+
+      boolean this_present_creds = true && this.is_set_creds();
+      boolean that_present_creds = true && that.is_set_creds();
+      if (this_present_creds || that_present_creds) {
+        if (!(this_present_creds && that_present_creds))
+          return false;
+        if (!this.creds.equals(that.creds))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_name = true && (is_set_name());
+      builder.append(present_name);
+      if (present_name)
+        builder.append(name);
+
+      boolean present_creds = true && (is_set_creds());
+      builder.append(present_creds);
+      if (present_creds)
+        builder.append(creds);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(uploadNewCredentials_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      uploadNewCredentials_args typedOther = (uploadNewCredentials_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_name()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, typedOther.name);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_creds()).compareTo(typedOther.is_set_creds());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_creds()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.creds, typedOther.creds);
+        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 {
+      org.apache.thrift.protocol.TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // NAME
+            if (field.type == org.apache.thrift.protocol.TType.STRING) {
+              this.name = iprot.readString();
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 2: // CREDS
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.creds = new Credentials();
+              this.creds.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(this.name);
+        oprot.writeFieldEnd();
+      }
+      if (this.creds != null) {
+        oprot.writeFieldBegin(CREDS_FIELD_DESC);
+        this.creds.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("uploadNewCredentials_args(");
+      boolean first = true;
+
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("creds:");
+      if (this.creds == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.creds);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+    }
+
+    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);
+      }
+    }
+
+  }
+
+  public static class uploadNewCredentials_result implements org.apache.thrift.TBase<uploadNewCredentials_result, uploadNewCredentials_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadNewCredentials_result");
+
+    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 ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", 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)3);
+
+    private NotAliveException e; // required
+    private InvalidTopologyException ite; // 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 {
+      E((short)1, "e"),
+      ITE((short)2, "ite"),
+      AZE((short)3, "aze");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // E
+            return E;
+          case 2: // ITE
+            return ITE;
+          case 3: // 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.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.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", 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(uploadNewCredentials_result.class, metaDataMap);
+    }
+
+    public uploadNewCredentials_result() {
+    }
+
+    public uploadNewCredentials_result(
+      NotAliveException e,
+      InvalidTopologyException ite,
+      AuthorizationException aze)
+    {
+      this();
+      this.e = e;
+      this.ite = ite;
+      this.aze = aze;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public uploadNewCredentials_result(uploadNewCredentials_result other) {
+      if (other.is_set_e()) {
+        this.e = new NotAliveException(other.e);
+      }
+      if (other.is_set_ite()) {
+        this.ite = new InvalidTopologyException(other.ite);
+      }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
+    }
+
+    public uploadNewCredentials_result deepCopy() {
+      return new uploadNewCredentials_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.e = null;
+      this.ite = null;
+      this.aze = 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 InvalidTopologyException get_ite() {
+      return this.ite;
+    }
+
+    public void set_ite(InvalidTopologyException ite) {
+      this.ite = ite;
+    }
+
+    public void unset_ite() {
+      this.ite = null;
+    }
+
+    /** Returns true if field ite is set (has been assigned a value) and false otherwise */
+    public boolean is_set_ite() {
+      return this.ite != null;
+    }
+
+    public void set_ite_isSet(boolean value) {
+      if (!value) {
+        this.ite = 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 E:
+        if (value == null) {
+          unset_e();
+        } else {
+          set_e((NotAliveException)value);
+        }
+        break;
+
+      case ITE:
+        if (value == null) {
+          unset_ite();
+        } else {
+          set_ite((InvalidTopologyException)value);
+        }
+        break;
+
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case E:
+        return get_e();
+
+      case ITE:
+        return get_ite();
+
+      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 E:
+        return is_set_e();
+      case ITE:
+        return is_set_ite();
+      case AZE:
+        return is_set_aze();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof uploadNewCredentials_result)
+        return this.equals((uploadNewCredentials_result)that);
+      return false;
+    }
+
+    public boolean equals(uploadNewCredentials_result that) {
+      if (that == null)
+        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_ite = true && this.is_set_ite();
+      boolean that_present_ite = true && that.is_set_ite();
+      if (this_present_ite || that_present_ite) {
+        if (!(this_present_ite && that_present_ite))
+          return false;
+        if (!this.ite.equals(that.ite))
+          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() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_e = true && (is_set_e());
+      builder.append(present_e);
+      if (present_e)
+        builder.append(e);
+
+      boolean present_ite = true && (is_set_ite());
+      builder.append(present_ite);
+      if (present_ite)
+        builder.append(ite);
+
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(uploadNewCredentials_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      uploadNewCredentials_result typedOther = (uploadNewCredentials_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_e()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, typedOther.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_ite()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ite, typedOther.ite);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.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 {
+      org.apache.thrift.protocol.TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // E
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.e = new NotAliveException();
+              this.e.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 2: // ITE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.ite = new InvalidTopologyException();
+              this.ite.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 3: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_e()) {
+        oprot.writeFieldBegin(E_FIELD_DESC);
+        this.e.write(oprot);
+        oprot.writeFieldEnd();
+      } else if (this.is_set_ite()) {
+        oprot.writeFieldBegin(ITE_FIELD_DESC);
+        this.ite.write(oprot);
+        oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("uploadNewCredentials_result(");
+      boolean first = true;
+
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("ite:");
+      if (this.ite == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.ite);
+      }
+      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();
     }
@@ -7106,12 +8837,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFileUpload_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 AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private String 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");
+      SUCCESS((short)0, "success"),
+      AZE((short)1, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -7128,6 +8862,8 @@
         switch(fieldId) {
           case 0: // SUCCESS
             return SUCCESS;
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -7174,6 +8910,8 @@
       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.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(beginFileUpload_result.class, metaDataMap);
     }
@@ -7182,10 +8920,12 @@
     }
 
     public beginFileUpload_result(
-      String success)
+      String success,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
+      this.aze = aze;
     }
 
     /**
@@ -7195,6 +8935,9 @@
       if (other.is_set_success()) {
         this.success = other.success;
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public beginFileUpload_result deepCopy() {
@@ -7204,6 +8947,7 @@
     @Override
     public void clear() {
       this.success = null;
+      this.aze = null;
     }
 
     public String get_success() {
@@ -7229,6 +8973,29 @@
       }
     }
 
+    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:
@@ -7239,6 +9006,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -7247,6 +9022,9 @@
       case SUCCESS:
         return get_success();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -7260,6 +9038,8 @@
       switch (field) {
       case SUCCESS:
         return is_set_success();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -7286,6 +9066,15 @@
           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;
     }
 
@@ -7298,6 +9087,11 @@
       if (present_success)
         builder.append(success);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -7319,6 +9113,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -7343,6 +9147,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -7359,6 +9171,10 @@
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         oprot.writeString(this.success);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -7376,6 +9192,14 @@
         sb.append(this.success);
       }
       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();
     }
@@ -7809,11 +9633,13 @@
   public static class uploadChunk_result implements org.apache.thrift.TBase<uploadChunk_result, uploadChunk_result._Fields>, java.io.Serializable, Cloneable   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadChunk_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 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<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -7828,6 +9654,8 @@
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -7866,9 +9694,14 @@
         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(uploadChunk_result.class, metaDataMap);
     }
@@ -7876,10 +9709,20 @@
     public uploadChunk_result() {
     }
 
+    public uploadChunk_result(
+      AuthorizationException aze)
+    {
+      this();
+      this.aze = aze;
+    }
+
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public uploadChunk_result(uploadChunk_result other) {
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public uploadChunk_result deepCopy() {
@@ -7888,15 +9731,50 @@
 
     @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();
     }
@@ -7908,6 +9786,8 @@
       }
 
       switch (field) {
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -7925,6 +9805,15 @@
       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;
     }
 
@@ -7932,6 +9821,11 @@
     public int hashCode() {
       HashCodeBuilder builder = new HashCodeBuilder();
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -7943,6 +9837,16 @@
       int lastComparison = 0;
       uploadChunk_result typedOther = (uploadChunk_result)other;
 
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -7960,6 +9864,14 @@
           break;
         }
         switch (field.id) {
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -7972,6 +9884,11 @@
     public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
       oprot.writeStructBegin(STRUCT_DESC);
 
+      if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -7981,6 +9898,13 @@
       StringBuilder sb = new StringBuilder("uploadChunk_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();
     }
@@ -8311,11 +10235,13 @@
   public static class finishFileUpload_result implements org.apache.thrift.TBase<finishFileUpload_result, finishFileUpload_result._Fields>, java.io.Serializable, Cloneable   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("finishFileUpload_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 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<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -8330,6 +10256,8 @@
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -8368,9 +10296,14 @@
         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(finishFileUpload_result.class, metaDataMap);
     }
@@ -8378,10 +10311,20 @@
     public finishFileUpload_result() {
     }
 
+    public finishFileUpload_result(
+      AuthorizationException aze)
+    {
+      this();
+      this.aze = aze;
+    }
+
     /**
      * Performs a deep copy on <i>other</i>.
      */
     public finishFileUpload_result(finishFileUpload_result other) {
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public finishFileUpload_result deepCopy() {
@@ -8390,15 +10333,50 @@
 
     @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();
     }
@@ -8410,6 +10388,8 @@
       }
 
       switch (field) {
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -8427,6 +10407,15 @@
       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;
     }
 
@@ -8434,6 +10423,11 @@
     public int hashCode() {
       HashCodeBuilder builder = new HashCodeBuilder();
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -8445,6 +10439,16 @@
       int lastComparison = 0;
       finishFileUpload_result typedOther = (finishFileUpload_result)other;
 
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -8462,6 +10466,14 @@
           break;
         }
         switch (field.id) {
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -8474,6 +10486,11 @@
     public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
       oprot.writeStructBegin(STRUCT_DESC);
 
+      if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -8483,6 +10500,13 @@
       StringBuilder sb = new StringBuilder("finishFileUpload_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();
     }
@@ -8814,12 +10838,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFileDownload_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 AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private String 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");
+      SUCCESS((short)0, "success"),
+      AZE((short)1, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -8836,6 +10863,8 @@
         switch(fieldId) {
           case 0: // SUCCESS
             return SUCCESS;
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -8882,6 +10911,8 @@
       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.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(beginFileDownload_result.class, metaDataMap);
     }
@@ -8890,10 +10921,12 @@
     }
 
     public beginFileDownload_result(
-      String success)
+      String success,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
+      this.aze = aze;
     }
 
     /**
@@ -8903,6 +10936,9 @@
       if (other.is_set_success()) {
         this.success = other.success;
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public beginFileDownload_result deepCopy() {
@@ -8912,6 +10948,7 @@
     @Override
     public void clear() {
       this.success = null;
+      this.aze = null;
     }
 
     public String get_success() {
@@ -8937,6 +10974,29 @@
       }
     }
 
+    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:
@@ -8947,6 +11007,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -8955,6 +11023,9 @@
       case SUCCESS:
         return get_success();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -8968,6 +11039,8 @@
       switch (field) {
       case SUCCESS:
         return is_set_success();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -8994,6 +11067,15 @@
           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;
     }
 
@@ -9006,6 +11088,11 @@
       if (present_success)
         builder.append(success);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -9027,6 +11114,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -9051,6 +11148,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -9067,6 +11172,10 @@
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         oprot.writeString(this.success);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -9084,6 +11193,14 @@
         sb.append(this.success);
       }
       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();
     }
@@ -9415,12 +11532,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("downloadChunk_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 AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private ByteBuffer 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");
+      SUCCESS((short)0, "success"),
+      AZE((short)1, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -9437,6 +11557,8 @@
         switch(fieldId) {
           case 0: // SUCCESS
             return SUCCESS;
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -9483,6 +11605,8 @@
       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          , true)));
+      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(downloadChunk_result.class, metaDataMap);
     }
@@ -9491,10 +11615,12 @@
     }
 
     public downloadChunk_result(
-      ByteBuffer success)
+      ByteBuffer success,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
+      this.aze = aze;
     }
 
     /**
@@ -9505,6 +11631,9 @@
         this.success = org.apache.thrift.TBaseHelper.copyBinary(other.success);
 ;
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public downloadChunk_result deepCopy() {
@@ -9514,6 +11643,7 @@
     @Override
     public void clear() {
       this.success = null;
+      this.aze = null;
     }
 
     public byte[] get_success() {
@@ -9548,6 +11678,29 @@
       }
     }
 
+    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:
@@ -9558,6 +11711,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -9566,6 +11727,9 @@
       case SUCCESS:
         return get_success();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -9579,6 +11743,8 @@
       switch (field) {
       case SUCCESS:
         return is_set_success();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -9605,6 +11771,15 @@
           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;
     }
 
@@ -9617,6 +11792,11 @@
       if (present_success)
         builder.append(success);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -9638,6 +11818,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -9662,6 +11852,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -9678,6 +11876,10 @@
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         oprot.writeBinary(this.success);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -9695,6 +11897,14 @@
         org.apache.thrift.TBaseHelper.toString(this.success, sb);
       }
       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();
     }
@@ -9927,12 +12137,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNimbusConf_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 AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private String 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");
+      SUCCESS((short)0, "success"),
+      AZE((short)1, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -9949,6 +12162,8 @@
         switch(fieldId) {
           case 0: // SUCCESS
             return SUCCESS;
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -9995,6 +12210,8 @@
       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.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(getNimbusConf_result.class, metaDataMap);
     }
@@ -10003,10 +12220,12 @@
     }
 
     public getNimbusConf_result(
-      String success)
+      String success,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
+      this.aze = aze;
     }
 
     /**
@@ -10016,6 +12235,9 @@
       if (other.is_set_success()) {
         this.success = other.success;
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public getNimbusConf_result deepCopy() {
@@ -10025,6 +12247,7 @@
     @Override
     public void clear() {
       this.success = null;
+      this.aze = null;
     }
 
     public String get_success() {
@@ -10050,6 +12273,29 @@
       }
     }
 
+    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:
@@ -10060,6 +12306,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -10068,6 +12322,9 @@
       case SUCCESS:
         return get_success();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -10081,6 +12338,8 @@
       switch (field) {
       case SUCCESS:
         return is_set_success();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -10107,6 +12366,15 @@
           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;
     }
 
@@ -10119,6 +12387,11 @@
       if (present_success)
         builder.append(success);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -10140,6 +12413,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -10164,6 +12447,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -10180,6 +12471,10 @@
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         oprot.writeString(this.success);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -10197,6 +12492,14 @@
         sb.append(this.success);
       }
       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();
     }
@@ -10429,12 +12732,15 @@
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterInfo_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 ClusterSummary 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");
+      SUCCESS((short)0, "success"),
+      AZE((short)1, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -10451,6 +12757,8 @@
         switch(fieldId) {
           case 0: // SUCCESS
             return SUCCESS;
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -10497,6 +12805,8 @@
       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, ClusterSummary.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(getClusterInfo_result.class, metaDataMap);
     }
@@ -10505,10 +12815,12 @@
     }
 
     public getClusterInfo_result(
-      ClusterSummary success)
+      ClusterSummary success,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
+      this.aze = aze;
     }
 
     /**
@@ -10518,6 +12830,9 @@
       if (other.is_set_success()) {
         this.success = new ClusterSummary(other.success);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public getClusterInfo_result deepCopy() {
@@ -10527,6 +12842,7 @@
     @Override
     public void clear() {
       this.success = null;
+      this.aze = null;
     }
 
     public ClusterSummary get_success() {
@@ -10552,6 +12868,29 @@
       }
     }
 
+    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:
@@ -10562,6 +12901,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -10570,6 +12917,9 @@
       case SUCCESS:
         return get_success();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -10583,6 +12933,8 @@
       switch (field) {
       case SUCCESS:
         return is_set_success();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -10609,6 +12961,15 @@
           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;
     }
 
@@ -10621,6 +12982,11 @@
       if (present_success)
         builder.append(success);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -10642,6 +13008,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -10667,6 +13043,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 1: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -10683,6 +13067,10 @@
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         this.success.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -10700,6 +13088,14 @@
         sb.append(this.success);
       }
       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();
     }
@@ -11032,14 +13428,17 @@
 
     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 TopologyInfo 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");
+      E((short)1, "e"),
+      AZE((short)2, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -11058,6 +13457,8 @@
             return SUCCESS;
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -11106,6 +13507,8 @@
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyInfo.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(getTopologyInfo_result.class, metaDataMap);
     }
@@ -11115,11 +13518,13 @@
 
     public getTopologyInfo_result(
       TopologyInfo success,
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -11132,6 +13537,9 @@
       if (other.is_set_e()) {
         this.e = new NotAliveException(other.e);
       }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
     }
 
     public getTopologyInfo_result deepCopy() {
@@ -11142,6 +13550,7 @@
     public void clear() {
       this.success = null;
       this.e = null;
+      this.aze = null;
     }
 
     public TopologyInfo get_success() {
@@ -11190,6 +13599,29 @@
       }
     }
 
+    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:
@@ -11208,6 +13640,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -11219,6 +13659,9 @@
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -11234,6 +13677,8 @@
         return is_set_success();
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -11269,6 +13714,15 @@
           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;
     }
 
@@ -11286,6 +13740,11 @@
       if (present_e)
         builder.append(e);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -11317,6 +13776,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -11350,6 +13819,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 2: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -11370,6 +13847,10 @@
         oprot.writeFieldBegin(E_FIELD_DESC);
         this.e.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -11395,6 +13876,14 @@
         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();
     }
@@ -11727,14 +14216,17 @@
 
     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 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");
+      E((short)1, "e"),
+      AZE((short)2, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -11753,6 +14245,8 @@
             return SUCCESS;
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -11801,6 +14295,8 @@
           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);
     }
@@ -11810,11 +14306,13 @@
 
     public getTopologyConf_result(
       String success,
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -11827,6 +14325,9 @@
       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() {
@@ -11837,6 +14338,7 @@
     public void clear() {
       this.success = null;
       this.e = null;
+      this.aze = null;
     }
 
     public String get_success() {
@@ -11885,6 +14387,29 @@
       }
     }
 
+    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:
@@ -11903,6 +14428,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -11914,6 +14447,9 @@
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -11929,6 +14465,8 @@
         return is_set_success();
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -11964,6 +14502,15 @@
           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;
     }
 
@@ -11981,6 +14528,11 @@
       if (present_e)
         builder.append(e);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -12012,6 +14564,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -12044,6 +14606,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 2: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -12064,6 +14634,10 @@
         oprot.writeFieldBegin(E_FIELD_DESC);
         this.e.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -12089,6 +14663,14 @@
         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();
     }
@@ -12421,14 +15003,17 @@
 
     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 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");
+      E((short)1, "e"),
+      AZE((short)2, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -12447,6 +15032,8 @@
             return SUCCESS;
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -12495,6 +15082,8 @@
           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);
     }
@@ -12504,11 +15093,13 @@
 
     public getTopology_result(
       StormTopology success,
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -12521,6 +15112,9 @@
       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() {
@@ -12531,6 +15125,7 @@
     public void clear() {
       this.success = null;
       this.e = null;
+      this.aze = null;
     }
 
     public StormTopology get_success() {
@@ -12579,6 +15174,29 @@
       }
     }
 
+    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:
@@ -12597,6 +15215,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -12608,6 +15234,9 @@
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -12623,6 +15252,8 @@
         return is_set_success();
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -12658,6 +15289,15 @@
           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;
     }
 
@@ -12675,6 +15315,11 @@
       if (present_e)
         builder.append(e);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -12706,6 +15351,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -12739,6 +15394,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 2: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -12759,6 +15422,10 @@
         oprot.writeFieldBegin(E_FIELD_DESC);
         this.e.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -12784,6 +15451,14 @@
         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();
     }
@@ -13116,14 +15791,17 @@
 
     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 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");
+      E((short)1, "e"),
+      AZE((short)2, "aze");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -13142,6 +15820,8 @@
             return SUCCESS;
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -13190,6 +15870,8 @@
           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);
     }
@@ -13199,11 +15881,13 @@
 
     public getUserTopology_result(
       StormTopology success,
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -13216,6 +15900,9 @@
       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() {
@@ -13226,6 +15913,7 @@
     public void clear() {
       this.success = null;
       this.e = null;
+      this.aze = null;
     }
 
     public StormTopology get_success() {
@@ -13274,6 +15962,29 @@
       }
     }
 
+    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:
@@ -13292,6 +16003,14 @@
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -13303,6 +16022,9 @@
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -13318,6 +16040,8 @@
         return is_set_success();
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -13353,6 +16077,15 @@
           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;
     }
 
@@ -13370,6 +16103,11 @@
       if (present_e)
         builder.append(e);
 
+      boolean present_aze = true && (is_set_aze());
+      builder.append(present_aze);
+      if (present_aze)
+        builder.append(aze);
+
       return builder.toHashCode();
     }
 
@@ -13401,6 +16139,16 @@
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -13434,6 +16182,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
+          case 2: // AZE
+            if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+              this.aze = new AuthorizationException();
+              this.aze.read(iprot);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
         }
@@ -13454,6 +16210,10 @@
         oprot.writeFieldBegin(E_FIELD_DESC);
         this.e.write(oprot);
         oprot.writeFieldEnd();
+      } else if (this.is_set_aze()) {
+        oprot.writeFieldBegin(AZE_FIELD_DESC);
+        this.aze.write(oprot);
+        oprot.writeFieldEnd();
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -13479,6 +16239,14 @@
         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();
     }
diff --git a/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java b/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
index 78d66c9..f885f69 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
@@ -42,8 +42,10 @@
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SubmitOptions");
 
   private static final org.apache.thrift.protocol.TField INITIAL_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("initial_status", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField CREDS_FIELD_DESC = new org.apache.thrift.protocol.TField("creds", org.apache.thrift.protocol.TType.STRUCT, (short)2);
 
   private TopologyInitialStatus initial_status; // required
+  private Credentials creds; // 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 {
@@ -51,7 +53,8 @@
      * 
      * @see TopologyInitialStatus
      */
-    INITIAL_STATUS((short)1, "initial_status");
+    INITIAL_STATUS((short)1, "initial_status"),
+    CREDS((short)2, "creds");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -68,6 +71,8 @@
       switch(fieldId) {
         case 1: // INITIAL_STATUS
           return INITIAL_STATUS;
+        case 2: // CREDS
+          return CREDS;
         default:
           return null;
       }
@@ -114,6 +119,8 @@
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.INITIAL_STATUS, new org.apache.thrift.meta_data.FieldMetaData("initial_status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyInitialStatus.class)));
+    tmpMap.put(_Fields.CREDS, new org.apache.thrift.meta_data.FieldMetaData("creds", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Credentials.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SubmitOptions.class, metaDataMap);
   }
@@ -135,6 +142,9 @@
     if (other.is_set_initial_status()) {
       this.initial_status = other.initial_status;
     }
+    if (other.is_set_creds()) {
+      this.creds = new Credentials(other.creds);
+    }
   }
 
   public SubmitOptions deepCopy() {
@@ -144,6 +154,7 @@
   @Override
   public void clear() {
     this.initial_status = null;
+    this.creds = null;
   }
 
   /**
@@ -177,6 +188,29 @@
     }
   }
 
+  public Credentials get_creds() {
+    return this.creds;
+  }
+
+  public void set_creds(Credentials creds) {
+    this.creds = creds;
+  }
+
+  public void unset_creds() {
+    this.creds = null;
+  }
+
+  /** Returns true if field creds is set (has been assigned a value) and false otherwise */
+  public boolean is_set_creds() {
+    return this.creds != null;
+  }
+
+  public void set_creds_isSet(boolean value) {
+    if (!value) {
+      this.creds = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case INITIAL_STATUS:
@@ -187,6 +221,14 @@
       }
       break;
 
+    case CREDS:
+      if (value == null) {
+        unset_creds();
+      } else {
+        set_creds((Credentials)value);
+      }
+      break;
+
     }
   }
 
@@ -195,6 +237,9 @@
     case INITIAL_STATUS:
       return get_initial_status();
 
+    case CREDS:
+      return get_creds();
+
     }
     throw new IllegalStateException();
   }
@@ -208,6 +253,8 @@
     switch (field) {
     case INITIAL_STATUS:
       return is_set_initial_status();
+    case CREDS:
+      return is_set_creds();
     }
     throw new IllegalStateException();
   }
@@ -234,6 +281,15 @@
         return false;
     }
 
+    boolean this_present_creds = true && this.is_set_creds();
+    boolean that_present_creds = true && that.is_set_creds();
+    if (this_present_creds || that_present_creds) {
+      if (!(this_present_creds && that_present_creds))
+        return false;
+      if (!this.creds.equals(that.creds))
+        return false;
+    }
+
     return true;
   }
 
@@ -246,6 +302,11 @@
     if (present_initial_status)
       builder.append(initial_status.getValue());
 
+    boolean present_creds = true && (is_set_creds());
+    builder.append(present_creds);
+    if (present_creds)
+      builder.append(creds);
+
     return builder.toHashCode();
   }
 
@@ -267,6 +328,16 @@
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_creds()).compareTo(typedOther.is_set_creds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_creds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.creds, typedOther.creds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -291,6 +362,14 @@
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;
+        case 2: // CREDS
+          if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
+            this.creds = new Credentials();
+            this.creds.read(iprot);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
         default:
           org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
       }
@@ -309,6 +388,13 @@
       oprot.writeI32(this.initial_status.getValue());
       oprot.writeFieldEnd();
     }
+    if (this.creds != null) {
+      if (is_set_creds()) {
+        oprot.writeFieldBegin(CREDS_FIELD_DESC);
+        this.creds.write(oprot);
+        oprot.writeFieldEnd();
+      }
+    }
     oprot.writeFieldStop();
     oprot.writeStructEnd();
   }
@@ -325,6 +411,16 @@
       sb.append(this.initial_status);
     }
     first = false;
+    if (is_set_creds()) {
+      if (!first) sb.append(", ");
+      sb.append("creds:");
+      if (this.creds == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.creds);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index 2ec9cb3..2c36d4e 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -47,6 +47,8 @@
   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)4);
   private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRING, (short)5);
   private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.MAP, (short)6);
+  private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)513);
+  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)514);
 
   private String id; // required
   private String name; // required
@@ -54,6 +56,8 @@
   private List<ExecutorSummary> executors; // required
   private String status; // required
   private Map<String,List<ErrorInfo>> errors; // required
+  private String sched_status; // 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 {
@@ -62,7 +66,9 @@
     UPTIME_SECS((short)3, "uptime_secs"),
     EXECUTORS((short)4, "executors"),
     STATUS((short)5, "status"),
-    ERRORS((short)6, "errors");
+    ERRORS((short)6, "errors"),
+    SCHED_STATUS((short)513, "sched_status"),
+    OWNER((short)514, "owner");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -89,6 +95,10 @@
           return STATUS;
         case 6: // ERRORS
           return ERRORS;
+        case 513: // SCHED_STATUS
+          return SCHED_STATUS;
+        case 514: // OWNER
+          return OWNER;
         default:
           return null;
       }
@@ -151,6 +161,10 @@
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             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, ErrorInfo.class)))));
+    tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap);
   }
@@ -217,6 +231,12 @@
       }
       this.errors = __this__errors;
     }
+    if (other.is_set_sched_status()) {
+      this.sched_status = other.sched_status;
+    }
+    if (other.is_set_owner()) {
+      this.owner = other.owner;
+    }
   }
 
   public TopologyInfo deepCopy() {
@@ -232,6 +252,8 @@
     this.executors = null;
     this.status = null;
     this.errors = null;
+    this.sched_status = null;
+    this.owner = null;
   }
 
   public String get_id() {
@@ -397,6 +419,52 @@
     }
   }
 
+  public String get_sched_status() {
+    return this.sched_status;
+  }
+
+  public void set_sched_status(String sched_status) {
+    this.sched_status = sched_status;
+  }
+
+  public void unset_sched_status() {
+    this.sched_status = null;
+  }
+
+  /** Returns true if field sched_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_sched_status() {
+    return this.sched_status != null;
+  }
+
+  public void set_sched_status_isSet(boolean value) {
+    if (!value) {
+      this.sched_status = null;
+    }
+  }
+
+  public String get_owner() {
+    return this.owner;
+  }
+
+  public void set_owner(String owner) {
+    this.owner = owner;
+  }
+
+  public void unset_owner() {
+    this.owner = 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_owner_isSet(boolean value) {
+    if (!value) {
+      this.owner = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ID:
@@ -447,6 +515,22 @@
       }
       break;
 
+    case SCHED_STATUS:
+      if (value == null) {
+        unset_sched_status();
+      } else {
+        set_sched_status((String)value);
+      }
+      break;
+
+    case OWNER:
+      if (value == null) {
+        unset_owner();
+      } else {
+        set_owner((String)value);
+      }
+      break;
+
     }
   }
 
@@ -470,6 +554,12 @@
     case ERRORS:
       return get_errors();
 
+    case SCHED_STATUS:
+      return get_sched_status();
+
+    case OWNER:
+      return get_owner();
+
     }
     throw new IllegalStateException();
   }
@@ -493,6 +583,10 @@
       return is_set_status();
     case ERRORS:
       return is_set_errors();
+    case SCHED_STATUS:
+      return is_set_sched_status();
+    case OWNER:
+      return is_set_owner();
     }
     throw new IllegalStateException();
   }
@@ -564,6 +658,24 @@
         return false;
     }
 
+    boolean this_present_sched_status = true && this.is_set_sched_status();
+    boolean that_present_sched_status = true && that.is_set_sched_status();
+    if (this_present_sched_status || that_present_sched_status) {
+      if (!(this_present_sched_status && that_present_sched_status))
+        return false;
+      if (!this.sched_status.equals(that.sched_status))
+        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))
+        return false;
+      if (!this.owner.equals(that.owner))
+        return false;
+    }
+
     return true;
   }
 
@@ -601,6 +713,16 @@
     if (present_errors)
       builder.append(errors);
 
+    boolean present_sched_status = true && (is_set_sched_status());
+    builder.append(present_sched_status);
+    if (present_sched_status)
+      builder.append(sched_status);
+
+    boolean present_owner = true && (is_set_owner());
+    builder.append(present_owner);
+    if (present_owner)
+      builder.append(owner);
+
     return builder.toHashCode();
   }
 
@@ -672,6 +794,26 @@
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(typedOther.is_set_sched_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_sched_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sched_status, typedOther.sched_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(typedOther.is_set_owner());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_owner()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, typedOther.owner);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -766,6 +908,20 @@
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;
+        case 513: // SCHED_STATUS
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.sched_status = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 514: // OWNER
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.owner = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
         default:
           org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
       }
@@ -829,6 +985,20 @@
       }
       oprot.writeFieldEnd();
     }
+    if (this.sched_status != null) {
+      if (is_set_sched_status()) {
+        oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
+        oprot.writeString(this.sched_status);
+        oprot.writeFieldEnd();
+      }
+    }
+    if (this.owner != null) {
+      if (is_set_owner()) {
+        oprot.writeFieldBegin(OWNER_FIELD_DESC);
+        oprot.writeString(this.owner);
+        oprot.writeFieldEnd();
+      }
+    }
     oprot.writeFieldStop();
     oprot.writeStructEnd();
   }
@@ -881,6 +1051,26 @@
       sb.append(this.errors);
     }
     first = false;
+    if (is_set_sched_status()) {
+      if (!first) sb.append(", ");
+      sb.append("sched_status:");
+      if (this.sched_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sched_status);
+      }
+      first = false;
+    }
+    if (is_set_owner()) {
+      if (!first) sb.append(", ");
+      sb.append("owner:");
+      if (this.owner == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.owner);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
index 97ae6d7..fea2137 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
@@ -48,6 +48,8 @@
   private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)5);
   private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)6);
   private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)513);
+  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)514);
 
   private String id; // required
   private String name; // required
@@ -56,6 +58,8 @@
   private int num_workers; // required
   private int uptime_secs; // required
   private String status; // required
+  private String sched_status; // 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 {
@@ -65,7 +69,9 @@
     NUM_EXECUTORS((short)4, "num_executors"),
     NUM_WORKERS((short)5, "num_workers"),
     UPTIME_SECS((short)6, "uptime_secs"),
-    STATUS((short)7, "status");
+    STATUS((short)7, "status"),
+    SCHED_STATUS((short)513, "sched_status"),
+    OWNER((short)514, "owner");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -94,6 +100,10 @@
           return UPTIME_SECS;
         case 7: // STATUS
           return STATUS;
+        case 513: // SCHED_STATUS
+          return SCHED_STATUS;
+        case 514: // OWNER
+          return OWNER;
         default:
           return null;
       }
@@ -157,6 +167,10 @@
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap);
   }
@@ -206,6 +220,12 @@
     if (other.is_set_status()) {
       this.status = other.status;
     }
+    if (other.is_set_sched_status()) {
+      this.sched_status = other.sched_status;
+    }
+    if (other.is_set_owner()) {
+      this.owner = other.owner;
+    }
   }
 
   public TopologySummary deepCopy() {
@@ -225,6 +245,8 @@
     set_uptime_secs_isSet(false);
     this.uptime_secs = 0;
     this.status = null;
+    this.sched_status = null;
+    this.owner = null;
   }
 
   public String get_id() {
@@ -384,6 +406,52 @@
     }
   }
 
+  public String get_sched_status() {
+    return this.sched_status;
+  }
+
+  public void set_sched_status(String sched_status) {
+    this.sched_status = sched_status;
+  }
+
+  public void unset_sched_status() {
+    this.sched_status = null;
+  }
+
+  /** Returns true if field sched_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_sched_status() {
+    return this.sched_status != null;
+  }
+
+  public void set_sched_status_isSet(boolean value) {
+    if (!value) {
+      this.sched_status = null;
+    }
+  }
+
+  public String get_owner() {
+    return this.owner;
+  }
+
+  public void set_owner(String owner) {
+    this.owner = owner;
+  }
+
+  public void unset_owner() {
+    this.owner = 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_owner_isSet(boolean value) {
+    if (!value) {
+      this.owner = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ID:
@@ -442,6 +510,22 @@
       }
       break;
 
+    case SCHED_STATUS:
+      if (value == null) {
+        unset_sched_status();
+      } else {
+        set_sched_status((String)value);
+      }
+      break;
+
+    case OWNER:
+      if (value == null) {
+        unset_owner();
+      } else {
+        set_owner((String)value);
+      }
+      break;
+
     }
   }
 
@@ -468,6 +552,12 @@
     case STATUS:
       return get_status();
 
+    case SCHED_STATUS:
+      return get_sched_status();
+
+    case OWNER:
+      return get_owner();
+
     }
     throw new IllegalStateException();
   }
@@ -493,6 +583,10 @@
       return is_set_uptime_secs();
     case STATUS:
       return is_set_status();
+    case SCHED_STATUS:
+      return is_set_sched_status();
+    case OWNER:
+      return is_set_owner();
     }
     throw new IllegalStateException();
   }
@@ -573,6 +667,24 @@
         return false;
     }
 
+    boolean this_present_sched_status = true && this.is_set_sched_status();
+    boolean that_present_sched_status = true && that.is_set_sched_status();
+    if (this_present_sched_status || that_present_sched_status) {
+      if (!(this_present_sched_status && that_present_sched_status))
+        return false;
+      if (!this.sched_status.equals(that.sched_status))
+        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))
+        return false;
+      if (!this.owner.equals(that.owner))
+        return false;
+    }
+
     return true;
   }
 
@@ -615,6 +727,16 @@
     if (present_status)
       builder.append(status);
 
+    boolean present_sched_status = true && (is_set_sched_status());
+    builder.append(present_sched_status);
+    if (present_sched_status)
+      builder.append(sched_status);
+
+    boolean present_owner = true && (is_set_owner());
+    builder.append(present_owner);
+    if (present_owner)
+      builder.append(owner);
+
     return builder.toHashCode();
   }
 
@@ -696,6 +818,26 @@
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(typedOther.is_set_sched_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_sched_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sched_status, typedOther.sched_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(typedOther.is_set_owner());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_owner()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, typedOther.owner);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -766,6 +908,20 @@
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;
+        case 513: // SCHED_STATUS
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.sched_status = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 514: // OWNER
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.owner = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
         default:
           org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
       }
@@ -806,6 +962,20 @@
       oprot.writeString(this.status);
       oprot.writeFieldEnd();
     }
+    if (this.sched_status != null) {
+      if (is_set_sched_status()) {
+        oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
+        oprot.writeString(this.sched_status);
+        oprot.writeFieldEnd();
+      }
+    }
+    if (this.owner != null) {
+      if (is_set_owner()) {
+        oprot.writeFieldBegin(OWNER_FIELD_DESC);
+        oprot.writeString(this.owner);
+        oprot.writeFieldEnd();
+      }
+    }
     oprot.writeFieldStop();
     oprot.writeStructEnd();
   }
@@ -854,6 +1024,26 @@
       sb.append(this.status);
     }
     first = false;
+    if (is_set_sched_status()) {
+      if (!first) sb.append(", ");
+      sb.append("sched_status:");
+      if (this.sched_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sched_status);
+      }
+      first = false;
+    }
+    if (is_set_owner()) {
+      if (!first) sb.append(", ");
+      sb.append("owner:");
+      if (this.owner == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.owner);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index 5e0b5af..e0c7cc7 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -35,6 +35,10 @@
      * key: topologyId, value: topology's current assignments.
      */
     private Map<String, SchedulerAssignmentImpl> assignments;
+    /**
+     * key topologyId, Value: scheduler's status.
+     */  
+    private Map<String, String> status;
 
     /**
      * a map from hostname to supervisor id.
@@ -50,6 +54,7 @@
         this.supervisors.putAll(supervisors);
         this.assignments = new HashMap<String, SchedulerAssignmentImpl>(assignments.size());
         this.assignments.putAll(assignments);
+        this.status = new HashMap<String, String>();
         this.hostToId = new HashMap<String, List<String>>();
         for (String nodeId : supervisors.keySet()) {
             SupervisorDetails supervisor = supervisors.get(nodeId);
@@ -432,4 +437,12 @@
     public Map<String, SupervisorDetails> getSupervisors() {
         return this.supervisors;
     }
+
+    public void setStatus(String topologyId, String status) {
+        this.status.put(topologyId, status);
+    }
+
+    public Map<String, String> getStatusMap() {
+        return this.status;
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java
new file mode 100644
index 0000000..3053b5b
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java
@@ -0,0 +1,219 @@
+/**
+ * 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 backtype.storm.scheduler.multitenant;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.scheduler.SchedulerAssignment;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+
+/**
+ * A pool of machines that anyone can use, but topologies are not isolated
+ */
+public class DefaultPool extends NodePool {
+  private static final Logger LOG = LoggerFactory.getLogger(DefaultPool.class);
+  private Set<Node> _nodes = new HashSet<Node>();
+  private HashMap<String, TopologyDetails> _tds = new HashMap<String, TopologyDetails>();
+  
+  @Override
+  public void addTopology(TopologyDetails td) {
+    String topId = td.getId();
+    LOG.debug("Adding in Topology {}", topId);
+    _tds.put(topId, td);
+    SchedulerAssignment assignment = _cluster.getAssignmentById(topId);
+    if (assignment != null) {
+      for (WorkerSlot ws: assignment.getSlots()) {
+        Node n = _nodeIdToNode.get(ws.getNodeId());
+        _nodes.add(n);
+      }
+    }
+  }
+
+  @Override
+  public boolean canAdd(TopologyDetails td) {
+    return true;
+  }
+
+  @Override
+  public Collection<Node> takeNodes(int nodesNeeded) {
+    HashSet<Node> ret = new HashSet<Node>();
+    LinkedList<Node> sortedNodes = new LinkedList<Node>(_nodes);
+    Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC);
+    for (Node n: sortedNodes) {
+      if (nodesNeeded <= ret.size()) {
+        break;
+      }
+      if (n.isAlive()) {
+        n.freeAllSlots(_cluster);
+        _nodes.remove(n);
+        ret.add(n);
+      }
+    }
+    return ret;
+  }
+  
+  @Override
+  public int nodesAvailable() {
+    int total = 0;
+    for (Node n: _nodes) {
+      if (n.isAlive()) total++;
+    }
+    return total;
+  }
+  
+  @Override
+  public int slotsAvailable() {
+    return Node.countTotalSlotsAlive(_nodes);
+  }
+
+  @Override
+  public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) {
+    int nodesFound = 0;
+    int slotsFound = 0;
+    LinkedList<Node> sortedNodes = new LinkedList<Node>(_nodes);
+    Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC);
+    for (Node n: sortedNodes) {
+      if (slotsNeeded <= 0) {
+        break;
+      }
+      if (n.isAlive()) {
+        nodesFound++;
+        int totalSlotsFree = n.totalSlots();
+        slotsFound += totalSlotsFree;
+        slotsNeeded -= totalSlotsFree;
+      }
+    }
+    return new NodeAndSlotCounts(nodesFound, slotsFound);
+  }
+  
+  @Override
+  public Collection<Node> takeNodesBySlots(int slotsNeeded) {
+    HashSet<Node> ret = new HashSet<Node>();
+    LinkedList<Node> sortedNodes = new LinkedList<Node>(_nodes);
+    Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC);
+    for (Node n: sortedNodes) {
+      if (slotsNeeded <= 0) {
+        break;
+      }
+      if (n.isAlive()) {
+        n.freeAllSlots(_cluster);
+        _nodes.remove(n);
+        ret.add(n);
+        slotsNeeded -= n.totalSlotsFree();
+      }
+    }
+    return ret;
+  }
+
+  @Override
+  public void scheduleAsNeeded(NodePool... lesserPools) {
+    for (TopologyDetails td : _tds.values()) {
+      String topId = td.getId();
+      if (_cluster.needsScheduling(td)) {
+        LOG.debug("Scheduling topology {}",topId);
+        int totalTasks = td.getExecutors().size();
+        int origRequest = td.getNumWorkers();
+        int slotsRequested = Math.min(totalTasks, origRequest);
+        int slotsUsed = Node.countSlotsUsed(topId, _nodes);
+        int slotsFree = Node.countFreeSlotsAlive(_nodes);
+        //Check to see if we have enough slots before trying to get them
+        int slotsAvailable = 0;
+        if (slotsRequested > slotsFree) {
+          slotsAvailable = NodePool.slotsAvailable(lesserPools);
+        }
+        int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree + slotsAvailable);
+        int executorsNotRunning = _cluster.getUnassignedExecutors(td).size();
+        LOG.debug("Slots... requested {} used {} free {} available {} to be used {}, executors not running {}", 
+            new Object[] {slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse, executorsNotRunning}); 
+        if (slotsToUse <= 0) {
+          if (executorsNotRunning > 0) {
+            _cluster.setStatus(topId,"Not fully scheduled (No free slots in default pool) "+executorsNotRunning+" executors not scheduled");
+          } else {
+            if (slotsUsed < slotsRequested) {
+              _cluster.setStatus(topId,"Running with fewer slots than requested ("+slotsUsed+"/"+origRequest+")");
+            } else { //slotsUsed < origRequest
+              _cluster.setStatus(topId,"Fully Scheduled (requested "+origRequest+" slots, but could only use "+slotsUsed+")");
+            }
+          }
+          continue;
+        }
+
+        int slotsNeeded = slotsToUse - slotsFree;
+        if (slotsNeeded > 0) {
+          _nodes.addAll(NodePool.takeNodesBySlot(slotsNeeded, lesserPools));
+        }
+
+        if (executorsNotRunning <= 0) {
+          //There are free slots that we can take advantage of now.
+          for (Node n: _nodes) {
+            n.freeTopology(topId, _cluster); 
+          }
+          slotsFree = Node.countFreeSlotsAlive(_nodes);
+          slotsToUse = Math.min(slotsRequested, slotsFree);
+        }
+        
+        RoundRobinSlotScheduler slotSched = 
+          new RoundRobinSlotScheduler(td, slotsToUse, _cluster);
+        
+        LinkedList<Node> nodes = new LinkedList<Node>(_nodes);
+        while (true) {
+          Node n = null;
+          do {
+            if (nodes.isEmpty()) {
+              throw new IllegalStateException("This should not happen, we" +
+              " messed up and did not get enough slots");
+            }
+            n = nodes.peekFirst();
+            if (n.totalSlotsFree() == 0) {
+              nodes.remove();
+              n = null;
+            }
+          } while (n == null);
+          if (!slotSched.assignSlotTo(n)) {
+            break;
+          }
+        }
+        int afterSchedSlotsUsed = Node.countSlotsUsed(topId, _nodes);
+        if (afterSchedSlotsUsed < slotsRequested) {
+          _cluster.setStatus(topId,"Running with fewer slots than requested ("+afterSchedSlotsUsed+"/"+origRequest+")");
+        } else if (afterSchedSlotsUsed < origRequest) {
+          _cluster.setStatus(topId,"Fully Scheduled (requested "+origRequest+" slots, but could only use "+afterSchedSlotsUsed+")");
+        } else {
+          _cluster.setStatus(topId,"Fully Scheduled");
+        }
+      } else {
+        _cluster.setStatus(topId,"Fully Scheduled");
+      }
+    }
+  }
+  
+  @Override
+  public String toString() {
+    return "DefaultPool  " + _nodes.size() + " nodes " + _tds.size() + " topologies";
+  }
+}
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java
new file mode 100644
index 0000000..c625895
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java
@@ -0,0 +1,125 @@
+/**
+ * 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 backtype.storm.scheduler.multitenant;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.TopologyDetails;
+
+/**
+ * All of the machines that currently have nothing assigned to them
+ */
+public class FreePool extends NodePool {
+  private static final Logger LOG = LoggerFactory.getLogger(FreePool.class);
+  private Set<Node> _nodes = new HashSet<Node>();
+  private int _totalSlots = 0;
+
+  @Override
+  public void init(Cluster cluster, Map<String, Node> nodeIdToNode) {
+    super.init(cluster, nodeIdToNode);
+    for (Node n: nodeIdToNode.values()) {
+      if(n.isTotallyFree() && n.isAlive()) {
+        _nodes.add(n);
+        _totalSlots += n.totalSlotsFree();
+      }
+    }
+    LOG.debug("Found {} nodes with {} slots", _nodes.size(), _totalSlots);
+  }
+  
+  @Override
+  public void addTopology(TopologyDetails td) {
+    throw new IllegalArgumentException("The free pool cannot run any topologies");
+  }
+
+  @Override
+  public boolean canAdd(TopologyDetails td) {
+    // The free pool never has anything running
+    return false;
+  }
+  
+  @Override
+  public Collection<Node> takeNodes(int nodesNeeded) {
+    HashSet<Node> ret = new HashSet<Node>();
+    Iterator<Node> it = _nodes.iterator();
+    while (it.hasNext() && nodesNeeded > ret.size()) {
+      Node n = it.next();
+      ret.add(n);
+      _totalSlots -= n.totalSlotsFree();
+      it.remove();
+    }
+    return ret;
+  }
+  
+  @Override
+  public int nodesAvailable() {
+    return _nodes.size();
+  }
+
+  @Override
+  public int slotsAvailable() {
+    return _totalSlots;
+  }
+
+  @Override
+  public Collection<Node> takeNodesBySlots(int slotsNeeded) {
+    HashSet<Node> ret = new HashSet<Node>();
+    Iterator<Node> it = _nodes.iterator();
+    while (it.hasNext() && slotsNeeded > 0) {
+      Node n = it.next();
+      ret.add(n);
+      _totalSlots -= n.totalSlotsFree();
+      slotsNeeded -= n.totalSlotsFree();